uatomic/x86: Remove redundant memory barriers
[urcu.git] / src / rculfhash.c
index 04fd49946aa940184015a4921971feff5d3818e5..10f5b8ed58a89c687421b72e513e218492299a0c 100644 (file)
@@ -1,24 +1,10 @@
+// SPDX-FileCopyrightText: 2010-2011 Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+// SPDX-FileCopyrightText: 2011 Lai Jiangshan <laijs@cn.fujitsu.com>
+//
+// SPDX-License-Identifier: LGPL-2.1-or-later
+
 /*
- * rculfhash.c
- *
  * Userspace RCU library - Lock-Free Resizable RCU Hash Table
- *
- * Copyright 2010-2011 - Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
- * Copyright 2011 - Lai Jiangshan <laijs@cn.fujitsu.com>
- *
- * This library is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License as published by the Free Software Foundation; either
- * version 2.1 of the License, or (at your option) any later version.
- *
- * This library is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this library; if not, write to the Free Software
- * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
  */
 
 /*
 #include <string.h>
 #include <sched.h>
 #include <unistd.h>
+#include <stdlib.h>
 
 #include "compat-getcpu.h"
 #include <urcu/assert.h>
 #include <urcu/uatomic.h>
 #include <urcu/compiler.h>
 #include <urcu/rculfhash.h>
-#include <urcu/static/urcu-signal-nr.h>
-#include <rculfhash-internal.h>
 #include <stdio.h>
 #include <pthread.h>
 #include <signal.h>
+#include "rculfhash-internal.h"
 #include "workqueue.h"
 #include "urcu-die.h"
 #include "urcu-utils.h"
+#include "compat-smp.h"
 
 /*
  * Split-counters lazily update the global counter each 1024
@@ -361,8 +348,12 @@ struct partition_resize_work {
                    unsigned long start, unsigned long len);
 };
 
+enum nr_cpus_mask_state {
+       NR_CPUS_MASK_INIT_FAILED = -2,
+       NR_CPUS_MASK_UNINITIALIZED = -1,
+};
+
 static struct urcu_workqueue *cds_lfht_workqueue;
-static unsigned long cds_lfht_workqueue_user_count;
 
 /*
  * Mutex ensuring mutual exclusion between workqueue initialization and
@@ -379,8 +370,8 @@ static struct urcu_atfork cds_lfht_atfork;
  */
 static int cds_lfht_workqueue_atfork_nesting;
 
+static void __attribute__((destructor)) cds_lfht_exit(void);
 static void cds_lfht_init_worker(const struct rcu_flavor_struct *flavor);
-static void cds_lfht_fini_worker(const struct rcu_flavor_struct *flavor);
 
 #ifdef CONFIG_CDS_LFHT_ITER_DEBUG
 
@@ -578,6 +569,50 @@ unsigned int cds_lfht_fls_ulong(unsigned long x)
 #endif
 }
 
+static void *cds_lfht_malloc(void *state __attribute__((unused)),
+               size_t size)
+{
+       return malloc(size);
+}
+
+static void *cds_lfht_calloc(void *state __attribute__((unused)),
+               size_t nmemb, size_t size)
+{
+       return calloc(nmemb, size);
+}
+
+static void *cds_lfht_realloc(void *state __attribute__((unused)),
+               void *ptr, size_t size)
+{
+       return realloc(ptr, size);
+}
+
+static void *cds_lfht_aligned_alloc(void *state __attribute__((unused)),
+               size_t alignment, size_t size)
+{
+       void *ptr;
+
+       if (posix_memalign(&ptr, alignment, size))
+               return NULL;
+       return ptr;
+}
+
+static void cds_lfht_free(void *state __attribute__((unused)), void *ptr)
+{
+       free(ptr);
+}
+
+
+/* Default memory allocator */
+static struct cds_lfht_alloc cds_lfht_default_alloc = {
+       .malloc = cds_lfht_malloc,
+       .calloc = cds_lfht_calloc,
+       .realloc = cds_lfht_realloc,
+       .aligned_alloc = cds_lfht_aligned_alloc,
+       .free = cds_lfht_free,
+       .state = NULL,
+};
+
 /*
  * Return the minimum order for which x <= (1UL << order).
  * Return -1 if x is 0.
@@ -623,9 +658,7 @@ static void mutex_lock(pthread_mutex_t *mutex)
                if (ret != EBUSY && ret != EINTR)
                        urcu_die(ret);
                if (CMM_LOAD_SHARED(URCU_TLS(rcu_reader).need_mb)) {
-                       cmm_smp_mb();
-                       _CMM_STORE_SHARED(URCU_TLS(rcu_reader).need_mb, 0);
-                       cmm_smp_mb();
+                       uatomic_store(&URCU_TLS(rcu_reader).need_mb, 0, CMM_SEQ_CST);
                }
                (void) poll(NULL, 0, 10);
        }
@@ -641,18 +674,17 @@ static void mutex_unlock(pthread_mutex_t *mutex)
                urcu_die(ret);
 }
 
-static long nr_cpus_mask = -1;
+static long nr_cpus_mask = NR_CPUS_MASK_UNINITIALIZED;
 static long split_count_mask = -1;
 static int split_count_order = -1;
 
-#if defined(HAVE_SYSCONF)
 static void ht_init_nr_cpus_mask(void)
 {
        long maxcpus;
 
-       maxcpus = sysconf(_SC_NPROCESSORS_CONF);
+       maxcpus = get_possible_cpus_array_len();
        if (maxcpus <= 0) {
-               nr_cpus_mask = -2;
+               nr_cpus_mask = NR_CPUS_MASK_INIT_FAILED;
                return;
        }
        /*
@@ -662,17 +694,11 @@ static void ht_init_nr_cpus_mask(void)
        maxcpus = 1UL << cds_lfht_get_count_order_ulong(maxcpus);
        nr_cpus_mask = maxcpus - 1;
 }
-#else /* #if defined(HAVE_SYSCONF) */
-static void ht_init_nr_cpus_mask(void)
-{
-       nr_cpus_mask = -2;
-}
-#endif /* #else #if defined(HAVE_SYSCONF) */
 
 static
 void alloc_split_items_count(struct cds_lfht *ht)
 {
-       if (nr_cpus_mask == -1) {
+       if (nr_cpus_mask == NR_CPUS_MASK_UNINITIALIZED) {
                ht_init_nr_cpus_mask();
                if (nr_cpus_mask < 0)
                        split_count_mask = DEFAULT_SPLIT_COUNT_MASK;
@@ -685,7 +711,7 @@ void alloc_split_items_count(struct cds_lfht *ht)
        urcu_posix_assert(split_count_mask >= 0);
 
        if (ht->flags & CDS_LFHT_ACCOUNTING) {
-               ht->split_count = calloc(split_count_mask + 1,
+               ht->split_count = ht->alloc->calloc(ht->alloc->state, split_count_mask + 1,
                                        sizeof(struct ht_items_count));
                urcu_posix_assert(ht->split_count);
        } else {
@@ -696,7 +722,7 @@ void alloc_split_items_count(struct cds_lfht *ht)
 static
 void free_split_items_count(struct cds_lfht *ht)
 {
-       poison_free(ht->split_count);
+       poison_free(ht->alloc, ht->split_count);
 }
 
 static
@@ -763,7 +789,7 @@ void ht_count_del(struct cds_lfht *ht, unsigned long size, unsigned long hash)
 
        if ((count >> CHAIN_LEN_RESIZE_THRESHOLD) >= size)
                return;
-       dbg_printf("del set global %ld\n", count);
+       dbg_printf("del set global %lu\n", count);
        /*
         * Don't shrink table if the number of nodes is below a
         * certain threshold.
@@ -851,6 +877,12 @@ int is_removal_owner(struct cds_lfht_node *node)
        return ((unsigned long) node) & REMOVAL_OWNER_FLAG;
 }
 
+static
+struct cds_lfht_node *flag_removed(struct cds_lfht_node *node)
+{
+       return (struct cds_lfht_node *) (((unsigned long) node) | REMOVED_FLAG);
+}
+
 static
 struct cds_lfht_node *flag_removal_owner(struct cds_lfht_node *node)
 {
@@ -884,8 +916,10 @@ unsigned long _uatomic_xchg_monotonic_increase(unsigned long *ptr,
        old1 = uatomic_read(ptr);
        do {
                old2 = old1;
-               if (old2 >= v)
+               if (old2 >= v) {
+                       cmm_smp_mb();
                        return old2;
+               }
        } while ((old1 = uatomic_cmpxchg(ptr, old2, v)) != old2);
        return old2;
 }
@@ -1169,6 +1203,7 @@ int _cds_lfht_del(struct cds_lfht *ht, unsigned long size,
                struct cds_lfht_node *node)
 {
        struct cds_lfht_node *bucket, *next;
+       uintptr_t *node_next;
 
        if (!node)      /* Return -ENOENT if asked to delete NULL node */
                return -ENOENT;
@@ -1191,15 +1226,18 @@ int _cds_lfht_del(struct cds_lfht *ht, unsigned long size,
        /*
         * The del operation semantic guarantees a full memory barrier
         * before the uatomic_or atomic commit of the deletion flag.
-        */
-       cmm_smp_mb__before_uatomic_or();
-       /*
+        *
         * We set the REMOVED_FLAG unconditionally. Note that there may
         * be more than one concurrent thread setting this flag.
         * Knowing which wins the race will be known after the garbage
         * collection phase, stay tuned!
+        *
+        * NOTE: The node_next variable is present to avoid breaking
+        * strict-aliasing rules.
         */
-       uatomic_or(&node->next, REMOVED_FLAG);
+       node_next = (uintptr_t*)&node->next;
+       uatomic_or_mo(node_next, REMOVED_FLAG, CMM_RELEASE);
+
        /* We performed the (logical) deletion. */
 
        /*
@@ -1224,7 +1262,7 @@ int _cds_lfht_del(struct cds_lfht *ht, unsigned long size,
         * was already set).
         */
        if (!is_removal_owner(uatomic_xchg(&node->next,
-                       flag_removal_owner(node->next))))
+                       flag_removal_owner(uatomic_load(&node->next, CMM_RELAXED)))))
                return 0;
        else
                return -ENOENT;
@@ -1251,8 +1289,9 @@ void partition_resize_helper(struct cds_lfht *ht, unsigned long i,
        struct partition_resize_work *work;
        int ret;
        unsigned long thread, nr_threads;
+       sigset_t newmask, oldmask;
 
-       urcu_posix_assert(nr_cpus_mask != -1);
+       urcu_posix_assert(nr_cpus_mask != NR_CPUS_MASK_UNINITIALIZED);
        if (nr_cpus_mask < 0 || len < 2 * MIN_PARTITION_PER_THREAD)
                goto fallback;
 
@@ -1268,18 +1307,25 @@ void partition_resize_helper(struct cds_lfht *ht, unsigned long i,
                nr_threads = 1;
        }
        partition_len = len >> cds_lfht_get_count_order_ulong(nr_threads);
-       work = calloc(nr_threads, sizeof(*work));
+       work = ht->alloc->calloc(ht->alloc->state, nr_threads, sizeof(*work));
        if (!work) {
                dbg_printf("error allocating for resize, single-threading\n");
                goto fallback;
        }
+
+       ret = sigfillset(&newmask);
+       urcu_posix_assert(!ret);
+       ret = pthread_sigmask(SIG_BLOCK, &newmask, &oldmask);
+       urcu_posix_assert(!ret);
+
        for (thread = 0; thread < nr_threads; thread++) {
                work[thread].ht = ht;
                work[thread].i = i;
                work[thread].len = partition_len;
                work[thread].start = thread * partition_len;
                work[thread].fct = fct;
-               ret = pthread_create(&(work[thread].thread_id), ht->resize_attr,
+               ret = pthread_create(&(work[thread].thread_id),
+                       ht->caller_resize_attr ? &ht->resize_attr : NULL,
                        partition_resize_thread, &work[thread]);
                if (ret == EAGAIN) {
                        /*
@@ -1294,11 +1340,15 @@ void partition_resize_helper(struct cds_lfht *ht, unsigned long i,
                }
                urcu_posix_assert(!ret);
        }
+
+       ret = pthread_sigmask(SIG_SETMASK, &oldmask, NULL);
+       urcu_posix_assert(!ret);
+
        for (thread = 0; thread < nr_threads; thread++) {
                ret = pthread_join(work[thread].thread_id, NULL);
                urcu_posix_assert(!ret);
        }
-       free(work);
+       ht->alloc->free(ht->alloc->state, work);
 
        /*
         * A pthread_create failure above will either lead in us having
@@ -1378,9 +1428,10 @@ void init_table(struct cds_lfht *ht,
 
                /*
                 * Update table size.
+                *
+                * Populate data before RCU size.
                 */
-               cmm_smp_wmb();  /* populate data before RCU size */
-               CMM_STORE_SHARED(ht->size, 1UL << i);
+               uatomic_store(&ht->size, 1UL << i, CMM_RELEASE);
 
                dbg_printf("init new size: %lu\n", 1UL << i);
                if (CMM_LOAD_SHARED(ht->in_progress_destroy))
@@ -1425,12 +1476,18 @@ void remove_table_partition(struct cds_lfht *ht, unsigned long i,
        for (j = size + start; j < size + start + len; j++) {
                struct cds_lfht_node *fini_bucket = bucket_at(ht, j);
                struct cds_lfht_node *parent_bucket = bucket_at(ht, j - size);
+               uintptr_t *fini_bucket_next;
 
                urcu_posix_assert(j >= size && j < (size << 1));
                dbg_printf("remove entry: order %lu index %lu hash %lu\n",
                           i, j, j);
-               /* Set the REMOVED_FLAG to freeze the ->next for gc */
-               uatomic_or(&fini_bucket->next, REMOVED_FLAG);
+               /* Set the REMOVED_FLAG to freeze the ->next for gc.
+                *
+                * NOTE: The fini_bucket_next variable is present to
+                * avoid breaking strict-aliasing rules.
+                */
+               fini_bucket_next = (uintptr_t*)&fini_bucket->next;
+               uatomic_or(fini_bucket_next, REMOVED_FLAG);
                _cds_lfht_gc_bucket(parent_bucket, fini_bucket);
        }
        ht->flavor->read_unlock();
@@ -1578,12 +1635,19 @@ const struct cds_lfht_mm_type *get_mm_type(
 }
 #endif
 
-struct cds_lfht *_cds_lfht_new(unsigned long init_size,
+void cds_lfht_node_init_deleted(struct cds_lfht_node *node)
+{
+       cds_lfht_node_init(node);
+       node->next = flag_removed(NULL);
+}
+
+struct cds_lfht *_cds_lfht_new_with_alloc(unsigned long init_size,
                        unsigned long min_nr_alloc_buckets,
                        unsigned long max_nr_buckets,
                        int flags,
                        const struct cds_lfht_mm_type *mm,
                        const struct rcu_flavor_struct *flavor,
+                       const struct cds_lfht_alloc *alloc,
                        pthread_attr_t *attr)
 {
        struct cds_lfht *ht;
@@ -1619,14 +1683,17 @@ struct cds_lfht *_cds_lfht_new(unsigned long init_size,
        max_nr_buckets = max(max_nr_buckets, min_nr_alloc_buckets);
        init_size = min(init_size, max_nr_buckets);
 
-       ht = mm->alloc_cds_lfht(min_nr_alloc_buckets, max_nr_buckets);
+       ht = mm->alloc_cds_lfht(min_nr_alloc_buckets, max_nr_buckets, alloc ? : &cds_lfht_default_alloc);
+
        urcu_posix_assert(ht);
        urcu_posix_assert(ht->mm == mm);
        urcu_posix_assert(ht->bucket_at == mm->bucket_at);
 
        ht->flags = flags;
        ht->flavor = flavor;
-       ht->resize_attr = attr;
+       ht->caller_resize_attr = attr;
+       if (attr)
+               ht->resize_attr = *attr;
        alloc_split_items_count(ht);
        /* this mutex should not nest in read-side C.S. */
        pthread_mutex_init(&ht->resize_mutex, NULL);
@@ -1637,6 +1704,19 @@ struct cds_lfht *_cds_lfht_new(unsigned long init_size,
        return ht;
 }
 
+struct cds_lfht *_cds_lfht_new(unsigned long init_size,
+                       unsigned long min_nr_alloc_buckets,
+                       unsigned long max_nr_buckets,
+                       int flags,
+                       const struct cds_lfht_mm_type *mm,
+                       const struct rcu_flavor_struct *flavor,
+                       pthread_attr_t *attr)
+{
+       return _cds_lfht_new_with_alloc(init_size,
+                       min_nr_alloc_buckets, max_nr_buckets,
+                       flags, mm, flavor, NULL, attr);
+}
+
 void cds_lfht_lookup(struct cds_lfht *ht, unsigned long hash,
                cds_lfht_match_fct match, const void *key,
                struct cds_lfht_iter *iter)
@@ -1648,7 +1728,14 @@ void cds_lfht_lookup(struct cds_lfht *ht, unsigned long hash,
 
        reverse_hash = bit_reverse_ulong(hash);
 
-       size = rcu_dereference(ht->size);
+       /*
+        * Use load acquire instead of rcu_dereference because there is no
+        * dependency between the table size and the dereference of the bucket
+        * content.
+        *
+        * This acquire is paired with the store release in init_table().
+        */
+       size = uatomic_load(&ht->size, CMM_ACQUIRE);
        bucket = lookup_bucket(ht, size, hash);
        /* We can always skip the bucket node initially */
        node = rcu_dereference(bucket->next);
@@ -1707,7 +1794,7 @@ void cds_lfht_next_duplicate(struct cds_lfht *ht __attribute__((unused)),
                }
                node = clear_flag(next);
        }
-       urcu_posix_assert(!node || !is_bucket(CMM_LOAD_SHARED(node->next)));
+       urcu_posix_assert(!node || !is_bucket(uatomic_load(&node->next, CMM_RELAXED)));
        iter->node = node;
        iter->next = next;
 }
@@ -1731,7 +1818,7 @@ void cds_lfht_next(struct cds_lfht *ht __attribute__((unused)),
                }
                node = clear_flag(next);
        }
-       urcu_posix_assert(!node || !is_bucket(CMM_LOAD_SHARED(node->next)));
+       urcu_posix_assert(!node || !is_bucket(uatomic_load(&node->next, CMM_RELAXED)));
        iter->node = node;
        iter->next = next;
 }
@@ -1743,7 +1830,7 @@ void cds_lfht_first(struct cds_lfht *ht, struct cds_lfht_iter *iter)
         * Get next after first bucket node. The first bucket node is the
         * first node of the linked list.
         */
-       iter->next = bucket_at(ht, 0)->next;
+       iter->next = uatomic_load(&bucket_at(ht, 0)->next, CMM_CONSUME);
        cds_lfht_next(ht, iter);
 }
 
@@ -1753,7 +1840,7 @@ void cds_lfht_add(struct cds_lfht *ht, unsigned long hash,
        unsigned long size;
 
        node->reverse_hash = bit_reverse_ulong(hash);
-       size = rcu_dereference(ht->size);
+       size = uatomic_load(&ht->size, CMM_ACQUIRE);
        _cds_lfht_add(ht, hash, NULL, NULL, size, node, NULL, 0);
        ht_count_add(ht, size, hash);
 }
@@ -1768,7 +1855,7 @@ struct cds_lfht_node *cds_lfht_add_unique(struct cds_lfht *ht,
        struct cds_lfht_iter iter;
 
        node->reverse_hash = bit_reverse_ulong(hash);
-       size = rcu_dereference(ht->size);
+       size = uatomic_load(&ht->size, CMM_ACQUIRE);
        _cds_lfht_add(ht, hash, match, key, size, node, &iter, 0);
        if (iter.node == node)
                ht_count_add(ht, size, hash);
@@ -1785,7 +1872,7 @@ struct cds_lfht_node *cds_lfht_add_replace(struct cds_lfht *ht,
        struct cds_lfht_iter iter;
 
        node->reverse_hash = bit_reverse_ulong(hash);
-       size = rcu_dereference(ht->size);
+       size = uatomic_load(&ht->size, CMM_ACQUIRE);
        for (;;) {
                _cds_lfht_add(ht, hash, match, key, size, node, &iter, 0);
                if (iter.node == node) {
@@ -1814,7 +1901,7 @@ int cds_lfht_replace(struct cds_lfht *ht,
                return -EINVAL;
        if (caa_unlikely(!match(old_iter->node, key)))
                return -EINVAL;
-       size = rcu_dereference(ht->size);
+       size = uatomic_load(&ht->size, CMM_ACQUIRE);
        return _cds_lfht_replace(ht, size, old_iter->node, old_iter->next,
                        new_node);
 }
@@ -1824,7 +1911,7 @@ int cds_lfht_del(struct cds_lfht *ht, struct cds_lfht_node *node)
        unsigned long size;
        int ret;
 
-       size = rcu_dereference(ht->size);
+       size = uatomic_load(&ht->size, CMM_ACQUIRE);
        ret = _cds_lfht_del(ht, size, node);
        if (!ret) {
                unsigned long hash;
@@ -1840,6 +1927,35 @@ int cds_lfht_is_node_deleted(const struct cds_lfht_node *node)
        return is_removed(CMM_LOAD_SHARED(node->next));
 }
 
+static
+bool cds_lfht_is_empty(struct cds_lfht *ht)
+{
+       struct cds_lfht_node *node, *next;
+       bool empty = true;
+       bool was_online;
+
+       was_online = ht->flavor->read_ongoing();
+       if (!was_online) {
+               ht->flavor->thread_online();
+               ht->flavor->read_lock();
+       }
+       /* Check that the table is empty */
+       node = bucket_at(ht, 0);
+       do {
+               next = rcu_dereference(node->next);
+               if (!is_bucket(next)) {
+                       empty = false;
+                       break;
+               }
+               node = clear_flag(next);
+       } while (!is_end(node));
+       if (!was_online) {
+               ht->flavor->read_unlock();
+               ht->flavor->thread_offline();
+       }
+       return empty;
+}
+
 static
 int cds_lfht_delete_bucket(struct cds_lfht *ht)
 {
@@ -1874,6 +1990,24 @@ int cds_lfht_delete_bucket(struct cds_lfht *ht)
        return 0;
 }
 
+static
+void do_auto_resize_destroy_cb(struct urcu_work *work)
+{
+       struct cds_lfht *ht = caa_container_of(work, struct cds_lfht, destroy_work);
+       int ret;
+
+       ht->flavor->register_thread();
+       ret = cds_lfht_delete_bucket(ht);
+       if (ret)
+               urcu_die(-ret);
+       free_split_items_count(ht);
+       ret = pthread_mutex_destroy(&ht->resize_mutex);
+       if (ret)
+               urcu_die(ret);
+       ht->flavor->unregister_thread();
+       poison_free(ht->alloc, ht);
+}
+
 /*
  * Should only be called when no more concurrent readers nor writers can
  * possibly access the table.
@@ -1883,23 +2017,39 @@ int cds_lfht_destroy(struct cds_lfht *ht, pthread_attr_t **attr)
        int ret;
 
        if (ht->flags & CDS_LFHT_AUTO_RESIZE) {
+               /*
+                * Perform error-checking for emptiness before queuing
+                * work, so we can return error to the caller. This runs
+                * concurrently with ongoing resize.
+                */
+               if (!cds_lfht_is_empty(ht))
+                       return -EPERM;
                /* Cancel ongoing resize operations. */
-               _CMM_STORE_SHARED(ht->in_progress_destroy, 1);
-               /* Wait for in-flight resize operations to complete */
-               urcu_workqueue_flush_queued_work(cds_lfht_workqueue);
+               uatomic_store(&ht->in_progress_destroy, 1, CMM_RELAXED);
+               if (attr) {
+                       *attr = ht->caller_resize_attr;
+                       ht->caller_resize_attr = NULL;
+               }
+               /*
+                * Queue destroy work after prior queued resize
+                * operations. Given there are no concurrent writers
+                * accessing the hash table at this point, no resize
+                * operations can be queued after this destroy work.
+                */
+               urcu_workqueue_queue_work(cds_lfht_workqueue,
+                       &ht->destroy_work, do_auto_resize_destroy_cb);
+               return 0;
        }
        ret = cds_lfht_delete_bucket(ht);
        if (ret)
                return ret;
        free_split_items_count(ht);
        if (attr)
-               *attr = ht->resize_attr;
+               *attr = ht->caller_resize_attr;
        ret = pthread_mutex_destroy(&ht->resize_mutex);
        if (ret)
                ret = -EBUSY;
-       if (ht->flags & CDS_LFHT_AUTO_RESIZE)
-               cds_lfht_fini_worker(ht->flavor);
-       poison_free(ht);
+       poison_free(ht->alloc, ht);
        return ret;
 }
 
@@ -1995,19 +2145,22 @@ void _do_cds_lfht_resize(struct cds_lfht *ht)
         * Resize table, re-do if the target size has changed under us.
         */
        do {
-               if (CMM_LOAD_SHARED(ht->in_progress_destroy))
+               if (uatomic_load(&ht->in_progress_destroy, CMM_RELAXED))
                        break;
-               ht->resize_initiated = 1;
+
+               uatomic_store(&ht->resize_initiated, 1, CMM_RELAXED);
+
                old_size = ht->size;
-               new_size = CMM_LOAD_SHARED(ht->resize_target);
+               new_size = uatomic_load(&ht->resize_target, CMM_RELAXED);
                if (old_size < new_size)
                        _do_cds_lfht_grow(ht, old_size, new_size);
                else if (old_size > new_size)
                        _do_cds_lfht_shrink(ht, old_size, new_size);
-               ht->resize_initiated = 0;
+
+               uatomic_store(&ht->resize_initiated, 0, CMM_RELAXED);
                /* write resize_initiated before read resize_target */
                cmm_smp_mb();
-       } while (ht->size != CMM_LOAD_SHARED(ht->resize_target));
+       } while (ht->size != uatomic_load(&ht->resize_target, CMM_RELAXED));
 }
 
 static
@@ -2028,7 +2181,12 @@ void resize_target_update_count(struct cds_lfht *ht,
 void cds_lfht_resize(struct cds_lfht *ht, unsigned long new_size)
 {
        resize_target_update_count(ht, new_size);
-       CMM_STORE_SHARED(ht->resize_initiated, 1);
+
+       /*
+        * Set flags has early as possible even in contention case.
+        */
+       uatomic_store(&ht->resize_initiated, 1, CMM_RELAXED);
+
        mutex_lock(&ht->resize_mutex);
        _do_cds_lfht_resize(ht);
        mutex_unlock(&ht->resize_mutex);
@@ -2046,7 +2204,7 @@ void do_resize_cb(struct urcu_work *work)
        _do_cds_lfht_resize(ht);
        mutex_unlock(&ht->resize_mutex);
        ht->flavor->unregister_thread();
-       poison_free(work);
+       poison_free(ht->alloc, work);
 }
 
 static
@@ -2054,13 +2212,15 @@ void __cds_lfht_resize_lazy_launch(struct cds_lfht *ht)
 {
        struct resize_work *work;
 
-       /* Store resize_target before read resize_initiated */
-       cmm_smp_mb();
-       if (!CMM_LOAD_SHARED(ht->resize_initiated)) {
-               if (CMM_LOAD_SHARED(ht->in_progress_destroy)) {
+       /*
+        * Store to resize_target is before read resize_initiated as guaranteed
+        * by either cmpxchg or _uatomic_xchg_monotonic_increase.
+        */
+       if (!uatomic_load(&ht->resize_initiated, CMM_RELAXED)) {
+               if (uatomic_load(&ht->in_progress_destroy, CMM_RELAXED)) {
                        return;
                }
-               work = malloc(sizeof(*work));
+               work = ht->alloc->malloc(ht->alloc->state, sizeof(*work));
                if (work == NULL) {
                        dbg_printf("error allocating resize work, bailing out\n");
                        return;
@@ -2068,7 +2228,7 @@ void __cds_lfht_resize_lazy_launch(struct cds_lfht *ht)
                work->ht = ht;
                urcu_workqueue_queue_work(cds_lfht_workqueue,
                        &work->work, do_resize_cb);
-               CMM_STORE_SHARED(ht->resize_initiated, 1);
+               uatomic_store(&ht->resize_initiated, 1, CMM_RELAXED);
        }
 }
 
@@ -2157,51 +2317,24 @@ static struct urcu_atfork cds_lfht_atfork = {
        .after_fork_child = cds_lfht_after_fork_child,
 };
 
-/*
- * Block all signals for the workqueue worker thread to ensure we don't
- * disturb the application. The SIGRCU signal needs to be unblocked for
- * the urcu-signal flavor.
- */
-static void cds_lfht_worker_init(
-               struct urcu_workqueue *workqueue __attribute__((unused)),
-               void *priv __attribute__((unused)))
-{
-       int ret;
-       sigset_t mask;
-
-       ret = sigfillset(&mask);
-       if (ret)
-               urcu_die(errno);
-       ret = sigdelset(&mask, SIGRCU);
-       if (ret)
-               urcu_die(errno);
-       ret = pthread_sigmask(SIG_SETMASK, &mask, NULL);
-       if (ret)
-               urcu_die(ret);
-}
-
 static void cds_lfht_init_worker(const struct rcu_flavor_struct *flavor)
 {
        flavor->register_rculfhash_atfork(&cds_lfht_atfork);
 
        mutex_lock(&cds_lfht_fork_mutex);
-       if (cds_lfht_workqueue_user_count++)
-               goto end;
-       cds_lfht_workqueue = urcu_workqueue_create(0, -1, NULL,
-               NULL, cds_lfht_worker_init, NULL, NULL, NULL, NULL, NULL);
-end:
+       if (!cds_lfht_workqueue)
+               cds_lfht_workqueue = urcu_workqueue_create(0, -1, NULL,
+                       NULL, NULL, NULL, NULL, NULL, NULL, NULL);
        mutex_unlock(&cds_lfht_fork_mutex);
 }
 
-static void cds_lfht_fini_worker(const struct rcu_flavor_struct *flavor)
+static void cds_lfht_exit(void)
 {
        mutex_lock(&cds_lfht_fork_mutex);
-       if (--cds_lfht_workqueue_user_count)
-               goto end;
-       urcu_workqueue_destroy(cds_lfht_workqueue);
-       cds_lfht_workqueue = NULL;
-end:
+       if (cds_lfht_workqueue) {
+               urcu_workqueue_flush_queued_work(cds_lfht_workqueue);
+               urcu_workqueue_destroy(cds_lfht_workqueue);
+               cds_lfht_workqueue = NULL;
+       }
        mutex_unlock(&cds_lfht_fork_mutex);
-
-       flavor->unregister_rculfhash_atfork(&cds_lfht_atfork);
 }
This page took 0.031693 seconds and 4 git commands to generate.