X-Git-Url: http://git.liburcu.org/?p=urcu.git;a=blobdiff_plain;f=urcu-bp.c;h=cbdebf407a892c2d201d42d948025723702414e7;hp=62d270412823062ac175aaeb4f412b1518c34c8f;hb=a59f39055b5ecb77b68cf78b9839aa9e8e4ec332;hpb=6cf3827cf5809fbcee555fb08286ad756be42dad diff --git a/urcu-bp.c b/urcu-bp.c index 62d2704..cbdebf4 100644 --- a/urcu-bp.c +++ b/urcu-bp.c @@ -24,6 +24,7 @@ */ #define _GNU_SOURCE +#define _LGPL_SOURCE #include #include #include @@ -35,48 +36,132 @@ #include #include -#include "urcu-bp-static.h" +#include "urcu/arch.h" +#include "urcu/wfcqueue.h" +#include "urcu/map/urcu-bp.h" +#include "urcu/static/urcu-bp.h" +#include "urcu-pointer.h" +#include "urcu/tls-compat.h" + +#include "urcu-die.h" + /* Do not #define _LGPL_SOURCE to ensure we can emit the wrapper symbols */ +#undef _LGPL_SOURCE #include "urcu-bp.h" +#define _LGPL_SOURCE -/* Sleep delay in us */ -#define RCU_SLEEP_DELAY 1000 -#define ARENA_INIT_ALLOC 16 +#ifndef MAP_ANONYMOUS +#define MAP_ANONYMOUS MAP_ANON +#endif -void __attribute__((destructor)) rcu_bp_exit(void); +#ifdef __linux__ +static +void *mremap_wrapper(void *old_address, size_t old_size, + size_t new_size, int flags) +{ + return mremap(old_address, old_size, new_size, flags); +} +#else -static pthread_mutex_t rcu_gp_lock = PTHREAD_MUTEX_INITIALIZER; +#define MREMAP_MAYMOVE 1 +#define MREMAP_FIXED 2 -#ifdef DEBUG_YIELD -unsigned int yield_active; -unsigned int __thread rand_yield; +/* + * mremap wrapper for non-Linux systems not allowing MAYMOVE. + * This is not generic. +*/ +static +void *mremap_wrapper(void *old_address, size_t old_size, + size_t new_size, int flags) +{ + assert(!(flags & MREMAP_MAYMOVE)); + + return MAP_FAILED; +} #endif +/* Sleep delay in ms */ +#define RCU_SLEEP_DELAY_MS 10 +#define INIT_NR_THREADS 8 +#define ARENA_INIT_ALLOC \ + sizeof(struct registry_chunk) \ + + INIT_NR_THREADS * sizeof(struct rcu_reader) + /* - * Global grace period counter. - * Contains the current RCU_GP_CTR_PHASE. - * Also has a RCU_GP_COUNT of 1, to accelerate the reader fast path. - * Written to only by writer with mutex taken. Read by both writer and readers. + * Active attempts to check for reader Q.S. before calling sleep(). */ -long rcu_gp_ctr = RCU_GP_COUNT; +#define RCU_QS_ACTIVE_ATTEMPTS 100 + +static +int rcu_bp_refcount; + +/* If the headers do not support membarrier system call, fall back smp_mb. */ +#ifdef __NR_membarrier +# define membarrier(...) syscall(__NR_membarrier, __VA_ARGS__) +#else +# define membarrier(...) -ENOSYS +#endif + +enum membarrier_cmd { + MEMBARRIER_CMD_QUERY = 0, + MEMBARRIER_CMD_SHARED = (1 << 0), +}; + +static +void __attribute__((constructor)) rcu_bp_init(void); +static +void __attribute__((destructor)) rcu_bp_exit(void); + +int urcu_bp_has_sys_membarrier; + +/* + * rcu_gp_lock ensures mutual exclusion between threads calling + * synchronize_rcu(). + */ +static pthread_mutex_t rcu_gp_lock = PTHREAD_MUTEX_INITIALIZER; +/* + * rcu_registry_lock ensures mutual exclusion between threads + * registering and unregistering themselves to/from the registry, and + * with threads reading that registry from synchronize_rcu(). However, + * this lock is not held all the way through the completion of awaiting + * for the grace period. It is sporadically released between iterations + * on the registry. + * rcu_registry_lock may nest inside rcu_gp_lock. + */ +static pthread_mutex_t rcu_registry_lock = PTHREAD_MUTEX_INITIALIZER; + +static pthread_mutex_t init_lock = PTHREAD_MUTEX_INITIALIZER; +static int initialized; + +static pthread_key_t urcu_bp_key; + +struct rcu_gp rcu_gp = { .ctr = RCU_GP_COUNT }; /* * Pointer to registry elements. Written to only by each individual reader. Read * by both the reader and the writers. */ -struct rcu_reader __thread *rcu_reader; +DEFINE_URCU_TLS(struct rcu_reader *, rcu_reader); static CDS_LIST_HEAD(registry); +struct registry_chunk { + size_t data_len; /* data length */ + size_t used; /* amount of data used */ + struct cds_list_head node; /* chunk_list node */ + char data[]; +}; + struct registry_arena { - void *p; - size_t len; - size_t used; + struct cds_list_head chunk_list; }; -static struct registry_arena registry_arena; +static struct registry_arena registry_arena = { + .chunk_list = CDS_LIST_HEAD_INIT(registry_arena.chunk_list), +}; -static void rcu_gc_registry(void); +/* Saved fork signal mask, protected by rcu_gp_lock */ +static sigset_t saved_fork_signal_mask; static void mutex_lock(pthread_mutex_t *mutex) { @@ -84,22 +169,12 @@ static void mutex_lock(pthread_mutex_t *mutex) #ifndef DISTRUST_SIGNALS_EXTREME ret = pthread_mutex_lock(mutex); - if (ret) { - perror("Error in pthread mutex lock"); - exit(-1); - } + if (ret) + urcu_die(ret); #else /* #ifndef DISTRUST_SIGNALS_EXTREME */ while ((ret = pthread_mutex_trylock(mutex)) != 0) { - if (ret != EBUSY && ret != EINTR) { - printf("ret = %d, errno = %d\n", ret, errno); - perror("Error in pthread mutex lock"); - exit(-1); - } - if (rcu_reader.need_mb) { - cmm_smp_mb(); - rcu_reader.need_mb = 0; - cmm_smp_mb(); - } + if (ret != EBUSY && ret != EINTR) + urcu_die(ret); poll(NULL,0,10); } #endif /* #else #ifndef DISTRUST_SIGNALS_EXTREME */ @@ -110,85 +185,123 @@ static void mutex_unlock(pthread_mutex_t *mutex) int ret; ret = pthread_mutex_unlock(mutex); - if (ret) { - perror("Error in pthread mutex unlock"); - exit(-1); - } + if (ret) + urcu_die(ret); } -void update_counter_and_wait(void) +static void smp_mb_master(void) { - CDS_LIST_HEAD(qsreaders); - int wait_loops = 0; - struct rcu_reader *index, *tmp; - - /* Switch parity: 0 -> 1, 1 -> 0 */ - CMM_STORE_SHARED(rcu_gp_ctr, rcu_gp_ctr ^ RCU_GP_CTR_PHASE); - - /* - * Must commit qparity update to memory before waiting for other parity - * quiescent state. Failure to do so could result in the writer waiting - * forever while new readers are always accessing data (no progress). - * Ensured by CMM_STORE_SHARED and CMM_LOAD_SHARED. - */ + if (caa_likely(urcu_bp_has_sys_membarrier)) + (void) membarrier(MEMBARRIER_CMD_SHARED, 0); + else + cmm_smp_mb(); +} - /* - * Adding a cmm_smp_mb() which is _not_ formally required, but makes the - * model easier to understand. It does not have a big performance impact - * anyway, given this is the write-side. - */ - cmm_smp_mb(); +/* + * Always called with rcu_registry lock held. Releases this lock between + * iterations and grabs it again. Holds the lock when it returns. + */ +static void wait_for_readers(struct cds_list_head *input_readers, + struct cds_list_head *cur_snap_readers, + struct cds_list_head *qsreaders) +{ + unsigned int wait_loops = 0; + struct rcu_reader *index, *tmp; /* - * Wait for each thread rcu_reader.ctr count to become 0. + * Wait for each thread URCU_TLS(rcu_reader).ctr to either + * indicate quiescence (not nested), or observe the current + * rcu_gp.ctr value. */ for (;;) { - wait_loops++; - cds_list_for_each_entry_safe(index, tmp, ®istry, node) { - if (!rcu_old_gp_ongoing(&index->ctr)) - cds_list_move(&index->node, &qsreaders); + if (wait_loops < RCU_QS_ACTIVE_ATTEMPTS) + wait_loops++; + + cds_list_for_each_entry_safe(index, tmp, input_readers, node) { + switch (rcu_reader_state(&index->ctr)) { + case RCU_READER_ACTIVE_CURRENT: + if (cur_snap_readers) { + cds_list_move(&index->node, + cur_snap_readers); + break; + } + /* Fall-through */ + case RCU_READER_INACTIVE: + cds_list_move(&index->node, qsreaders); + break; + case RCU_READER_ACTIVE_OLD: + /* + * Old snapshot. Leaving node in + * input_readers will make us busy-loop + * until the snapshot becomes current or + * the reader becomes inactive. + */ + break; + } } - if (cds_list_empty(®istry)) { + if (cds_list_empty(input_readers)) { break; } else { - if (wait_loops == RCU_QS_ACTIVE_ATTEMPTS) - usleep(RCU_SLEEP_DELAY); + /* Temporarily unlock the registry lock. */ + mutex_unlock(&rcu_registry_lock); + if (wait_loops >= RCU_QS_ACTIVE_ATTEMPTS) + (void) poll(NULL, 0, RCU_SLEEP_DELAY_MS); else caa_cpu_relax(); + /* Re-lock the registry lock before the next loop. */ + mutex_lock(&rcu_registry_lock); } } - /* put back the reader list in the registry */ - cds_list_splice(&qsreaders, ®istry); } void synchronize_rcu(void) { + CDS_LIST_HEAD(cur_snap_readers); + CDS_LIST_HEAD(qsreaders); sigset_t newmask, oldmask; int ret; - ret = sigemptyset(&newmask); + ret = sigfillset(&newmask); assert(!ret); - ret = pthread_sigmask(SIG_SETMASK, &newmask, &oldmask); + ret = pthread_sigmask(SIG_BLOCK, &newmask, &oldmask); assert(!ret); mutex_lock(&rcu_gp_lock); + mutex_lock(&rcu_registry_lock); + if (cds_list_empty(®istry)) goto out; /* All threads should read qparity before accessing data structure * where new ptr points to. */ /* Write new ptr before changing the qparity */ + smp_mb_master(); + + /* + * Wait for readers to observe original parity or be quiescent. + * wait_for_readers() can release and grab again rcu_registry_lock + * interally. + */ + wait_for_readers(®istry, &cur_snap_readers, &qsreaders); + + /* + * Adding a cmm_smp_mb() which is _not_ formally required, but makes the + * model easier to understand. It does not have a big performance impact + * anyway, given this is the write-side. + */ cmm_smp_mb(); - /* Remove old registry elements */ - rcu_gc_registry(); + /* Switch parity: 0 -> 1, 1 -> 0 */ + CMM_STORE_SHARED(rcu_gp.ctr, rcu_gp.ctr ^ RCU_GP_CTR_PHASE); /* - * Wait for previous parity to be empty of readers. + * Must commit qparity update to memory before waiting for other parity + * quiescent state. Failure to do so could result in the writer waiting + * forever while new readers are always accessing data (no progress). + * Ensured by CMM_STORE_SHARED and CMM_LOAD_SHARED. */ - update_counter_and_wait(); /* 0 -> 1, wait readers in parity 0 */ /* * Adding a cmm_smp_mb() which is _not_ formally required, but makes the @@ -198,16 +311,24 @@ void synchronize_rcu(void) cmm_smp_mb(); /* - * Wait for previous parity to be empty of readers. + * Wait for readers to observe new parity or be quiescent. + * wait_for_readers() can release and grab again rcu_registry_lock + * interally. + */ + wait_for_readers(&cur_snap_readers, NULL, &qsreaders); + + /* + * Put quiescent reader list back into registry. */ - update_counter_and_wait(); /* 1 -> 0, wait readers in parity 1 */ + cds_list_splice(&qsreaders, ®istry); /* * Finish waiting for reader threads before letting the old ptr being * freed. */ - cmm_smp_mb(); + smp_mb_master(); out: + mutex_unlock(&rcu_registry_lock); mutex_unlock(&rcu_gp_lock); ret = pthread_sigmask(SIG_SETMASK, &oldmask, NULL); assert(!ret); @@ -227,84 +348,168 @@ void rcu_read_unlock(void) _rcu_read_unlock(); } +int rcu_read_ongoing(void) +{ + return _rcu_read_ongoing(); +} + /* - * only grow for now. + * Only grow for now. If empty, allocate a ARENA_INIT_ALLOC sized chunk. + * Else, try expanding the last chunk. If this fails, allocate a new + * chunk twice as big as the last chunk. + * Memory used by chunks _never_ moves. A chunk could theoretically be + * freed when all "used" slots are released, but we don't do it at this + * point. */ -static void resize_arena(struct registry_arena *arena, size_t len) +static +void expand_arena(struct registry_arena *arena) { - void *new_arena; + struct registry_chunk *new_chunk, *last_chunk; + size_t old_chunk_len, new_chunk_len; + + /* No chunk. */ + if (cds_list_empty(&arena->chunk_list)) { + assert(ARENA_INIT_ALLOC >= + sizeof(struct registry_chunk) + + sizeof(struct rcu_reader)); + new_chunk_len = ARENA_INIT_ALLOC; + new_chunk = mmap(NULL, new_chunk_len, + PROT_READ | PROT_WRITE, + MAP_ANONYMOUS | MAP_PRIVATE, + -1, 0); + if (new_chunk == MAP_FAILED) + abort(); + memset(new_chunk, 0, new_chunk_len); + new_chunk->data_len = + new_chunk_len - sizeof(struct registry_chunk); + cds_list_add_tail(&new_chunk->node, &arena->chunk_list); + return; /* We're done. */ + } - if (!arena->p) - new_arena = mmap(arena->p, len, - PROT_READ | PROT_WRITE, - MAP_ANONYMOUS | MAP_PRIVATE, - -1, 0); - else - new_arena = mremap(arena->p, arena->len, - len, MREMAP_MAYMOVE); - assert(new_arena != MAP_FAILED); + /* Try expanding last chunk. */ + last_chunk = cds_list_entry(arena->chunk_list.prev, + struct registry_chunk, node); + old_chunk_len = + last_chunk->data_len + sizeof(struct registry_chunk); + new_chunk_len = old_chunk_len << 1; + + /* Don't allow memory mapping to move, just expand. */ + new_chunk = mremap_wrapper(last_chunk, old_chunk_len, + new_chunk_len, 0); + if (new_chunk != MAP_FAILED) { + /* Should not have moved. */ + assert(new_chunk == last_chunk); + memset((char *) last_chunk + old_chunk_len, 0, + new_chunk_len - old_chunk_len); + last_chunk->data_len = + new_chunk_len - sizeof(struct registry_chunk); + return; /* We're done. */ + } - /* - * re-used the same region ? - */ - if (new_arena == arena->p) - return; + /* Remap did not succeed, we need to add a new chunk. */ + new_chunk = mmap(NULL, new_chunk_len, + PROT_READ | PROT_WRITE, + MAP_ANONYMOUS | MAP_PRIVATE, + -1, 0); + if (new_chunk == MAP_FAILED) + abort(); + memset(new_chunk, 0, new_chunk_len); + new_chunk->data_len = + new_chunk_len - sizeof(struct registry_chunk); + cds_list_add_tail(&new_chunk->node, &arena->chunk_list); +} + +static +struct rcu_reader *arena_alloc(struct registry_arena *arena) +{ + struct registry_chunk *chunk; + struct rcu_reader *rcu_reader_reg; + int expand_done = 0; /* Only allow to expand once per alloc */ + size_t len = sizeof(struct rcu_reader); + +retry: + cds_list_for_each_entry(chunk, &arena->chunk_list, node) { + if (chunk->data_len - chunk->used < len) + continue; + /* Find spot */ + for (rcu_reader_reg = (struct rcu_reader *) &chunk->data[0]; + rcu_reader_reg < (struct rcu_reader *) &chunk->data[chunk->data_len]; + rcu_reader_reg++) { + if (!rcu_reader_reg->alloc) { + rcu_reader_reg->alloc = 1; + chunk->used += len; + return rcu_reader_reg; + } + } + } - memcpy(new_arena, arena->p, arena->len); - bzero(new_arena + arena->len, len - arena->len); - arena->p = new_arena; + if (!expand_done) { + expand_arena(arena); + expand_done = 1; + goto retry; + } + + return NULL; } /* Called with signals off and mutex locked */ -static void add_thread(void) +static +void add_thread(void) { struct rcu_reader *rcu_reader_reg; + int ret; - if (registry_arena.len - < registry_arena.used + sizeof(struct rcu_reader)) - resize_arena(®istry_arena, - max(registry_arena.len << 1, ARENA_INIT_ALLOC)); - /* - * Find a free spot. - */ - for (rcu_reader_reg = registry_arena.p; - (void *)rcu_reader_reg < registry_arena.p + registry_arena.len; - rcu_reader_reg++) { - if (!rcu_reader_reg->alloc) - break; - } - rcu_reader_reg->alloc = 1; - registry_arena.used += sizeof(struct rcu_reader); + rcu_reader_reg = arena_alloc(®istry_arena); + if (!rcu_reader_reg) + abort(); + ret = pthread_setspecific(urcu_bp_key, rcu_reader_reg); + if (ret) + abort(); /* Add to registry */ rcu_reader_reg->tid = pthread_self(); assert(rcu_reader_reg->ctr == 0); cds_list_add(&rcu_reader_reg->node, ®istry); - rcu_reader = rcu_reader_reg; + /* + * Reader threads are pointing to the reader registry. This is + * why its memory should never be relocated. + */ + URCU_TLS(rcu_reader) = rcu_reader_reg; } -/* Called with signals off and mutex locked */ -static void rcu_gc_registry(void) +/* Called with mutex locked */ +static +void cleanup_thread(struct registry_chunk *chunk, + struct rcu_reader *rcu_reader_reg) { - struct rcu_reader *rcu_reader_reg; - pthread_t tid; - int ret; + rcu_reader_reg->ctr = 0; + cds_list_del(&rcu_reader_reg->node); + rcu_reader_reg->tid = 0; + rcu_reader_reg->alloc = 0; + chunk->used -= sizeof(struct rcu_reader); +} - for (rcu_reader_reg = registry_arena.p; - (void *)rcu_reader_reg < registry_arena.p + registry_arena.len; - rcu_reader_reg++) { - if (!rcu_reader_reg->alloc) +static +struct registry_chunk *find_chunk(struct rcu_reader *rcu_reader_reg) +{ + struct registry_chunk *chunk; + + cds_list_for_each_entry(chunk, ®istry_arena.chunk_list, node) { + if (rcu_reader_reg < (struct rcu_reader *) &chunk->data[0]) continue; - tid = rcu_reader_reg->tid; - ret = pthread_kill(tid, 0); - assert(ret != EINVAL); - if (ret == ESRCH) { - cds_list_del(&rcu_reader_reg->node); - rcu_reader_reg->ctr = 0; - rcu_reader_reg->alloc = 0; - registry_arena.used -= sizeof(struct rcu_reader); - } + if (rcu_reader_reg >= (struct rcu_reader *) &chunk->data[chunk->data_len]) + continue; + return chunk; } + return NULL; +} + +/* Called with signals off and mutex locked */ +static +void remove_thread(struct rcu_reader *rcu_reader_reg) +{ + cleanup_thread(find_chunk(rcu_reader_reg), rcu_reader_reg); + URCU_TLS(rcu_reader) = NULL; } /* Disable signals, take mutex, add to registry */ @@ -313,26 +518,201 @@ void rcu_bp_register(void) sigset_t newmask, oldmask; int ret; - ret = sigemptyset(&newmask); - assert(!ret); - ret = pthread_sigmask(SIG_SETMASK, &newmask, &oldmask); - assert(!ret); + ret = sigfillset(&newmask); + if (ret) + abort(); + ret = pthread_sigmask(SIG_BLOCK, &newmask, &oldmask); + if (ret) + abort(); /* * Check if a signal concurrently registered our thread since - * the check in rcu_read_lock(). */ - if (rcu_reader) + * the check in rcu_read_lock(). + */ + if (URCU_TLS(rcu_reader)) goto end; - mutex_lock(&rcu_gp_lock); + /* + * Take care of early registration before urcu_bp constructor. + */ + rcu_bp_init(); + + mutex_lock(&rcu_registry_lock); add_thread(); - mutex_unlock(&rcu_gp_lock); + mutex_unlock(&rcu_registry_lock); end: + ret = pthread_sigmask(SIG_SETMASK, &oldmask, NULL); + if (ret) + abort(); +} + +/* Disable signals, take mutex, remove from registry */ +static +void rcu_bp_unregister(struct rcu_reader *rcu_reader_reg) +{ + sigset_t newmask, oldmask; + int ret; + + ret = sigfillset(&newmask); + if (ret) + abort(); + ret = pthread_sigmask(SIG_BLOCK, &newmask, &oldmask); + if (ret) + abort(); + + mutex_lock(&rcu_registry_lock); + remove_thread(rcu_reader_reg); + mutex_unlock(&rcu_registry_lock); + ret = pthread_sigmask(SIG_SETMASK, &oldmask, NULL); + if (ret) + abort(); + rcu_bp_exit(); +} + +/* + * Remove thread from the registry when it exits, and flag it as + * destroyed so garbage collection can take care of it. + */ +static +void urcu_bp_thread_exit_notifier(void *rcu_key) +{ + rcu_bp_unregister(rcu_key); +} + +static +void rcu_bp_init(void) +{ + mutex_lock(&init_lock); + if (!rcu_bp_refcount++) { + int ret; + + ret = pthread_key_create(&urcu_bp_key, + urcu_bp_thread_exit_notifier); + if (ret) + abort(); + ret = membarrier(MEMBARRIER_CMD_QUERY, 0); + if (ret >= 0 && (ret & MEMBARRIER_CMD_SHARED)) { + urcu_bp_has_sys_membarrier = 1; + } + initialized = 1; + } + mutex_unlock(&init_lock); +} + +static +void rcu_bp_exit(void) +{ + mutex_lock(&init_lock); + if (!--rcu_bp_refcount) { + struct registry_chunk *chunk, *tmp; + int ret; + + cds_list_for_each_entry_safe(chunk, tmp, + ®istry_arena.chunk_list, node) { + munmap(chunk, chunk->data_len + + sizeof(struct registry_chunk)); + } + CDS_INIT_LIST_HEAD(®istry_arena.chunk_list); + ret = pthread_key_delete(urcu_bp_key); + if (ret) + abort(); + } + mutex_unlock(&init_lock); +} + +/* + * Holding the rcu_gp_lock and rcu_registry_lock across fork will make + * sure we fork() don't race with a concurrent thread executing with + * any of those locks held. This ensures that the registry and data + * protected by rcu_gp_lock are in a coherent state in the child. + */ +void rcu_bp_before_fork(void) +{ + sigset_t newmask, oldmask; + int ret; + + ret = sigfillset(&newmask); + assert(!ret); + ret = pthread_sigmask(SIG_BLOCK, &newmask, &oldmask); + assert(!ret); + mutex_lock(&rcu_gp_lock); + mutex_lock(&rcu_registry_lock); + saved_fork_signal_mask = oldmask; +} + +void rcu_bp_after_fork_parent(void) +{ + sigset_t oldmask; + int ret; + + oldmask = saved_fork_signal_mask; + mutex_unlock(&rcu_registry_lock); + mutex_unlock(&rcu_gp_lock); + ret = pthread_sigmask(SIG_SETMASK, &oldmask, NULL); + assert(!ret); +} + +/* + * Prune all entries from registry except our own thread. Fits the Linux + * fork behavior. Called with rcu_gp_lock and rcu_registry_lock held. + */ +static +void urcu_bp_prune_registry(void) +{ + struct registry_chunk *chunk; + struct rcu_reader *rcu_reader_reg; + + cds_list_for_each_entry(chunk, ®istry_arena.chunk_list, node) { + for (rcu_reader_reg = (struct rcu_reader *) &chunk->data[0]; + rcu_reader_reg < (struct rcu_reader *) &chunk->data[chunk->data_len]; + rcu_reader_reg++) { + if (!rcu_reader_reg->alloc) + continue; + if (rcu_reader_reg->tid == pthread_self()) + continue; + cleanup_thread(chunk, rcu_reader_reg); + } + } +} + +void rcu_bp_after_fork_child(void) +{ + sigset_t oldmask; + int ret; + + urcu_bp_prune_registry(); + oldmask = saved_fork_signal_mask; + mutex_unlock(&rcu_registry_lock); + mutex_unlock(&rcu_gp_lock); ret = pthread_sigmask(SIG_SETMASK, &oldmask, NULL); assert(!ret); } -void rcu_bp_exit() +void *rcu_dereference_sym_bp(void *p) +{ + return _rcu_dereference(p); +} + +void *rcu_set_pointer_sym_bp(void **p, void *v) +{ + cmm_wmb(); + uatomic_set(p, v); + return v; +} + +void *rcu_xchg_pointer_sym_bp(void **p, void *v) +{ + cmm_wmb(); + return uatomic_xchg(p, v); +} + +void *rcu_cmpxchg_pointer_sym_bp(void **p, void *old, void *_new) { - munmap(registry_arena.p, registry_arena.len); + cmm_wmb(); + return uatomic_cmpxchg(p, old, _new); } + +DEFINE_RCU_FLAVOR(rcu_flavor); + +#include "urcu-call-rcu-impl.h" +#include "urcu-defer-impl.h"