[ovs-dev] [PATCH v5 13/27] id-fpool: Module for fast ID generation

Gaetan Rivet grive at u256.net
Wed Sep 8 09:47:37 UTC 2021


The current id-pool module is slow to allocate the
next valid ID, and can be optimized when restricting
some properties of the pool.

Those restrictions are:

  * No ability to add a random ID to the pool.

  * A new ID is no more the smallest possible ID.
    It is however guaranteed to be in the range of

       [floor, last_alloc + nb_user * cache_size + 1].

    where 'cache_size' is the number of ID in each per-user
    cache.  It is defined as 'ID_FPOOL_CACHE_SIZE' to 64.

  * A user should never free an ID that is not allocated.
    No checks are done and doing so will duplicate the spurious
    ID.  Refcounting or other memory management scheme should
    be used to ensure an object and its ID are only freed once.

This allocator is designed to scale reasonably well in multithread
setup.  As it is aimed at being a faster replacement to the current
id-pool, a benchmark has been implemented alongside unit tests.

The benchmark is composed of 4 rounds: 'new', 'del', 'mix', and 'rnd'.
Respectively

  + 'new': only allocate IDs
  + 'del': only free IDs
  + 'mix': allocate, sequential free, then allocate ID.
  + 'rnd': allocate, random free, allocate ID.

Randomized freeing is done by swapping the latest allocated ID with any
from the range of currently allocated ID, which is reminiscent of the
Fisher-Yates shuffle.  This evaluates freeing non-sequential IDs,
which is the more natural use-case.

For this specific round, the id-pool performance is such that a timeout
of 10 seconds is added to the benchmark:

   $ ./tests/ovstest test-id-fpool benchmark 10000 1
   Benchmarking n=10000 on 1 thread.
    type\thread:       1    Avg
   id-fpool new:       1      1 ms
   id-fpool del:       1      1 ms
   id-fpool mix:       2      2 ms
   id-fpool rnd:       2      2 ms
    id-pool new:       4      4 ms
    id-pool del:       2      2 ms
    id-pool mix:       6      6 ms
    id-pool rnd:     431    431 ms

   $ ./tests/ovstest test-id-fpool benchmark 100000 1
   Benchmarking n=100000 on 1 thread.
    type\thread:       1    Avg
   id-fpool new:       2      2 ms
   id-fpool del:       2      2 ms
   id-fpool mix:       3      3 ms
   id-fpool rnd:       4      4 ms
    id-pool new:      12     12 ms
    id-pool del:       5      5 ms
    id-pool mix:      16     16 ms
    id-pool rnd:  10000+     -1 ms

   $ ./tests/ovstest test-id-fpool benchmark 1000000 1
   Benchmarking n=1000000 on 1 thread.
    type\thread:       1    Avg
   id-fpool new:      15     15 ms
   id-fpool del:      12     12 ms
   id-fpool mix:      34     34 ms
   id-fpool rnd:      48     48 ms
    id-pool new:     276    276 ms
    id-pool del:     286    286 ms
    id-pool mix:     448    448 ms
    id-pool rnd:  10000+     -1 ms

Running only a performance test on the fast pool:

   $ ./tests/ovstest test-id-fpool perf 1000000 1
   Benchmarking n=1000000 on 1 thread.
    type\thread:       1    Avg
   id-fpool new:      15     15 ms
   id-fpool del:      12     12 ms
   id-fpool mix:      34     34 ms
   id-fpool rnd:      47     47 ms

   $ ./tests/ovstest test-id-fpool perf 1000000 2
   Benchmarking n=1000000 on 2 threads.
    type\thread:       1      2    Avg
   id-fpool new:      11     11     11 ms
   id-fpool del:      10     10     10 ms
   id-fpool mix:      24     24     24 ms
   id-fpool rnd:      30     30     30 ms

   $ ./tests/ovstest test-id-fpool perf 1000000 4
   Benchmarking n=1000000 on 4 threads.
    type\thread:       1      2      3      4    Avg
   id-fpool new:       9     11     11     10     10 ms
   id-fpool del:       5      6      6      5      5 ms
   id-fpool mix:      16     16     16     16     16 ms
   id-fpool rnd:      20     20     20     20     20 ms

Signed-off-by: Gaetan Rivet <grive at u256.net>
Reviewed-by: Maxime Coquelin <maxime.coquelin at redhat.com>
---
 lib/automake.mk       |   2 +
 lib/id-fpool.c        | 279 +++++++++++++++++++
 lib/id-fpool.h        |  66 +++++
 tests/automake.mk     |   1 +
 tests/library.at      |   4 +
 tests/test-id-fpool.c | 615 ++++++++++++++++++++++++++++++++++++++++++
 6 files changed, 967 insertions(+)
 create mode 100644 lib/id-fpool.c
 create mode 100644 lib/id-fpool.h
 create mode 100644 tests/test-id-fpool.c

diff --git a/lib/automake.mk b/lib/automake.mk
index 098337078..ec1306b49 100644
--- a/lib/automake.mk
+++ b/lib/automake.mk
@@ -151,6 +151,8 @@ lib_libopenvswitch_la_SOURCES = \
 	lib/hmap.c \
 	lib/hmapx.c \
 	lib/hmapx.h \
+	lib/id-fpool.c \
+	lib/id-fpool.h \
 	lib/id-pool.c \
 	lib/id-pool.h \
 	lib/if-notifier-manual.c \
diff --git a/lib/id-fpool.c b/lib/id-fpool.c
new file mode 100644
index 000000000..15cef5d00
--- /dev/null
+++ b/lib/id-fpool.c
@@ -0,0 +1,279 @@
+/*
+ * Copyright (c) 2021 NVIDIA Corporation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <config.h>
+
+#include "openvswitch/list.h"
+#include "openvswitch/thread.h"
+#include "openvswitch/util.h"
+#include "ovs-atomic.h"
+#include "id-fpool.h"
+
+#ifdef HAVE_PTHREAD_SPIN_LOCK
+#define id_fpool_lock_type ovs_spin
+#define id_fpool_lock_init(l) do { ovs_spin_init(l); } while (0)
+#define id_fpool_lock_destroy(l) do { ovs_spin_destroy(l); } while (0)
+#define id_fpool_lock(l) do { ovs_spin_lock(l); } while (0)
+#define id_fpool_unlock(l) do { ovs_spin_unlock(l); } while (0)
+#else
+#define id_fpool_lock_type ovs_mutex
+#define id_fpool_lock_init(l) do { ovs_mutex_init(l); } while (0)
+#define id_fpool_lock_destroy(l) do { ovs_mutex_destroy(l); } while (0)
+#define id_fpool_lock(l) do { ovs_mutex_lock(l); } while (0)
+#define id_fpool_unlock(l) do { ovs_mutex_unlock(l); } while (0)
+#endif
+
+struct id_slab {
+    struct ovs_list node;
+    uint32_t pos;
+    uint32_t ids[ID_FPOOL_CACHE_SIZE];
+};
+
+struct per_user {
+PADDED_MEMBERS(CACHE_LINE_SIZE,
+    struct id_fpool_lock_type user_lock;
+    struct id_slab *slab;
+);};
+
+struct id_fpool {
+    /* Constants */
+    uint32_t floor; /* IDs are in the range of [floor, ceiling). */
+    uint32_t ceiling;
+    size_t nb_user; /* Number of concurrent users. */
+
+    /* Shared mutable data protected by global lock. */
+    struct id_fpool_lock_type pool_lock;
+    struct ovs_list free_slabs;
+    uint32_t next_id;
+
+    /* Per-user mutable data protected by user locks. */
+    struct per_user per_users[0];
+};
+
+/* Lock precedence is
+ * 1: per_users.user_lock
+ * 2: pool_lock
+ */
+
+static struct id_slab *
+id_slab_create(uint32_t *next_id, uint32_t max)
+{
+    struct id_slab *slab;
+    size_t n_ids;
+    size_t pos;
+
+    if (next_id[0] == max) {
+        return NULL;
+    }
+
+    n_ids = max - next_id[0];
+    slab = xmalloc(sizeof *slab);
+    ovs_list_init(&slab->node);
+    slab->pos = 0;
+
+    for (pos = MIN(n_ids, ARRAY_SIZE(slab->ids)); pos > 0; pos--) {
+        slab->ids[pos - 1] = next_id[0];
+        next_id[0]++;
+        slab->pos++;
+    }
+
+    return slab;
+}
+
+static bool
+id_slab_insert(struct id_slab *slab, uint32_t id)
+{
+    if (slab == NULL) {
+        return false;
+    }
+    if (slab->pos >= ARRAY_SIZE(slab->ids)) {
+        return false;
+    }
+    slab->ids[slab->pos++] = id;
+    return true;
+}
+
+static bool
+id_slab_remove(struct id_slab *slab, uint32_t *id)
+{
+    if (slab == NULL) {
+        return false;
+    }
+    if (slab->pos == 0) {
+        return false;
+    }
+    *id = slab->ids[--slab->pos];
+    return true;
+}
+
+static void
+per_user_init(struct per_user *pu, uint32_t *next_id, uint32_t max)
+{
+    id_fpool_lock_init(&pu->user_lock);
+    pu->slab = id_slab_create(next_id, max);
+}
+
+static void
+per_user_destroy(struct per_user *pu)
+{
+    id_fpool_lock(&pu->user_lock);
+    free(pu->slab);
+    pu->slab = NULL;
+    id_fpool_unlock(&pu->user_lock);
+    id_fpool_lock_destroy(&pu->user_lock);
+}
+
+struct id_fpool *
+id_fpool_create(unsigned int nb_user, uint32_t floor, uint32_t n_ids)
+{
+    struct id_fpool *pool;
+    size_t i;
+
+    ovs_assert(nb_user != 0);
+    ovs_assert(floor <= UINT32_MAX - n_ids);
+
+    pool = xmalloc(sizeof *pool + nb_user * sizeof(struct per_user));
+    pool->next_id = floor;
+    pool->floor = floor;
+    pool->ceiling = floor + n_ids;
+
+    for (i = 0; i < nb_user; i++) {
+        per_user_init(&pool->per_users[i],
+                      &pool->next_id, pool->ceiling);
+    }
+    pool->nb_user = nb_user;
+
+    id_fpool_lock_init(&pool->pool_lock);
+    ovs_list_init(&pool->free_slabs);
+
+    return pool;
+}
+
+void
+id_fpool_destroy(struct id_fpool *pool)
+{
+    struct id_slab *slab;
+    struct id_slab *next;
+    size_t i;
+
+    id_fpool_lock(&pool->pool_lock);
+    LIST_FOR_EACH_SAFE (slab, next, node, &pool->free_slabs) {
+        free(slab);
+    }
+    ovs_list_poison(&pool->free_slabs);
+    id_fpool_unlock(&pool->pool_lock);
+    id_fpool_lock_destroy(&pool->pool_lock);
+
+    for (i = 0; i < pool->nb_user; i++) {
+        per_user_destroy(&pool->per_users[i]);
+    }
+    free(pool);
+}
+
+bool
+id_fpool_new_id(struct id_fpool *pool, unsigned int uid, uint32_t *id)
+{
+    struct per_user *pu;
+    unsigned int uid2;
+    bool res = false;
+
+    ovs_assert(uid < pool->nb_user);
+    pu = &pool->per_users[uid];
+
+    id_fpool_lock(&pu->user_lock);
+
+    if (id_slab_remove(pu->slab, id)) {
+        res = true;
+        goto unlock_and_ret;
+    }
+    free(pu->slab);
+
+    id_fpool_lock(&pool->pool_lock);
+    if (!ovs_list_is_empty(&pool->free_slabs)) {
+        pu->slab = CONTAINER_OF(ovs_list_pop_front(&pool->free_slabs),
+                                struct id_slab, node);
+    } else {
+        pu->slab = id_slab_create(&pool->next_id, pool->ceiling);
+    }
+    id_fpool_unlock(&pool->pool_lock);
+
+    if (pu->slab != NULL) {
+        res = id_slab_remove(pu->slab, id);
+        goto unlock_and_ret;
+    }
+
+    id_fpool_unlock(&pu->user_lock);
+
+    /* No ID available in local slab, no slab available in shared list.
+     * The shared counter is maxed out. Attempt to steal an ID from another
+     * user slab. */
+
+    for (uid2 = 0; uid2 < pool->nb_user; uid2++) {
+        struct per_user *pu2 = &pool->per_users[uid2];
+
+        if (uid == uid2) {
+            continue;
+        }
+        id_fpool_lock(&pu2->user_lock);;
+        res = id_slab_remove(pu2->slab, id);
+        id_fpool_unlock(&pu2->user_lock);;
+        if (res) {
+            break;
+        }
+    }
+
+    goto out;
+
+unlock_and_ret:
+    id_fpool_unlock(&pu->user_lock);
+out:
+    return res;
+}
+
+void
+id_fpool_free_id(struct id_fpool *pool, unsigned int uid, uint32_t id)
+{
+    struct per_user *pu;
+
+    if (id < pool->floor || id >= pool->ceiling) {
+        return;
+    }
+
+    ovs_assert(uid < pool->nb_user);
+    pu = &pool->per_users[uid];
+
+    id_fpool_lock(&pu->user_lock);
+
+    if (pu->slab == NULL) {
+        /* Create local slab with a single ID. */
+        pu->slab = id_slab_create(&id, id + 1);
+        goto unlock;
+    }
+
+    if (id_slab_insert(pu->slab, id)) {
+        goto unlock;
+    }
+
+    id_fpool_lock(&pool->pool_lock);
+    ovs_list_push_back(&pool->free_slabs, &pu->slab->node);
+    id_fpool_unlock(&pool->pool_lock);
+
+    /* Create local slab with a single ID. */
+    pu->slab = id_slab_create(&id, id + 1);
+
+unlock:
+    id_fpool_unlock(&pu->user_lock);
+}
diff --git a/lib/id-fpool.h b/lib/id-fpool.h
new file mode 100644
index 000000000..f8d855938
--- /dev/null
+++ b/lib/id-fpool.h
@@ -0,0 +1,66 @@
+/*
+ * Copyright (c) 2021 NVIDIA Corporation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef ID_FPOOL_H
+#define ID_FPOOL_H
+
+#include <stdbool.h>
+#include <stddef.h>
+#include <stdint.h>
+
+/*
+ * Fast ID pool.
+ * =============
+ *
+ * Unordered pool of unique 32 bits IDs.
+ *
+ * Multiple users are registered at initialization.  Each one gets a cache
+ * of ID.  When each thread allocates from the pool using its own user ID,
+ * the pool scales for concurrent allocation.
+ *
+ * New IDs are always in the range of '[floor, next_id]', where 'next_id' is
+ * in the range of '[last_allocated_ID + nb_user * cache_size + 1]'.
+ * This means that a new ID is not always the smallest available ID, but it is
+ * still from a limited range.
+ *
+ * Users should ensure that an ID is *never* freed twice.  Not doing so will
+ * have the effect of double-allocating such ID afterward.
+ *
+ * Thread-safety
+ * =============
+ *
+ * APIs are thread safe.
+ * Multiple threads can share the same user ID if necessary.
+ */
+
+#define ID_FPOOL_CACHE_SIZE 64
+
+struct id_fpool;
+
+/* nb_user is the number of expected users of the pool,
+ * in terms of execution threads. */
+struct id_fpool *id_fpool_create(unsigned int nb_user,
+                                 uint32_t base, uint32_t n_ids);
+void id_fpool_destroy(struct id_fpool *pool);
+
+/* uid is the thread user-id. It should be within '[0, nb_user)'. */
+bool id_fpool_new_id(struct id_fpool *pool, unsigned int uid, uint32_t *id);
+
+/* uid is the thread user-id. It should be within '[0, nb_user)'.
+ * An allocated ID must never be freed twice. */
+void id_fpool_free_id(struct id_fpool *pool, unsigned int uid, uint32_t id);
+
+#endif  /* ID_FPOOL_H */
diff --git a/tests/automake.mk b/tests/automake.mk
index 8fdec27ef..8a9151f81 100644
--- a/tests/automake.mk
+++ b/tests/automake.mk
@@ -467,6 +467,7 @@ tests_ovstest_SOURCES = \
 	tests/test-heap.c \
 	tests/test-hindex.c \
 	tests/test-hmap.c \
+	tests/test-id-fpool.c \
 	tests/test-json.c \
 	tests/test-jsonrpc.c \
 	tests/test-list.c \
diff --git a/tests/library.at b/tests/library.at
index 661e95727..db4997d8f 100644
--- a/tests/library.at
+++ b/tests/library.at
@@ -270,3 +270,7 @@ AT_SETUP([mpsc-queue module])
 AT_CHECK([ovstest test-mpsc-queue check], [0], [....
 ])
 AT_CLEANUP
+
+AT_SETUP([id-fpool module])
+AT_CHECK([ovstest test-id-fpool check], [0], [])
+AT_CLEANUP
diff --git a/tests/test-id-fpool.c b/tests/test-id-fpool.c
new file mode 100644
index 000000000..25275d9ae
--- /dev/null
+++ b/tests/test-id-fpool.c
@@ -0,0 +1,615 @@
+/*
+ * Copyright (c) 2021 NVIDIA Corporation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#undef NDEBUG
+#include <assert.h>
+#include <getopt.h>
+#include <string.h>
+
+#include <config.h>
+
+#include "command-line.h"
+#include "id-fpool.h"
+#include "id-pool.h"
+#include "openvswitch/vlog.h"
+#include "openvswitch/util.h"
+#include "ovs-thread.h"
+#include "ovs-rcu.h"
+#include "ovs-numa.h"
+#include "ovstest.h"
+#include "random.h"
+#include "timeval.h"
+#include "util.h"
+
+static void
+test_id_fpool_alloc(void)
+{
+    const uint32_t base = 0;
+    const uint32_t n_id = 10;
+    struct id_fpool *pool = id_fpool_create(1, base, n_id);
+    uint32_t ids[10];
+    size_t i;
+
+    /* Can do n_id allocs. */
+    for (i = 0; i < n_id; i++) {
+        ovs_assert(id_fpool_new_id(pool, 0, &ids[i]));
+        ovs_assert(ids[i] >= base);
+        ovs_assert(ids[i] < base + n_id);
+    }
+    /* Only n_id successful allocations. */
+    ovs_assert(id_fpool_new_id(pool, 0, NULL) == false);
+
+    /* Monotonic alloc. */
+    for (i = 0; i < n_id - 1; i++) {
+        ovs_assert(ids[i] < ids[i + 1]);
+    }
+
+    for (i = 0; i < n_id; i++) {
+        id_fpool_free_id(pool, 0, ids[i]);
+    }
+
+    /* Can do n_id new allocs. */
+    for (i = 0; i < n_id; i++) {
+        ovs_assert(id_fpool_new_id(pool, 0, &ids[i]));
+        ovs_assert(ids[i] >= base);
+        ovs_assert(ids[i] < base + n_id);
+    }
+    /* Only n_id successful allocations. */
+    ovs_assert(id_fpool_new_id(pool, 0, NULL) == false);
+
+    for (i = 0; i < n_id; i++) {
+        id_fpool_free_id(pool, 0, ids[i]);
+    }
+
+    id_fpool_destroy(pool);
+}
+
+static void
+test_id_fpool_alloc_range(void)
+{
+    const uint32_t base = 200;
+    const uint32_t n_id = 100;
+    const uint32_t ceil = base + n_id;
+    struct id_fpool *pool = id_fpool_create(1, base, n_id);
+    bool id_allocated[100];
+    size_t i;
+
+    memset(id_allocated, 0, sizeof id_allocated);
+
+    /* Allocate all IDs only once. */
+    for (i = 0; i < n_id; i++) {
+        uint32_t id;
+
+        ovs_assert(id_fpool_new_id(pool, 0, &id));
+        ovs_assert(id >= base);
+        ovs_assert(id < ceil);
+
+        ovs_assert(id_allocated[id - base] == false);
+        id_allocated[id - base] = true;
+    }
+    /* Only n_id successful allocations. */
+    ovs_assert(id_fpool_new_id(pool, 0, NULL) == false);
+
+    for (i = 0; i < n_id; i++) {
+        ovs_assert(id_allocated[i]);
+        id_fpool_free_id(pool, 0, base + i);
+        id_allocated[i] = false;
+    }
+
+    /* The full range is again fully available. */
+    for (i = 0; i < n_id; i++) {
+        uint32_t id;
+
+        ovs_assert(id_fpool_new_id(pool, 0, &id));
+        ovs_assert(id >= base);
+        ovs_assert(id < ceil);
+
+        ovs_assert(id_allocated[id - base] == false);
+        id_allocated[id - base] = true;
+    }
+
+    id_fpool_destroy(pool);
+}
+
+static void
+test_id_fpool_alloc_steal(void)
+{
+    /* N must be less than a slab size to force the second user
+     * to steal from the first.
+     */
+#define N (ID_FPOOL_CACHE_SIZE / 2)
+    bool ids[N];
+    struct id_fpool *pool;
+    uint32_t id;
+    size_t i;
+
+    memset(ids, 0, sizeof ids);
+    pool = id_fpool_create(2, 0, N);
+
+    /* Fill up user 0 cache. */
+    ovs_assert(id_fpool_new_id(pool, 0, &id));
+    for (i = 0; i < N - 1; i++) {
+        /* Check that user 1 can still alloc from user 0 cache. */
+        ovs_assert(id_fpool_new_id(pool, 1, &id));
+    }
+
+    id_fpool_destroy(pool);
+}
+
+static void
+test_id_fpool_alloc_under_limit(void)
+{
+    const size_t n_id = 100;
+    uint32_t ids[100];
+    unsigned int limit;
+    struct id_fpool *pool;
+    size_t i;
+
+    memset(ids, 0, sizeof ids);
+    pool = id_fpool_create(1, 0, n_id);
+
+    for (limit = 1; limit < n_id; limit++) {
+        /* Allocate until arbitrary limit then free allocated ids. */
+        for (i = 0; i < limit; i++) {
+            ovs_assert(id_fpool_new_id(pool, 0, &ids[i]));
+        }
+        for (i = 0; i < limit; i++) {
+            id_fpool_free_id(pool, 0, ids[i]);
+        }
+        /* Verify that the N='limit' next allocations are under limit. */
+        for (i = 0; i < limit; i++) {
+            ovs_assert(id_fpool_new_id(pool, 0, &ids[i]));
+            ovs_assert(ids[i] < limit + ID_FPOOL_CACHE_SIZE);
+        }
+        for (i = 0; i < limit; i++) {
+            id_fpool_free_id(pool, 0, ids[i]);
+        }
+    }
+
+    id_fpool_destroy(pool);
+}
+
+static void
+run_tests(struct ovs_cmdl_context *ctx OVS_UNUSED)
+{
+    test_id_fpool_alloc();
+    test_id_fpool_alloc_range();
+    test_id_fpool_alloc_steal();
+    test_id_fpool_alloc_under_limit();
+}
+
+static uint32_t *ids;
+static uint64_t *thread_working_ms; /* Measured work time. */
+
+static unsigned int n_threads;
+static unsigned int n_ids;
+
+static struct ovs_barrier barrier;
+
+#define TIMEOUT_MS (10 * 1000) /* 10 sec timeout */
+static int running_time_ms;
+static volatile bool stop = false;
+
+static int
+elapsed(int *start)
+{
+    return running_time_ms - *start;
+}
+
+static void
+swap_u32(uint32_t *a, uint32_t *b)
+{
+    uint32_t t;
+    t = *a;
+    *a = *b;
+    *b = t;
+}
+
+static void
+shuffle(uint32_t *p, size_t n)
+{
+    for (; n > 1; n--, p++) {
+        uint32_t *q = &p[random_range(n)];
+        swap_u32(p, q);
+    }
+}
+
+static void
+print_result(const char *prefix)
+{
+    uint64_t avg;
+    size_t i;
+
+    avg = 0;
+    for (i = 0; i < n_threads; i++) {
+        avg += thread_working_ms[i];
+    }
+    avg /= n_threads;
+    printf("%s: ", prefix);
+    for (i = 0; i < n_threads; i++) {
+        if (thread_working_ms[i] >= TIMEOUT_MS) {
+            printf(" %5" PRIu64 "+", thread_working_ms[i]);
+        } else {
+            printf(" %6" PRIu64, thread_working_ms[i]);
+        }
+    }
+    if (avg >= TIMEOUT_MS) {
+        printf("     -1 ms\n");
+    } else {
+        printf(" %6" PRIu64 " ms\n", avg);
+    }
+}
+
+struct id_fpool_aux {
+    struct id_fpool *pool;
+    atomic_uint thread_id;
+};
+
+static void *
+id_fpool_thread(void *aux_)
+{
+    unsigned int n_ids_per_thread;
+    struct id_fpool_aux *aux = aux_;
+    uint32_t *th_ids;
+    unsigned int tid;
+    int start;
+    size_t i;
+
+    atomic_add(&aux->thread_id, 1u, &tid);
+    n_ids_per_thread = n_ids / n_threads;
+    th_ids = &ids[tid * n_ids_per_thread];
+
+    /* NEW / ALLOC */
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        ignore(id_fpool_new_id(aux->pool, tid, &th_ids[i]));
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    ovs_barrier_block(&barrier);
+
+    /* DEL */
+
+    shuffle(th_ids, n_ids_per_thread);
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        id_fpool_free_id(aux->pool, tid, th_ids[i]);
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    ovs_barrier_block(&barrier);
+
+    /* MIX */
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        ignore(id_fpool_new_id(aux->pool, tid, &th_ids[i]));
+        id_fpool_free_id(aux->pool, tid, th_ids[i]);
+        ignore(id_fpool_new_id(aux->pool, tid, &th_ids[i]));
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    ovs_barrier_block(&barrier);
+
+    /* Do not interfere with other threads still in 'MIX' phase. */
+    for (i = 0; i < n_ids_per_thread; i++) {
+        id_fpool_free_id(aux->pool, tid, th_ids[i]);
+    }
+
+    ovs_barrier_block(&barrier);
+
+    /* MIX SHUFFLED */
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        if (elapsed(&start) >= TIMEOUT_MS) {
+            break;
+        }
+        ignore(id_fpool_new_id(aux->pool, tid, &th_ids[i]));
+        swap_u32(&th_ids[i], &th_ids[random_range(i + 1)]);
+        id_fpool_free_id(aux->pool, tid, th_ids[i]);
+        ignore(id_fpool_new_id(aux->pool, tid, &th_ids[i]));
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    return NULL;
+}
+
+static void
+benchmark_id_fpool(void)
+{
+    pthread_t *threads;
+    struct id_fpool_aux aux;
+    size_t i;
+
+    memset(ids, 0, n_ids & sizeof *ids);
+    memset(thread_working_ms, 0, n_threads & sizeof *thread_working_ms);
+
+    aux.pool = id_fpool_create(n_threads, 0, n_ids);
+    atomic_store(&aux.thread_id, 0);
+
+    for (i = n_ids - (n_ids % n_threads); i < n_ids; i++) {
+        id_fpool_new_id(aux.pool, 0, &ids[i]);
+    }
+
+    threads = xmalloc(n_threads * sizeof *threads);
+    ovs_barrier_init(&barrier, n_threads + 1);
+
+    for (i = 0; i < n_threads; i++) {
+        threads[i] = ovs_thread_create("id_fpool_alloc",
+                                       id_fpool_thread, &aux);
+    }
+
+    ovs_barrier_block(&barrier);
+
+    print_result("id-fpool new");
+
+    ovs_barrier_block(&barrier);
+
+    print_result("id-fpool del");
+
+    ovs_barrier_block(&barrier);
+    /* Cleanup. */
+    ovs_barrier_block(&barrier);
+
+    print_result("id-fpool mix");
+
+    for (i = 0; i < n_threads; i++) {
+        xpthread_join(threads[i], NULL);
+    }
+
+    print_result("id-fpool rnd");
+
+    id_fpool_destroy(aux.pool);
+    ovs_barrier_destroy(&barrier);
+    free(threads);
+}
+
+struct id_pool_aux {
+    struct id_pool *pool;
+    struct ovs_mutex *lock;
+    atomic_uint thread_id;
+};
+
+static void *
+id_pool_thread(void *aux_)
+{
+    unsigned int n_ids_per_thread;
+    struct id_pool_aux *aux = aux_;
+    uint32_t *th_ids;
+    unsigned int tid;
+    int start;
+    size_t i;
+
+    atomic_add(&aux->thread_id, 1u, &tid);
+    n_ids_per_thread = n_ids / n_threads;
+    th_ids = &ids[tid * n_ids_per_thread];
+
+    /* NEW */
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        ovs_mutex_lock(aux->lock);
+        ovs_assert(id_pool_alloc_id(aux->pool, &th_ids[i]));
+        ovs_mutex_unlock(aux->lock);
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    ovs_barrier_block(&barrier);
+
+    /* DEL */
+
+    shuffle(th_ids, n_ids_per_thread);
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        ovs_mutex_lock(aux->lock);
+        id_pool_free_id(aux->pool, th_ids[i]);
+        ovs_mutex_unlock(aux->lock);
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    ovs_barrier_block(&barrier);
+
+    /* MIX */
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        ovs_mutex_lock(aux->lock);
+        ignore(id_pool_alloc_id(aux->pool, &th_ids[i]));
+        id_pool_free_id(aux->pool, th_ids[i]);
+        ignore(id_pool_alloc_id(aux->pool, &th_ids[i]));
+        ovs_mutex_unlock(aux->lock);
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    ovs_barrier_block(&barrier);
+
+    /* Do not interfere with other threads still in 'MIX' phase. */
+    ovs_mutex_lock(aux->lock);
+    for (i = 0; i < n_ids_per_thread; i++) {
+        id_pool_free_id(aux->pool, th_ids[i]);
+    }
+    ovs_mutex_unlock(aux->lock);
+
+    ovs_barrier_block(&barrier);
+
+    /* MIX SHUFFLED */
+
+    start = running_time_ms;
+    for (i = 0; i < n_ids_per_thread; i++) {
+        if (elapsed(&start) >= TIMEOUT_MS) {
+            break;
+        }
+        ovs_mutex_lock(aux->lock);
+        ignore(id_pool_alloc_id(aux->pool, &th_ids[i]));
+        swap_u32(&th_ids[i], &th_ids[random_range(i + 1)]);
+        id_pool_free_id(aux->pool, th_ids[i]);
+        ignore(id_pool_alloc_id(aux->pool, &th_ids[i]));
+        ovs_mutex_unlock(aux->lock);
+    }
+    thread_working_ms[tid] = elapsed(&start);
+
+    return NULL;
+}
+
+OVS_UNUSED
+static void
+benchmark_id_pool(void)
+{
+    pthread_t *threads;
+    struct id_pool_aux aux;
+    struct ovs_mutex lock;
+    size_t i;
+
+    memset(ids, 0, n_ids & sizeof *ids);
+    memset(thread_working_ms, 0, n_threads & sizeof *thread_working_ms);
+
+    aux.pool = id_pool_create(0, n_ids);
+    aux.lock = &lock;
+    ovs_mutex_init(&lock);
+    atomic_store(&aux.thread_id, 0);
+
+    for (i = n_ids - (n_ids % n_threads); i < n_ids; i++) {
+        id_pool_alloc_id(aux.pool, &ids[i]);
+    }
+
+    threads = xmalloc(n_threads * sizeof *threads);
+    ovs_barrier_init(&barrier, n_threads + 1);
+
+    for (i = 0; i < n_threads; i++) {
+        threads[i] = ovs_thread_create("id_pool_alloc", id_pool_thread, &aux);
+    }
+
+    ovs_barrier_block(&barrier);
+
+    print_result(" id-pool new");
+
+    ovs_barrier_block(&barrier);
+
+    print_result(" id-pool del");
+
+    ovs_barrier_block(&barrier);
+    /* Cleanup. */
+    ovs_barrier_block(&barrier);
+
+    print_result(" id-pool mix");
+
+    for (i = 0; i < n_threads; i++) {
+        xpthread_join(threads[i], NULL);
+    }
+
+    print_result(" id-pool rnd");
+
+    id_pool_destroy(aux.pool);
+    ovs_barrier_destroy(&barrier);
+    free(threads);
+}
+
+static void *
+clock_main(void *arg OVS_UNUSED)
+{
+    struct timeval start;
+    struct timeval end;
+
+    xgettimeofday(&start);
+    while (!stop) {
+        xgettimeofday(&end);
+        running_time_ms = timeval_to_msec(&end) - timeval_to_msec(&start);
+        xnanosleep(1000);
+    }
+
+    return NULL;
+}
+
+static void
+do_perf_test(struct ovs_cmdl_context *ctx, bool test_id_pool)
+{
+    pthread_t clock;
+    long int l_threads;
+    long int l_ids;
+    size_t i;
+
+    l_ids = strtol(ctx->argv[1], NULL, 10);
+    l_threads = strtol(ctx->argv[2], NULL, 10);
+    ovs_assert(l_ids > 0 && l_threads > 0);
+
+    n_ids = l_ids;
+    n_threads = l_threads;
+
+    ids = xcalloc(n_ids, sizeof *ids);
+    thread_working_ms = xcalloc(n_threads, sizeof *thread_working_ms);
+
+    clock = ovs_thread_create("clock", clock_main, NULL);
+
+    printf("Benchmarking n=%u on %u thread%s.\n", n_ids, n_threads,
+           n_threads > 1 ? "s" : "");
+
+    printf(" type\\thread:  ");
+    for (i = 0; i < n_threads; i++) {
+        printf("   %3" PRIuSIZE " ", i + 1);
+    }
+    printf("   Avg\n");
+
+    ovsrcu_quiesce_start();
+
+    benchmark_id_fpool();
+    if (test_id_pool) {
+        benchmark_id_pool();
+    }
+
+    stop = true;
+
+    free(thread_working_ms);
+    xpthread_join(clock, NULL);
+}
+
+static void
+run_benchmark(struct ovs_cmdl_context *ctx)
+{
+    do_perf_test(ctx, true);
+}
+
+static void
+run_perf(struct ovs_cmdl_context *ctx)
+{
+    do_perf_test(ctx, false);
+}
+
+static const struct ovs_cmdl_command commands[] = {
+    {"check", NULL, 0, 0, run_tests, OVS_RO},
+    {"benchmark", "<nb elem> <nb threads>", 2, 2, run_benchmark, OVS_RO},
+    {"perf", "<nb elem> <nb threads>", 2, 2, run_perf, OVS_RO},
+    {NULL, NULL, 0, 0, NULL, OVS_RO},
+};
+
+static void
+id_fpool_test_main(int argc, char *argv[])
+{
+    struct ovs_cmdl_context ctx = {
+        .argc = argc - optind,
+        .argv = argv + optind,
+    };
+
+    vlog_set_levels(NULL, VLF_ANY_DESTINATION, VLL_OFF);
+
+    set_program_name(argv[0]);
+    ovs_cmdl_run_command(&ctx, commands);
+}
+
+OVSTEST_REGISTER("test-id-fpool", id_fpool_test_main);
-- 
2.31.1



More information about the dev mailing list