This is being done ahead of removing malloc calls in the rbtree.
talloc_set_log_stderr();
- filename_tree = rbtree_talloc_alloc(NULL, filename_cmp, rc_file_pair_t, NULL, 0);
+ filename_tree = rbtree_talloc_alloc(NULL, rc_file_pair_t, node, filename_cmp, NULL, 0);
if (!filename_tree) {
oom:
ERROR("Out of memory");
} rc_stats_t;
typedef struct {
+ fr_rb_node_t node; //!< rbtree node data.
char const *packets; //!< The file containing the request packet
char const *filters; //!< The file containing the definition of the
//!< packet we want to match.
usage(64);
}
- link_tree = rbtree_talloc_alloc(conf, (rbcmp) rs_rtx_cmp, rs_request_t, _unmark_link, 0);
+ link_tree = rbtree_talloc_alloc(conf, rs_request_t, link_node, (rbcmp) rs_rtx_cmp, _unmark_link, 0);
if (!link_tree) {
ERROR("Failed creating RTX tree");
goto finish;
/*
* Setup the request tree
*/
- request_tree = rbtree_talloc_alloc(conf, (rbcmp) rs_packet_cmp, rs_request_t, _unmark_request, 0);
+ request_tree = rbtree_talloc_alloc(conf, rs_request_t, request_node, (rbcmp) rs_packet_cmp, _unmark_request, 0);
if (!request_tree) {
ERROR("Failed creating request tree");
goto finish;
//!< ignore stats about packet loss.
-
+ fr_rb_node_t request_node;
+ fr_rb_node_t link_node;
bool in_request_tree; //!< Whether the request is currently in the request tree.
bool in_link_tree; //!< Whether the request is currently in the linked tree.
} rs_request_t;
*/
typedef struct fr_listen fr_listen_t;
struct fr_listen {
+ fr_rb_node_t virtual_server_node; //!< Entry into the virtual server's tree of listeners.
+
int fd; //!< file descriptor for this socket - set by open
char const *name; //!< printable name for this socket - set by open
void const *app_io_instance; //!< I/O path configuration context.
void *thread_instance; //!< thread / socket context
- fr_socket_t *app_io_addr; //!< for tracking duplicate sockets
+ fr_socket_t *app_io_addr; //!< for tracking duplicate sockets
fr_app_t const *app;
void const *app_instance;
* #todo - unify the code with static clients?
*/
if (inst->app_io->track_duplicates) {
- MEM(connection->client->table = rbtree_talloc_alloc(client, track_connected_cmp, fr_io_track_t,
- NULL, RBTREE_FLAG_NONE));
+ MEM(connection->client->table = rbtree_talloc_alloc(client, fr_io_track_t, node,
+ track_connected_cmp, NULL, RBTREE_FLAG_NONE));
}
/*
*/
if (inst->app_io->track_duplicates) {
fr_assert(inst->app_io->compare != NULL);
- MEM(client->table = rbtree_talloc_alloc(client, track_cmp, fr_io_track_t,
+ MEM(client->table = rbtree_talloc_alloc(client, fr_io_track_t, node, track_cmp,
NULL, RBTREE_FLAG_NONE));
}
typedef struct fr_io_client_s fr_io_client_t;
typedef struct {
+ fr_rb_node_t node; //!< rbtree node in the tracking tree.
fr_event_timer_t const *ev; //!< when we clean up this tracking entry
fr_time_t timestamp; //!< when this packet was received
fr_time_t expires; //!< when this packet expires
} fr_network_worker_t;
typedef struct {
+ fr_rb_node_t listen_node; //!< rbtree node for looking up by listener.
+ fr_rb_node_t num_node; //!< rbtree node for looking up by number.
+
fr_network_t *nr; //!< O(N) issues in talloc
int number; //!< unique ID
int heap_id; //!< for the sockets_by_num heap
/*
* Create the various heaps.
*/
- nr->sockets = rbtree_talloc_alloc(nr, socket_listen_cmp, fr_network_socket_t, NULL, RBTREE_FLAG_NONE);
+ nr->sockets = rbtree_talloc_alloc(nr, fr_network_socket_t, listen_node, socket_listen_cmp, NULL, RBTREE_FLAG_NONE);
if (!nr->sockets) {
fr_strerror_const_push("Failed creating listen tree for sockets");
goto fail2;
}
- nr->sockets_by_num = rbtree_talloc_alloc(nr, socket_num_cmp, fr_network_socket_t, NULL, RBTREE_FLAG_NONE);
+ nr->sockets_by_num = rbtree_talloc_alloc(nr, fr_network_socket_t, num_node, socket_num_cmp, NULL, RBTREE_FLAG_NONE);
if (!nr->sockets_by_num) {
fr_strerror_const_push("Failed creating number tree for sockets");
goto fail2;
goto fail;
}
- worker->dedup = rbtree_talloc_alloc(worker, worker_dedup_cmp, request_t, NULL, RBTREE_FLAG_NONE);
+ worker->dedup = rbtree_talloc_alloc(worker, request_t, dedup_node, worker_dedup_cmp, NULL, RBTREE_FLAG_NONE);
if (!worker->dedup) {
fr_strerror_const("Failed creating de_dup tree");
goto fail;
* Passed as opaque data to pools which open connection to nodes.
*/
struct fr_redis_cluster_node_s {
+ fr_rb_node_t rbnode; //!< Entry into the tree of redis nodes.
+
char name[INET6_ADDRSTRLEN]; //!< Buffer to hold IP string.
//!< text for debug messages.
uint8_t id; //!< Node ID (index in node array).
cluster->node = talloc_zero_array(cluster, fr_redis_cluster_node_t, conf->max_nodes + 1);
if (!cluster->node) goto oom;
- cluster->used_nodes = rbtree_alloc(cluster, _cluster_node_cmp, NULL, 0);
+ cluster->used_nodes = rbtree_alloc(cluster, fr_redis_cluster_node_t, rbnode, _cluster_node_cmp, NULL, 0);
if (!cluster->used_nodes) goto oom;
cluster->free_nodes = fr_fifo_create(cluster, conf->max_nodes, NULL);
cf_item_add(cs, &(cp->item));
- MEM(tree = rbtree_talloc_alloc(cs, _inode_cmp, cf_file_t, NULL, 0));
+ MEM(tree = rbtree_talloc_alloc(cs, cf_file_t, node, _inode_cmp, NULL, 0));
cf_data_add(cs, tree, "filename", false);
*
*/
struct cf_item {
+ fr_rb_node_t ident1_node; //!< Entry in the ident1 tree.
+ fr_rb_node_t ident2_node; //!< Entry in the ident2 tree.
+
CONF_ITEM_TYPE type; //!< Whether the config item is a config_pair, conf_section or cf_data.
CONF_ITEM *next; //!< Sibling.
};
typedef struct {
+ fr_rb_node_t node;
char const *filename; //!< name of the file
CONF_SECTION *cs; //!< CONF_SECTION associated with the file
struct stat buf; //!< stat about the file
/*
* New child, add child trees.
*/
- if (!parent->ident1) parent->ident1 = rbtree_alloc(parent, _cf_ident1_cmp, NULL, RBTREE_FLAG_NONE);
- if (!parent->ident2) parent->ident2 = rbtree_alloc(parent, _cf_ident2_cmp, NULL, RBTREE_FLAG_NONE);
+ if (!parent->ident1) parent->ident1 = rbtree_alloc(parent, CONF_ITEM, ident1_node,
+ _cf_ident1_cmp, NULL, RBTREE_FLAG_NONE);
+ if (!parent->ident2) parent->ident2 = rbtree_alloc(parent, CONF_ITEM, ident2_node,
+ _cf_ident2_cmp, NULL, RBTREE_FLAG_NONE);
fr_cursor_init(&to_merge, &child);
#else /* WITH_TRIE */
if (!clients->tree[client->ipaddr.prefix]) {
- clients->tree[client->ipaddr.prefix] = rbtree_talloc_alloc(clients, client_cmp, RADCLIENT,
- NULL, RBTREE_FLAG_NONE);
+ clients->tree[client->ipaddr.prefix] = rbtree_talloc_alloc(clients, RADCLIENT, node, client_cmp,
+ NULL, RBTREE_FLAG_NONE);
if (!clients->tree[client->ipaddr.prefix]) {
return false;
}
*
*/
struct rad_client {
+ fr_rb_node_t node; //!< Entry in the client tree.
+
fr_ipaddr_t ipaddr; //!< IPv4/IPv6 address of the host.
fr_ipaddr_t src_ipaddr; //!< IPv4/IPv6 address to send responses
//!< from (family must match ipaddr).
dl_search_path_prepend(dl_module_loader->dl_loader, lib_dir);
dl_module_loader->inst_data_tree = rbtree_talloc_alloc(dl_module_loader,
- dl_module_inst_data_cmp, dl_module_inst_t, NULL, 0);
+ dl_module_inst_t, node,
+ dl_module_inst_data_cmp, NULL, 0);
if (!dl_module_loader->inst_data_tree) {
ERROR("Failed initialising dl->inst_data_tree");
goto error;
}
dl_module_loader->module_tree = rbtree_talloc_alloc(dl_module_loader,
- dl_module_cmp, dl_module_t, NULL, 0);
+ dl_module_t, node,
+ dl_module_cmp, NULL, 0);
if (!dl_module_loader->inst_data_tree) {
ERROR("Failed initialising dl->module_tree");
goto error;
# include <dlfcn.h>
#endif
-#include <freeradius-devel/util/version.h>
-#include <freeradius-devel/util/dl.h>
#include <freeradius-devel/server/cf_parse.h>
+#include <freeradius-devel/util/dl.h>
+#include <freeradius-devel/util/rbtree.h>
+#include <freeradius-devel/util/version.h>
#ifdef __cplusplus
extern "C" {
*/
typedef struct dl_module_s dl_module_t;
struct dl_module_s {
+ fr_rb_node_t node; //!< Entry in the module tree.
+
dl_t *dl; //!< Dynamic loader handle.
dl_module_t const *parent; //!< of this module.
*/
typedef struct dl_module_instance_s dl_module_inst_t;
struct dl_module_instance_s {
+ fr_rb_node_t node; //!< Entry in the module instance tree.
char const *name; //!< Instance name.
dl_module_t const *module; //!< Module
void *data; //!< Module instance's parsed configuration.
fr_assert(name && name[0]);
if (!map_proc_root) {
- map_proc_root = rbtree_talloc_alloc(NULL, map_proc_cmp, map_proc_t,
- _map_proc_tree_free, RBTREE_FLAG_REPLACE);
+ map_proc_root = rbtree_talloc_alloc(NULL, map_proc_t, node,
+ map_proc_cmp, _map_proc_tree_free, RBTREE_FLAG_REPLACE);
if (!map_proc_root) {
DEBUG("map_proc: Failed to create tree");
return -1;
/** Map processor registration
*/
struct map_proc {
+ fr_rb_node_t node; //!< Entry in the map processor tree.
void *mod_inst; //!< Module instance.
char name[FR_MAX_STRING_LEN]; //!< Name of the map function.
int length; //!< Length of name.
int modules_init(void)
{
- MEM(module_instance_name_tree = rbtree_alloc(NULL, module_instance_name_cmp, NULL, RBTREE_FLAG_NONE));
- MEM(module_instance_data_tree = rbtree_alloc(NULL, module_instance_data_cmp, NULL, RBTREE_FLAG_NONE));
+ MEM(module_instance_name_tree = rbtree_alloc(NULL, module_instance_t, name_node,
+ module_instance_name_cmp, NULL, RBTREE_FLAG_NONE));
+ MEM(module_instance_data_tree = rbtree_alloc(NULL, module_instance_t, data_node,
+ module_instance_data_cmp, NULL, RBTREE_FLAG_NONE));
instance_ctx = talloc_init("module instance context");
return 0;
* data structures.
*/
struct module_instance_s {
+ fr_rb_node_t name_node; //!< Entry in the name tree.
+ fr_rb_node_t data_node; //!< Entry in the data tree.
+
char const *name; //!< Instance name e.g. user_database.
dl_module_inst_t *dl_inst; //!< Structure containing the module's instance data,
fr_event_timer_t const *ev; //!< Event in event loop tied to this request.
+ fr_rb_node_t dedup_node; //!< entry in the deduplication tree.
int32_t runnable_id; //!< entry in the queue / heap of runnable packets
int32_t time_order_id; //!< entry in the queue / heap of time ordered packets
*
*/
typedef struct {
- uint64_t id; //!< State number within state tree.
+ uint64_t id; //!< State number within state heap.
+ fr_rb_node_t node; //!< Entry in the state rbtree.
union {
/** Server ID components
*
* are freed before it's destroyed. Hence
* it being parented from the NULL ctx.
*/
- state->tree = rbtree_talloc_alloc(NULL, state_entry_cmp, fr_state_entry_t, NULL, 0);
+ state->tree = rbtree_talloc_alloc(NULL, fr_state_entry_t, node, state_entry_cmp, NULL, 0);
if (!state->tree) {
talloc_free(state);
return NULL;
*
*/
typedef struct {
+ fr_rb_node_t node; //!< Entry in the trigger last fired tree.
CONF_ITEM *ci; //!< Config item this rate limit counter is associated with.
time_t last_fired; //!< When this trigger last fired.
} trigger_last_fired_t;
}
MEM(trigger_last_fired_tree = rbtree_talloc_alloc(talloc_null_ctx(),
- _trigger_last_fired_cmp, trigger_last_fired_t,
- _trigger_last_fired_free, 0));
+ trigger_last_fired_t, node,
+ _trigger_last_fired_cmp, _trigger_last_fired_free, 0));
trigger_mutex = talloc(talloc_null_ctx(), pthread_mutex_t);
pthread_mutex_init(trigger_mutex, 0);
#include <talloc.h>
typedef struct pair_list {
+ fr_rb_node_t node; //!< Entry into the tree of pair lists.
+
char const *name;
map_t *check;
map_t *reply;
#include <freeradius-devel/unlang/base.h>
typedef struct {
+ fr_rb_node_t node; //!< Entry in the namespace tree.
char const *namespace; //!< Namespace function is registered to.
fr_dict_t const *dict; //!< dictionary to use
fr_virtual_server_compile_t func; //!< Function to call to compile sections.
* virtual_server_root.
*/
MEM(vns_tree = rbtree_talloc_alloc(NULL,
- _virtual_namespace_cmp, fr_virtual_namespace_t,
- _virtual_namespace_free, RBTREE_FLAG_REPLACE));
+ fr_virtual_namespace_t, node,
+ _virtual_namespace_cmp,
+ _virtual_namespace_free, RBTREE_FLAG_REPLACE));
if (!cf_data_add(virtual_server_root, vns_tree, "vns_tree", true)) {
ERROR("Failed adding namespace tree data to config");
return -1;
}
- MEM(listen_addr_root = rbtree_alloc(NULL, listen_addr_cmp, NULL, RBTREE_FLAG_NONE));
- MEM(server_section_name_tree = rbtree_alloc(NULL, server_section_name_cmp, NULL, RBTREE_FLAG_NONE));
+ MEM(listen_addr_root = rbtree_alloc(NULL, fr_listen_t, virtual_server_node,
+ listen_addr_cmp, NULL, RBTREE_FLAG_NONE));
+ MEM(server_section_name_tree = rbtree_alloc(NULL, virtual_server_compile_t, node,
+ server_section_name_cmp, NULL, RBTREE_FLAG_NONE));
return 0;
}
bool listen_record(fr_listen_t *li) CC_HINT(nonnull);
int fr_app_process_type_parse(TALLOC_CTX *ctx, dl_module_inst_t **module_inst,
- CONF_ITEM *ci, fr_dict_attr_t const *packet_type,
+ CONF_ITEM *ci, fr_dict_attr_t const *packet_type,
char const *proto_name,
char const **type_table, size_t type_table_len,
dl_module_inst_t **type_submodule_by_code, uint32_t code_max);
*
*/
typedef struct {
+ fr_rb_node_t node; //!< Entry into the tree of sections.
char const *name; //!< Name of the processing section, such as "recv" or "send"
char const *name2; //!< Second name, such as "Access-Request"
rlm_components_t component; //!< Sets the default list of actions for this section
*
*/
struct xlat_inst {
+ fr_rb_node_t inst_node; //!< Entry in rbtree of thread instances.
+
xlat_exp_t const *node; //!< Node this data relates to.
void *data; //!< xlat node specific instance data.
};
*
*/
struct xlat_thread_inst {
+ fr_rb_node_t inst_node; //!< Entry in rbtree of thread instances.
+
xlat_exp_t const *node; //!< Node this data relates to.
void *data; //!< Thread specific instance data.
return XLAT_ACTION_DONE;
}
-
-
/** Global initialisation for xlat
*
* @note Free memory with #xlat_free
/*
* Create the function tree
*/
- xlat_root = rbtree_talloc_alloc(NULL, xlat_cmp, xlat_t, _xlat_func_tree_free, RBTREE_FLAG_REPLACE);
+ xlat_root = rbtree_talloc_alloc(NULL, xlat_t, node, xlat_cmp, _xlat_func_tree_free, RBTREE_FLAG_REPLACE);
if (!xlat_root) {
ERROR("%s: Failed to create tree", __FUNCTION__);
return -1;
if (!xlat_inst_tree) return 0;
if (!xlat_thread_inst_tree) {
- MEM(xlat_thread_inst_tree = rbtree_talloc_alloc(ctx, _xlat_thread_inst_cmp,
- xlat_thread_inst_t, _xlat_thread_inst_free, 0));
+ MEM(xlat_thread_inst_tree = rbtree_talloc_alloc(ctx, xlat_thread_inst_t, inst_node,
+ _xlat_thread_inst_cmp, _xlat_thread_inst_free, 0));
}
/*
{
if (xlat_inst_tree) return 0;
- xlat_inst_tree = rbtree_talloc_alloc(NULL, _xlat_inst_cmp,
- xlat_inst_t, _xlat_inst_free, RBTREE_FLAG_NONE);
+ xlat_inst_tree = rbtree_talloc_alloc(NULL, xlat_inst_t, inst_node,
+ _xlat_inst_cmp, _xlat_inst_free, RBTREE_FLAG_NONE);
if (!xlat_inst_tree) return -1;
return 0;
} xlat_func_legacy_type_t;
typedef struct xlat_s {
+ fr_rb_node_t node; //!< Entry in the xlat function tree.
char const *name; //!< Name of xlat function.
union {
return NULL;
}
- dl_loader->tree = rbtree_talloc_alloc(dl_loader, dl_handle_cmp, dl_t, NULL, 0);
+ dl_loader->tree = rbtree_talloc_alloc(dl_loader, dl_t, node, dl_handle_cmp, NULL, 0);
if (!dl_loader->tree) {
fr_strerror_const("Failed initialising dl->tree");
error:
# include <dlfcn.h>
#endif
#include <freeradius-devel/util/version.h>
+#include <freeradius-devel/util/rbtree.h>
#include <talloc.h>
#include <stdbool.h>
* Contains module's dlhandle, and the functions it exports.
*/
typedef struct dl_s {
+ fr_rb_node_t node; //!< Entry in the rbtree module handles.
+
char const *name; //!< Name of the module e.g. sql.
void *handle; //!< Handle returned by dlopen.
dl_loader_t *loader; //!< Loader that owns this dl.
};
static fr_table_num_sorted_t const fr_event_fd_type_table[] = {
- { L("directory"), FR_EVENT_FD_DIRECTORY },
+ { L("directory"), FR_EVENT_FD_DIRECTORY },
{ L("file"), FR_EVENT_FD_FILE },
{ L("pcap"), FR_EVENT_FD_PCAP },
{ L("socket"), FR_EVENT_FD_SOCKET }
*
*/
struct fr_event_fd {
+ fr_rb_node_t node; //!< Entry in the tree of file descriptor handles.
+ ///< this should really go away and we should pass around
+ ///< handles directly.
+
fr_event_list_t *el; //!< because talloc_parent() is O(N) in number of objects
fr_event_filter_t filter;
int fd; //!< File descriptor we're listening for events on.
return NULL;
}
- el->fds = rbtree_talloc_alloc(el, fr_event_fd_cmp, fr_event_fd_t, NULL, 0);
+ el->fds = rbtree_talloc_alloc(el, fr_event_fd_t, node, fr_event_fd_cmp, NULL, 0);
if (!el->fds) {
fr_strerror_const("Failed allocating FD tree");
goto error;
};
typedef struct {
+ fr_rb_node_t node;
char const *file;
int line;
uint32_t count;
}
for (i = 0; i < NUM_ELEMENTS(decades); i++) {
- locations[i] = rbtree_alloc(tmp_ctx, event_timer_location_cmp, NULL, 0);
+ locations[i] = rbtree_alloc(tmp_ctx, fr_event_counter_t, node, event_timer_location_cmp, NULL, 0);
if (!locations[i]) goto oom;
}
* data,data_len: Used between fr_radius_recv and fr_radius_decode.
*/
typedef struct {
+ fr_rb_node_t node; //!< Allows insertion into the list.c
+ ///< rbtree, may be removed in future.
+
fr_socket_t socket; //!< This packet was received on.
int id; //!< Packet ID (used to link requests/responses).
/** Create a new RED-BLACK tree
*
* @note Due to the node memory being allocated from a different pool to the main
+ *
+ * @param[in] ctx to allocate the tree in.
+ * Only the tree is allocated in this context, the memory
+ * for the #fr_rb_node_t is allocated as part of the data
+ * being inserted into the tree.
+ * @param[in] offset offsetof the #fr_rb_node_t field in the data being inserted.
+ * @param[in] type Talloc type of structures being inserted, may be NULL.
+ * @param[in] compare Comparator function for ordering data in the tree.
+ * @param[in] node_free Free function to call whenever data is deleted or replaced.
+ * @param[in] flags A bitfield of flags.
+ * - RBTREE_FLAG_REPLACE - replace nodes if a duplicate is found.
+ * - RBTREE_FLAG_LOCK - use a mutex to prevent concurrent access
+ * to the tree.
+ * @return
+ * - A new tree on success.
+ * - NULL on failure.
*/
-rbtree_t *_rbtree_alloc(TALLOC_CTX *ctx, fr_rb_cmp_t compare,
- char const *type, fr_rb_free_t node_free, int flags)
+rbtree_t *_rbtree_alloc(TALLOC_CTX *ctx,
+ size_t offset, char const *type,
+ fr_rb_cmp_t compare, fr_rb_free_t node_free,
+ int flags)
{
rbtree_t *tree;
.magic = RBTREE_MAGIC,
#endif
.root = NIL,
-// .offset = offset,
+ .offset = offset,
.type = type,
.compare = compare,
.replace = ((flags & RBTREE_FLAG_REPLACE) != 0),
if (tree->lock) pthread_mutex_init(&tree->mutex, NULL);
talloc_set_destructor(tree, _tree_free);
- tree->free = node_free;
- tree->type = type;
return tree;
}
* @param[in] _ctx to tie tree lifetime to.
* If ctx is freed, tree will free any nodes, calling the
* free function if set.
+ * @param[in] _type of item being stored in the tree, e.g. fr_value_box_t.
+ * @param[in] _field Containing the #fr_rb_node_t within item being stored.
* @param[in] _cmp Comparator used to compare nodes.
- * @param[in] _talloc_type of elements.
* @param[in] _node_free Optional function used to free data if tree nodes are
* deleted or replaced.
* @param[in] _flags To modify tree behaviour.
* - A new rbtree on success.
* - NULL on failure.
*/
-#define rbtree_talloc_alloc(_ctx, _cmp, _talloc_type, _node_free, _flags) \
- _rbtree_alloc(_ctx, _cmp, #_talloc_type, _node_free, _flags)
+#define rbtree_talloc_alloc(_ctx, _type, _field, _cmp, _node_free, _flags) \
+ _Generic((((_type *)0)->_field), \
+ fr_rb_node_t: _rbtree_alloc(_ctx, offsetof(_type, _field), #_type, _cmp, _node_free, _flags) \
+ )
/** Creates a red black tree
*
* @param[in] _ctx to tie tree lifetime to.
* If ctx is freed, tree will free any nodes, calling the
* free function if set.
+ * @param[in] _type of item being stored in the tree, e.g. fr_value_box_t.
+ * @param[in] _field Containing the #fr_rb_node_t within item being stored.
* @param[in] _cmp Comparator used to compare nodes.
* @param[in] _node_free Optional function used to free data if tree nodes are
* deleted or replaced.
* - A new rbtree on success.
* - NULL on failure.
*/
-#define rbtree_alloc(_ctx, _cmp, _node_free, _flags) \
- _rbtree_alloc(_ctx, _cmp, NULL, _node_free, _flags)
+#define rbtree_alloc(_ctx, _type, _field, _cmp, _node_free, _flags) \
+ _Generic((((_type *)0)->_field), \
+ fr_rb_node_t: _rbtree_alloc(_ctx, offsetof(_type, _field), NULL, _cmp, _node_free, _flags) \
+ )
-rbtree_t *_rbtree_alloc(TALLOC_CTX *ctx, fr_rb_cmp_t compare,
- char const *type, fr_rb_free_t node_free, int flags);
+rbtree_t *_rbtree_alloc(TALLOC_CTX *ctx, size_t offset, char const *type,
+ fr_rb_cmp_t compare, fr_rb_free_t node_free, int flags);
void rbtree_node_talloc_free(void *data);
#define BFD_AUTH_INVALID (BFD_AUTH_MET_KEYED_SHA1 + 1)
typedef struct {
+ fr_rb_node_t node; //!< Entry in the tree of sessions.
+
int number;
fr_socket_t socket;
}
}
- sock->session_tree = rbtree_talloc_alloc(sock, bfd_session_cmp, bfd_state_t, bfd_session_free, 0);
+ sock->session_tree = rbtree_talloc_alloc(sock, bfd_state_t, node, bfd_session_cmp, bfd_session_free, 0);
if (!sock->session_tree) {
ERROR("Failed creating session tree!");
return -1;
#include "sync.h"
struct sync_state_s {
- fr_ldap_connection_t *conn;
+ fr_rb_node_t node; //!< Entry in the tree of nodes.
+
+ fr_ldap_connection_t *conn;
sync_config_t const *config;
* these are specific to the connection.
*/
if (!conn->uctx) {
- MEM(tree = rbtree_talloc_alloc(conn, _sync_cmp, sync_state_t, NULL, RBTREE_FLAG_NONE));
+ MEM(tree = rbtree_talloc_alloc(conn, sync_state_t, node, _sync_cmp, NULL, RBTREE_FLAG_NONE));
conn->uctx = tree;
} else {
tree = talloc_get_type_abort(conn->uctx, rbtree_t);
typedef struct {
rlm_cache_entry_t fields; //!< Entry data.
- int32_t heap_id; //!< Offset used for heap.
+
+ fr_rb_node_t node; //!< Entry used for lookups.
+ int32_t heap_id; //!< Offset used for expiry heap.
} rlm_cache_rbtree_entry_t;
/** Compare two entries by key
/*
* The cache.
*/
- driver->cache = rbtree_talloc_alloc(NULL, cache_entry_cmp, rlm_cache_rbtree_entry_t, NULL, 0);
+ driver->cache = rbtree_talloc_alloc(NULL, rlm_cache_rbtree_entry_t, node, cache_entry_cmp, NULL, 0);
if (!driver->cache) {
ERROR("Failed to create cache");
return -1;
typedef struct rlm_csv_entry_s rlm_csv_entry_t;
struct rlm_csv_entry_s {
+ fr_rb_node_t node;
rlm_csv_entry_t *next;
fr_value_box_t *key;
char *data[];
(inst->key_data_type == FR_TYPE_IPV6_ADDR) || (inst->key_data_type == FR_TYPE_IPV6_PREFIX)) {
MEM(inst->trie = fr_trie_alloc(inst));
} else {
- MEM(inst->tree = rbtree_talloc_alloc(inst, csv_entry_cmp, rlm_csv_entry_t, NULL, 0));
+ MEM(inst->tree = rbtree_talloc_alloc(inst, rlm_csv_entry_t, node, csv_entry_cmp, NULL, 0));
}
if ((*inst->index_field_name == ',') || (*inst->index_field_name == *inst->delimiter)) {
entry = entry->next;
}
- tree = rbtree_alloc(ctx, pairlist_cmp, NULL, RBTREE_FLAG_NONE);
+ tree = rbtree_alloc(ctx, PAIR_LIST, node, pairlist_cmp, NULL, RBTREE_FLAG_NONE);
if (!tree) {
pairlist_free(&users);
return -1;
} rlm_icmp_thread_t;
typedef struct {
+ fr_rb_node_t node; //!< Entry in the outstanding list of echo requests.
bool replied; //!< do we have a reply?
fr_value_box_t *ip; //!< the IP we're pinging
uint32_t counter; //!< for pinging the same IP multiple times
rlm_icmp_thread_t *t = talloc_get_type_abort(thread, rlm_icmp_thread_t);
fr_ipaddr_t ipaddr, *src;
- MEM(t->tree = rbtree_alloc(t, echo_cmp, NULL, RBTREE_FLAG_NONE));
+ MEM(t->tree = rbtree_alloc(t, rlm_icmp_echo_t, node, echo_cmp, NULL, RBTREE_FLAG_NONE));
t->inst = inst;
t->el = el;
* If needed, allocate a subtree.
*/
if (!tt->subtree[tt->next_id]) {
- MEM(tt->subtree[tt->next_id] = rbtree_talloc_alloc(tt, te_cmp, radius_track_entry_t,
- NULL, RBTREE_FLAG_NONE));
+ MEM(tt->subtree[tt->next_id] = rbtree_talloc_alloc(tt, radius_track_entry_t, node,
+ te_cmp, NULL, RBTREE_FLAG_NONE));
}
/*
*
*/
struct radius_track_entry_s {
+ fr_rb_node_t node; //!< Entry in the tracking tree.
+
radius_track_t *tt;
radius_track_entry_t ***binding; //!< Binding chunk we use to release the entry
* Lookup sessions in the tree. We don't free them in
* the tree, as that's taken care of elsewhere...
*/
- inst->session_tree = rbtree_talloc_alloc(NULL, securid_session_cmp, SECURID_SESSION NULL, 0);
+ inst->session_tree = rbtree_talloc_alloc(NULL, SECURID_SESSION, node, securid_session_cmp, NULL, 0);
if (!inst->session_tree) {
ERROR("Cannot initialize session tree");
return -1;
#define SECURID_STATE_LEN 32
typedef struct {
- struct _securid_session_t *prev, *next;
- SDI_HANDLE sdiHandle;
- SECURID_SESSION_STATE securidSessionState;
+ struct _securid_session_t *prev, *next;
+ fr_rb_node_t node;
+ SDI_HANDLE sdiHandle;
+ SECURID_SESSION_STATE securidSessionState;
- char state[SECURID_STATE_LEN];
+ char state[SECURID_STATE_LEN];
- fr_ipaddr_t src_ipaddr;
- time_t timestamp;
- unsigned int session_id;
- uint32_t trips;
+ fr_ipaddr_t src_ipaddr;
+ time_t timestamp;
+ unsigned int session_id;
+ uint32_t trips;
- char *pin; /* previous pin if user entered it during NEW-PIN mode process */
- char *identity; /* save user's identity name for future use */
+ char *pin; /* previous pin if user entered it during NEW-PIN mode process */
+ char *identity; /* save user's identity name for future use */
} SECURID_SESSION;
return 0;
}
- txn_tree = rbtree_talloc_alloc(NULL, sigtran_txn_cmp, sigtran_transaction_t, false, 0);
+ txn_tree = rbtree_talloc_alloc(NULL, sigtran_transaction_t, node, sigtran_txn_cmp, false, 0);
if (!txn_tree) return -1;
txn_tree_inst++;
* and won't mind extra memory being allocated from this ctx.
*/
typedef struct sigtran_transaction {
+ fr_rb_node_t node; //!< Entry in the tree of transactions.
+
struct {
sigtran_request_type_t type; //!< Type of request
void *data; //!< Data for the request.
} rlm_stats_t;
typedef struct {
+ fr_rb_node_t src_node;
+ fr_rb_node_t dst_node;
fr_ipaddr_t ipaddr; //!< IP address of this thing
fr_time_t created; //!< when it was created
fr_time_t last_packet; //!< when we last saw a packet
t->inst = inst;
- t->src = rbtree_talloc_alloc(t, data_cmp, rlm_stats_data_t, NULL, RBTREE_FLAG_LOCK);
- t->dst = rbtree_talloc_alloc(t, data_cmp, rlm_stats_data_t, NULL, RBTREE_FLAG_LOCK);
+ t->src = rbtree_talloc_alloc(t, rlm_stats_data_t, src_node, data_cmp, NULL, RBTREE_FLAG_LOCK);
+ t->dst = rbtree_talloc_alloc(t, rlm_stats_data_t, dst_node, data_cmp, NULL, RBTREE_FLAG_LOCK);
pthread_mutex_lock(&inst->mutex);
fr_dlist_insert_head(&inst->list, t);
pl = talloc_zero(NULL, fr_packet_list_t);
if (!pl) return NULL;
- pl->tree = rbtree_alloc(pl, fr_packet_cmp, NULL, 0); /* elements not talloc safe */
+ pl->tree = rbtree_alloc(pl, fr_radius_packet_t, node, fr_packet_cmp, NULL, 0); /* elements not talloc safe */
if (!pl->tree) {
fr_packet_list_free(pl);
return NULL;
fprintf(stderr, "filter = %x mask = %x n = %i\n", thresh.num, mask, n);
- t = rbtree_alloc(NULL, comp, freenode, RBTREE_FLAG_LOCK);
+ t = rbtree_alloc(NULL, fr_rb_test_node_t, node, comp, freenode, RBTREE_FLAG_LOCK);
for (i = 0; i < n; i++) {
fr_rb_test_node_t *p;