DPDK patches and discussions
 help / color / mirror / Atom feed
* [RFC 0/6] Stage-Ordered API and other extensions for ring library
@ 2024-08-15  8:53 Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
                   ` (6 more replies)
  0 siblings, 7 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Konstantin Ananyev (6):
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  examples/l3fwd: make ACL work in pipeline and eventdev modes
  ring: minimize reads of the counterpart cache-line

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- next two patches introduce SORING API into the ring library and
  provide UT for it.
- patch #5 extends l3fwd sample app to work in pipeline (worker-pool) mode. 
  Right now it is done for demonstration and performance comparison pruposes:
  it makes possible to run l3fwd in different modes:
  run-to-completion, eventdev, pipeline
  and perform sort-of 'apple-to-apple' performance comparisons.
  I am aware that in general community consensus on l3fwd is to keep its
  functionality simple and limited. From other side we already do have
  eventdev  mode for it, so why pipeline should be prohibited?
  Though if l3fwd is not an option, then we need to select some other
  existing sample app to integrate with. Probably ipsec-secgw would be the
  second best choice from my perspective, though it would require much more
  effort.
  Have to say that current l3fwd patch is way too big and unfinished,
  so if we'll decide to go forward with it, it has to be split and reworked.   
- patch #6 - attempt to optimize (by caching counter-part tail value)
  enqueue/dequeue operations for vanilla rte_ring. Logically tt is not linked
  with patches 3-5 and probably should be in a separate series.
  I put it here for now just to minimize 'Depends-on' hassle, so everyone
  can build/try everything in one go.

Seeking community help/feedback (apart from usual patch review activity):
=========================================================================
- While we tested these changes quite extensively, our platform coverage
  is limited to x86 right now.
  So would appreciate the feedback how it behaves on other architectures
  DPDK supports (ARM, PPC, etc.).
  Specially for patch #6: so far we didn't observe noticeable performance
  improvement with it on x86_64,
  So if there would be no real gain on other platforms (or scenarios) - 
  I am ok to drop that patch.
- Adding new (pipeline) mode for l3fwd sample app.
  Is it worth it? If not, what other sample app should be used to
  demonstrate new functionality we worked on? ipsec-secgw? Something else?  

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK rte_ring,
re-uses many of its concepts, and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' - given thread
  can safely assume that it has exclusive ownership of these objects till
  it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.  
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.     
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz, 
l3fwd (--lookup=acl) in pipeline mode (see patch #5 for details) both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf

 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 452 ++++++++++++++++
 app/test/test_soring_mt_stress.c   |  45 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 832 +++++++++++++++++++++++++++++
 drivers/net/mlx5/mlx5_hws_cnt.h    |   5 +-
 drivers/net/ring/rte_eth_ring.c    |   2 +-
 examples/l3fwd/l3fwd.h             |  55 ++
 examples/l3fwd/l3fwd_acl.c         | 125 ++++-
 examples/l3fwd/l3fwd_acl_event.h   | 258 +++++++++
 examples/l3fwd/l3fwd_event.c       |  14 +
 examples/l3fwd/l3fwd_event.h       |   1 +
 examples/l3fwd/l3fwd_sse.h         |  49 +-
 examples/l3fwd/l3fwd_wqp.c         | 274 ++++++++++
 examples/l3fwd/l3fwd_wqp.h         | 132 +++++
 examples/l3fwd/main.c              |  75 ++-
 examples/l3fwd/meson.build         |   1 +
 lib/ring/meson.build               |   4 +-
 lib/ring/rte_ring.c                |   6 +-
 lib/ring/rte_ring_c11_pvt.h        | 134 +----
 lib/ring/rte_ring_core.h           |  12 +-
 lib/ring/rte_ring_elem_pvt.h       | 183 +++++--
 lib/ring/rte_ring_generic_pvt.h    | 153 ++----
 lib/ring/rte_ring_hts_elem_pvt.h   |  85 +--
 lib/ring/rte_ring_peek_elem_pvt.h  |   4 +-
 lib/ring/rte_ring_rts_elem_pvt.h   |  85 +--
 lib/ring/rte_soring.c              | 144 +++++
 lib/ring/rte_soring.h              | 270 ++++++++++
 lib/ring/soring.c                  | 446 ++++++++++++++++
 lib/ring/soring.h                  | 124 +++++
 lib/ring/version.map               |  13 +
 32 files changed, 3632 insertions(+), 437 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 examples/l3fwd/l3fwd_acl_event.h
 create mode 100644 examples/l3fwd/l3fwd_wqp.c
 create mode 100644 examples/l3fwd/l3fwd_wqp.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFC 1/6] ring: common functions for 'move head' ops
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-08-15  8:53 ` Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 2/6] ring: make copying functions generic Konstantin Ananyev
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFC 2/6] ring: make copying functions generic
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-08-15  8:53 ` Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 3/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 117 ++++++++++++++++++++---------------
 1 file changed, 68 insertions(+), 49 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..216cb6089f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
-	uint32_t *obj = (uint32_t *)obj_table;
+	uint32_t *obj = obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFC 3/6] ring/soring: introduce Staged Ordered Ring
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 2/6] ring: make copying functions generic Konstantin Ananyev
@ 2024-08-15  8:53 ` Konstantin Ananyev
  2024-08-15 11:11   ` Morten Brørup
  2024-08-26 19:04   ` Mattias Rönnblom
  2024-08-15  8:53 ` [RFC 4/6] app/test: add unit tests for soring API Konstantin Ananyev
                   ` (3 subsequent siblings)
  6 siblings, 2 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/meson.build  |   4 +-
 lib/ring/rte_soring.c | 144 ++++++++++++++
 lib/ring/rte_soring.h | 270 ++++++++++++++++++++++++++
 lib/ring/soring.c     | 431 ++++++++++++++++++++++++++++++++++++++++++
 lib/ring/soring.h     | 124 ++++++++++++
 lib/ring/version.map  |  13 ++
 6 files changed, 984 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..17b1b73a42
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,144 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->esize, prm->stsize, count, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->esize, prm->stsize, count, prm->stages,
+			NULL, NULL, NULL);
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t elst_ofs, stage_ofs, state_ofs;
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->esize, prm->stsize, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->esize, prm->stsize, n, prm->stages, &elst_ofs,
+			&state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->esize;
+	r->stsize = prm->stsize;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->stsize != 0)
+		r->elemst = (void *)((uintptr_t)r + elst_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..fb0e75b39a
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,270 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/* max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the ring */
+	const char *name;
+	/** number of elemnts in the ring */
+	uint32_t elems;
+	/** size of elements in the ring, must be a multiple of 4 */
+	uint32_t esize;
+	/**
+	 * size of retcode for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of ret-codes associated with each object in the soring.
+	 * While element size is configurable (see @esize parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * ret-code together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to ret-code array
+	 * dequeue() - reads both objects and ret-code array
+	 */
+	uint32_t stsize;
+	/** number of stages in the ring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a ring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual ring elements and theri rec-codes. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use rte_ring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param obj_table
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as @esize parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param objst
+ *   A pointer to an array of status values for each object to enqueue.
+ *   Note that if user not using object status values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @stsize parameter
+ *   used while creating the ring. If user created the soring with
+ *   @stsize value equals zero, then @objst parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the obj_table.
+ * @param behavior
+ *   Behavior type, one of:
+ *   - RTE_RING_QUEUE_FIXED: enqueue either exactly @n objects or none.
+ *   - RTE_RING_QUEUE_VARIABLE: enqueue up to @n objects.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue(struct rte_soring *r, const void *obj_table,
+	const void *objst, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param obj_table
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as @esize parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param objst
+ *   A pointer to array of status values for each object to dequeue.
+ *   Note that if user not using object status values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @stsize parameter
+ *   used while creating the ring. If user created the soring with
+ *   @stsize value equals zero, then @objst parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the obj_table.
+ * @param behavior
+ *   Behavior type, one of:
+ *   - RTE_RING_QUEUE_FIXED: dequeue either exactly @n objects or none.
+ *   - RTE_RING_QUEUE_VARIABLE: dequeue up to @n objects.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue(struct rte_soring *r, void *obj_table, void *objst,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as @esize parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param objst
+ *   A pointer to an array of status values for each for each acquired object.
+ *   Note that if user not using object status values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @stsize parameter
+ *   used while creating the ring. If user created the soring with
+ *   @stsize value equals zero, then @objst parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param behavior
+ *   Behavior type, one of:
+ *   - RTE_RING_QUEUE_FIXED: acquire either exactly @n objects or none.
+ *   - RTE_RING_QUEUE_VARIABLE: acquire up to @n objects.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire(struct rte_soring *r, void *objs, void *objst,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as @esize parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param objst
+ *   A pointer to an array of status values for each object to release.
+ *   Note that if user not using object status values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @stsize parameter
+ *   used while creating the ring. If user created the soring with
+ *   @stsize value equals zero, then objst parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ * @return
+ *   - None.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	const void *objst, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..929bde9697
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,431 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (RTE_SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'RTE_SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in RTE_SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		&ot.raw, nt.raw, rte_memory_order_acquire,
+		rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & RTE_SORING_ST_MASK) != RTE_SORING_ST_FINISH ||
+				st.ftoken != tail)
+			break;
+
+		k = st.stnum & ~RTE_SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_acquire);
+
+	do {
+		n = num;
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_relaxed);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
+
+	return n;
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+uint32_t
+rte_soring_enqueue(struct rte_soring *r, const void *obj_table,
+	const void *objst, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(objst == NULL || r->elemst != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (objst != NULL)
+			__rte_ring_do_enqueue_elems(r->elemst, objst, r->size,
+				prod_head & r->mask, r->stsize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+uint32_t
+rte_soring_dequeue(struct rte_soring *r, void *obj_table, void *objst,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(objst == NULL || r->elemst != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (objst != NULL)
+			__rte_ring_do_dequeue_elems(objst, r->elemst, r->size,
+				cons_head & r->mask, r->stsize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+uint32_t
+rte_soring_acquire(struct rte_soring *r, void *objs, void *objst,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	union soring_state st;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(objst == NULL || r->elemst != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, r->state,
+				r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (objst != NULL)
+			__rte_ring_do_dequeue_elems(objst, r->elemst,
+				r->size, idx, r->stsize, n);
+
+		/* update status ring */
+		st.ftoken = head;
+		st.stnum = (RTE_SORING_ST_START | n);
+
+		rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+				rte_memory_order_relaxed);
+		*ftoken = head;
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	const void *objst, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(objst == NULL || r->elemst != NULL);
+
+	stg = r->stage + stage;
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+
+	idx = ftoken & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_acquire);
+
+	/* check state ring contents */
+	RTE_VERIFY(st.stnum == (RTE_SORING_ST_START | n) &&
+		st.ftoken == ftoken);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (objst != NULL)
+		__rte_ring_do_enqueue_elems(r->elemst, objst, r->size, idx,
+			r->stsize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	st.stnum = RTE_SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_release);
+
+	if (tail == ftoken)
+		__rte_soring_stage_finalize(&stg->sht, r->state, r->mask,
+				r->capacity);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..3a3f6efa76
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,124 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_START	RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define RTE_SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define RTE_SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t stsize;
+	/**< size of status value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where status values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *elemst;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..a1f95a500f 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,16 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_soring_acquire;
+	rte_soring_count;
+	rte_soring_dequeue;
+	rte_soring_enqueue;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFC 4/6] app/test: add unit tests for soring API
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                   ` (2 preceding siblings ...)
  2024-08-15  8:53 ` [RFC 3/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-08-15  8:53 ` Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 5/6] examples/l3fwd: make ACL work in pipeline and eventdev modes Konstantin Ananyev
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev, Eimear Morrissey

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 452 ++++++++++++++++
 app/test/test_soring_mt_stress.c   |  45 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 832 +++++++++++++++++++++++++++++
 6 files changed, 1415 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index e29258e6ec..c290162e43 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -175,6 +175,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..381979bc6f
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,452 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->esize = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->stsize = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire(sor, acquired_objs, NULL, stage,
+			num_packets, RTE_RING_QUEUE_FIXED, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueue(sor, objs, rcs, 32,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 32,
+				RTE_RING_QUEUE_FIXED, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquire(sor, acquired_objs,
+				acquired_rcs, stage, 32, RTE_RING_QUEUE_FIXED,
+				&ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_release(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, dequeued_rcs, 32,
+			RTE_RING_QUEUE_FIXED, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue(sor, queue_objs_p, NULL, 5,
+			RTE_RING_QUEUE_VARIABLE, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue(sor, queue_objs_p, NULL, 7,
+			RTE_RING_QUEUE_FIXED, &free_space);
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue(sor, queue_objs_p + 5, NULL, 7,
+			RTE_RING_QUEUE_VARIABLE, &free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 10,
+			RTE_RING_QUEUE_FIXED, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 10,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 10,
+			RTE_RING_QUEUE_FIXED, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 10,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 1,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, NULL, 2,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueue(sor, queue_objs, rc_objs, 5,
+			RTE_RING_QUEUE_VARIABLE, &free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue(sor, queue_objs + 5, NULL, 5,
+			RTE_RING_QUEUE_VARIABLE, &free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquire(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, RTE_RING_QUEUE_VARIABLE, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquire(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, RTE_RING_QUEUE_VARIABLE,
+			&ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_release(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquire(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, RTE_RING_QUEUE_VARIABLE,
+			&ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeue(sor, dequeued_objs, dequeued_rcs, 10,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..f18ea54444
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,45 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_bulk(struct rte_soring *r, void **obj, uint32_t n,
+	enum rte_ring_queue_behavior bhv, uint32_t *avail)
+{
+	return rte_soring_dequeue(r, obj, NULL, n, bhv, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	enum rte_ring_queue_behavior bhv, uint32_t *free)
+{
+	return rte_soring_enqueue(r, obj, NULL, n, bhv, free);
+}
+
+static inline uint32_t
+_st_ring_stage_acquire(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, enum rte_ring_queue_behavior bhv, uint32_t *token,
+	uint32_t *avail)
+{
+	return rte_soring_acquire(r, obj, NULL, stage, num, bhv,
+			token, avail);
+}
+
+static inline void
+_st_ring_stage_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, NULL, stage, num, token);
+}
+
+static const enum rte_ring_queue_behavior ring_behavior =
+	RTE_RING_QUEUE_VARIABLE;
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..a64c9eaaea
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,832 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static const enum rte_ring_queue_behavior ring_behavior;
+
+static uint32_t
+_st_ring_dequeue_bulk(struct rte_soring *r, void **obj, uint32_t n,
+	enum rte_ring_queue_behavior bhv, uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	enum rte_ring_queue_behavior bhv, uint32_t *free);
+
+static uint32_t
+_st_ring_stage_acquire(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, enum rte_ring_queue_behavior bhv, uint32_t *token,
+	uint32_t *avail);
+
+static void
+_st_ring_stage_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill)
+{
+	uint32_t i;
+
+	static rte_spinlock_t dump_lock;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				__func__, rte_lcore_id(), num, i, elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname)
+{
+	if (bhv == RTE_RING_QUEUE_FIXED && exp != res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	if (bhv == RTE_RING_QUEUE_VARIABLE && exp < res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM], enum rte_ring_queue_behavior bhv,
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_stage_acquire(la->rng, i, (void **)obj, num,
+				bhv, &tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, bhv, fname,
+			RTE_STR(_st_ring_stage_acquire));
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_stage_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM], enum rte_ring_queue_behavior bhv,
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_bulk(la->rng, (void **)obj, num, bhv, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, bhv, fname,
+			RTE_STR(_st_ring_dequeue_bulk));
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n,
+			RTE_RING_QUEUE_FIXED, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk));
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				ring_behavior, &def_elm, &loc_elm, stg_elm,
+				prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				ring_behavior, &def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.esize = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, RTE_RING_QUEUE_FIXED,
+				NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFC 5/6] examples/l3fwd: make ACL work in pipeline and eventdev modes
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                   ` (3 preceding siblings ...)
  2024-08-15  8:53 ` [RFC 4/6] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-08-15  8:53 ` Konstantin Ananyev
  2024-08-15  8:53 ` [RFC 6/6] ring: minimize reads of the counterpart cache-line Konstantin Ananyev
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront:
This is a huge commit that is combined from several ones.
For now, I submit it just for reference and demonstration purposes and
will probably remove it in future versions.
If will decide to go ahead with it, then it needs to be reworked and split
into several proper commits.

It adds for l3fwd:
 - eventdev mode for ACL lookup-mode
 - Introduce a worker-pool-mode
   (right now implemented for ACL lookup-mode only).
Worker-Pool mode is a simple pipeline model, with the following stages:
 1) I/O thread receives packets from NIC RX HW queues and enqueues them
    into the work queue
 2) Worker thread reads packets from the work queue(s),
    process them and then puts processed packets back into the
    work queue along with the processing status (routing info/error code).
 3) I/O thread dequeues packets and their status from the work queue,
    and based on it either TX packet or drops it.
Very similar to l3fwd-eventdev working model.

Note that it could be several I/O threads, each can serve one or multiple
HW RX queues. Also there could be several Worker threads, each of them can
process packets from multiple work queues in round-robin fashion.

Work queue can be one of the following types:
 - wqorder: allows Worker threads to process packets in any order,
   but guarantees that on dequeue stage the ingress order of packets
   will be preserved. I.E. at stage #3, I/O thread will get packets
   exactly in the same order as they were enqueued at stage #1.
 - wqunorder: doesn't provide any ordered guarantees.

'wqunroder' mode is implemented using 2 rte_ring structures per queue.
'wqorder' mode is implemtened using rte_soring structure per queue.

To facilitate this new functionality, command line parameters were
extended:
 --mode:
   Possible values one of: poll/eventdev/wqorder/wqorderS/wqunorder/wqunorderS
   Default value: poll
   - wqorder: Worker-Pool ordered mode with a separate work queue for each
     HW RX queue.
   - wqorderS: Worker-Pool ordered mode with one work queue per I/O thread.
   - wqunorder: Worker-Pool un-ordered mode with a separate work queue for each
     HW RX queue.
   - wqunorderS: Worker-Pool un-ordered mode with oen work queue per I/O thread.
 --wqsize: number of elements for each worker queue.
 --lookup-iter: forces to perform ACL lookup several times over the same
   packet. This is artificial parameter and is added temporally for
   benchmarking purposes. Will be removed in latest versions (if any).

Note that in Worker-Pool mode all free lcores that were not assigned as
I/O threads will be used as Worker threads.
As an example:
dpdk-l3fwd --lcores=53,55,57,59,61 ... -- \
-P -p f --config '(0,0,53)(1,0,53)(2,0,53)(3,0,53)' --lookup acl \
--parse-ptype --mode=wqorder ...
In that case lcore 53 will be used as I/O thread (stages #1,3)
to serve 4 HW RX queues,
while lcores 55,57,59,61 will serve as Worker threads (stage #2).

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 examples/l3fwd/l3fwd.h           |  55 +++++++
 examples/l3fwd/l3fwd_acl.c       | 125 +++++++++++---
 examples/l3fwd/l3fwd_acl_event.h | 258 +++++++++++++++++++++++++++++
 examples/l3fwd/l3fwd_event.c     |  14 ++
 examples/l3fwd/l3fwd_event.h     |   1 +
 examples/l3fwd/l3fwd_sse.h       |  49 +++++-
 examples/l3fwd/l3fwd_wqp.c       | 274 +++++++++++++++++++++++++++++++
 examples/l3fwd/l3fwd_wqp.h       | 132 +++++++++++++++
 examples/l3fwd/main.c            |  75 ++++++++-
 examples/l3fwd/meson.build       |   1 +
 10 files changed, 956 insertions(+), 28 deletions(-)
 create mode 100644 examples/l3fwd/l3fwd_acl_event.h
 create mode 100644 examples/l3fwd/l3fwd_wqp.c
 create mode 100644 examples/l3fwd/l3fwd_wqp.h

diff --git a/examples/l3fwd/l3fwd.h b/examples/l3fwd/l3fwd.h
index 93ce652d02..218f363764 100644
--- a/examples/l3fwd/l3fwd.h
+++ b/examples/l3fwd/l3fwd.h
@@ -77,6 +77,42 @@ struct __rte_cache_aligned lcore_rx_queue {
 	uint16_t queue_id;
 };
 
+enum L3FWD_WORKER_MODE {
+	L3FWD_WORKER_POLL,
+	L3FWD_WORKER_UNQUE,
+	L3FWD_WORKER_ORQUE,
+};
+
+struct l3fwd_wqp_param {
+	enum L3FWD_WORKER_MODE mode;
+	uint32_t qsize;    /**< Number of elems in worker queue */
+	int32_t single;    /**< use single queue per I/O (poll) thread */
+};
+
+extern struct l3fwd_wqp_param l3fwd_wqp_param;
+
+enum {
+	LCORE_WQ_IN,
+	LCORE_WQ_OUT,
+	LCORE_WQ_NUM,
+};
+
+union lcore_wq {
+	struct rte_ring *r[LCORE_WQ_NUM];
+	struct {
+		struct rte_soring *sor;
+		/* used by WQ, sort of thred-local var */
+		uint32_t ftoken;
+	};
+};
+
+struct lcore_wq_pool {
+	uint32_t nb_queue;
+	uint32_t qmask;
+	union lcore_wq queue[MAX_RX_QUEUE_PER_LCORE];
+	struct l3fwd_wqp_param prm;
+};
+
 struct __rte_cache_aligned lcore_conf {
 	uint16_t n_rx_queue;
 	struct lcore_rx_queue rx_queue_list[MAX_RX_QUEUE_PER_LCORE];
@@ -86,6 +122,7 @@ struct __rte_cache_aligned lcore_conf {
 	struct mbuf_table tx_mbufs[RTE_MAX_ETHPORTS];
 	void *ipv4_lookup_struct;
 	void *ipv6_lookup_struct;
+	struct lcore_wq_pool wqpool;
 };
 
 extern volatile bool force_quit;
@@ -115,6 +152,8 @@ extern struct acl_algorithms acl_alg[];
 
 extern uint32_t max_pkt_len;
 
+extern uint32_t l3fwd_lookup_iter_num;
+
 /* Send burst of packets on an output interface */
 static inline int
 send_burst(struct lcore_conf *qconf, uint16_t n, uint16_t port)
@@ -308,6 +347,22 @@ fib_event_main_loop_tx_q_vector(__rte_unused void *dummy);
 int
 fib_event_main_loop_tx_q_burst_vector(__rte_unused void *dummy);
 
+int
+acl_event_main_loop_tx_d(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_d_burst(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q_burst(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_d_vector(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_d_burst_vector(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q_vector(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q_burst_vector(__rte_unused void *dummy);
 
 /* Return ipv4/ipv6 fwd lookup struct for ACL, LPM, EM or FIB. */
 void *
diff --git a/examples/l3fwd/l3fwd_acl.c b/examples/l3fwd/l3fwd_acl.c
index b635011ef7..8b550e495a 100644
--- a/examples/l3fwd/l3fwd_acl.c
+++ b/examples/l3fwd/l3fwd_acl.c
@@ -4,6 +4,7 @@
 
 #include "l3fwd.h"
 #include "l3fwd_route.h"
+#include "l3fwd_wqp.h"
 
 /*
  * Rule and trace formats definitions.
@@ -1003,19 +1004,21 @@ acl_process_pkts(struct rte_mbuf *pkts[MAX_PKT_BURST],
 	/* split packets burst depending on packet type (IPv4/IPv6) */
 	l3fwd_acl_prepare_acl_parameter(pkts, &acl_search, num);
 
-	if (acl_search.num_ipv4)
-		rte_acl_classify(acl_config.acx_ipv4[socketid],
+	for (i = l3fwd_lookup_iter_num; i != 0; i--) {
+		if (acl_search.num_ipv4)
+			rte_acl_classify(acl_config.acx_ipv4[socketid],
 				acl_search.data_ipv4,
 				acl_search.res_ipv4,
 				acl_search.num_ipv4,
 				DEFAULT_MAX_CATEGORIES);
 
-	if (acl_search.num_ipv6)
-		rte_acl_classify(acl_config.acx_ipv6[socketid],
+		if (acl_search.num_ipv6)
+			rte_acl_classify(acl_config.acx_ipv6[socketid],
 				acl_search.data_ipv6,
 				acl_search.res_ipv6,
 				acl_search.num_ipv6,
 				DEFAULT_MAX_CATEGORIES);
+	}
 
 	/* combine lookup results back, into one array of next hops */
 	n4 = 0;
@@ -1042,34 +1045,36 @@ acl_process_pkts(struct rte_mbuf *pkts[MAX_PKT_BURST],
 
 static inline void
 acl_send_packets(struct lcore_conf *qconf, struct rte_mbuf *pkts[],
-	uint16_t hops[], uint32_t num)
+	uint16_t hops[], uint32_t num, int step3)
 {
 #if defined ACL_SEND_MULTI
-	send_packets_multi(qconf, pkts, hops, num);
+	__send_packets_multi(qconf, pkts, hops, num, step3);
 #else
-	send_packets_single(qconf, pkts, hops, num);
+	if (step3 != 0)
+		send_packets_single(qconf, pkts, hops, num);
+	else {
+		uint32_t i;
+		for (i = 0; i != num; i++)
+			send_single_packet(qconf, pkts[i], hops[i]);
+	}
 #endif
 }
 
 /* main processing loop */
-int
-acl_main_loop(__rte_unused void *dummy)
+static int
+acl_poll_loop(struct lcore_conf *qconf, uint32_t lcore_id)
 {
 	struct rte_mbuf *pkts_burst[MAX_PKT_BURST];
 	uint16_t hops[MAX_PKT_BURST];
-	unsigned int lcore_id;
 	uint64_t prev_tsc, diff_tsc, cur_tsc;
-	int i, nb_rx;
+	uint32_t i, n, nb_rx;
 	uint16_t portid;
 	uint16_t queueid;
-	struct lcore_conf *qconf;
 	int socketid;
 	const uint64_t drain_tsc = (rte_get_tsc_hz() + US_PER_S - 1)
 			/ US_PER_S * BURST_TX_DRAIN_US;
 
 	prev_tsc = 0;
-	lcore_id = rte_lcore_id();
-	qconf = &lcore_conf[lcore_id];
 	socketid = rte_lcore_to_socket_id(lcore_id);
 
 	if (qconf->n_rx_queue == 0) {
@@ -1121,17 +1126,99 @@ acl_main_loop(__rte_unused void *dummy)
 			nb_rx = rte_eth_rx_burst(portid, queueid,
 				pkts_burst, MAX_PKT_BURST);
 
-			if (nb_rx > 0) {
-				acl_process_pkts(pkts_burst, hops, nb_rx,
-					socketid);
-				acl_send_packets(qconf, pkts_burst, hops,
-					nb_rx);
+			if (nb_rx != 0) {
+				if (l3fwd_wqp_param.mode == L3FWD_WORKER_POLL) {
+					acl_process_pkts(pkts_burst, hops,
+						nb_rx, socketid);
+					acl_send_packets(qconf, pkts_burst,
+						hops, nb_rx, 1);
+				} else {
+					n = lcore_wq_submit(&qconf->wqpool, i,
+						pkts_burst, nb_rx);
+					if (n != nb_rx) {
+						/* update stats counter */
+						rte_pktmbuf_free_bulk(
+							pkts_burst + n,
+							nb_rx - n);
+					}
+				}
+			}
+			if (l3fwd_wqp_param.mode != L3FWD_WORKER_POLL) {
+				nb_rx = lcore_wq_receive(&qconf->wqpool, i,
+					pkts_burst, hops, MAX_PKT_BURST);
+				if (nb_rx != 0)
+					acl_send_packets(qconf, pkts_burst,
+						hops, nb_rx, 0);
+			}
+		}
+	}
+	return 0;
+}
+
+/* WT processing loop */
+static int
+acl_wqp_loop(struct lcore_conf *qconf, uint32_t lcore_id)
+{
+	int32_t socketid;
+	uint32_t i, k, n;
+	struct rte_mbuf *pkts[MAX_PKT_BURST];
+	uint16_t hops[MAX_PKT_BURST];
+
+	socketid = rte_lcore_to_socket_id(lcore_id);
+
+	if (qconf->wqpool.nb_queue == 0) {
+		RTE_LOG(INFO, L3FWD, "%s: lcore %u has nothing to do\n",
+			__func__, lcore_id);
+		return 0;
+	}
+
+	RTE_LOG(INFO, L3FWD, "%s: entering loop on lcore %u\n",
+		__func__, lcore_id);
+
+	while (!force_quit) {
+
+		/*
+		 * Read packet from internal queues and process them
+		 */
+		for (i = 0; i < qconf->wqpool.nb_queue; ++i) {
+
+			n = lcore_wq_pull(&qconf->wqpool, i, pkts,
+				RTE_DIM(pkts));
+			if (n == 0)
+				continue;
+
+			acl_process_pkts(pkts, hops, n, socketid);
+			process_step3_burst(pkts, hops, n);
+			k = lcore_wq_push(&qconf->wqpool, i, pkts, hops, n);
+			if (n != k) {
+				/* stats update */
+				rte_pktmbuf_free_bulk(pkts + k, n - k);
 			}
 		}
 	}
 	return 0;
 }
 
+/* main processing loop */
+int
+acl_main_loop(__rte_unused void *dummy)
+{
+	uint32_t lcore_id;
+	struct lcore_conf *qconf;
+
+	lcore_id = rte_lcore_id();
+	qconf = &lcore_conf[lcore_id];
+
+	if (qconf->n_rx_queue != 0)
+		return acl_poll_loop(qconf, lcore_id);
+	else
+		return acl_wqp_loop(qconf, lcore_id);
+}
+
+#ifdef RTE_LIB_EVENTDEV
+#include "l3fwd_acl_event.h"
+#endif
+
 /* Not used by L3fwd ACL. */
 void *
 acl_get_ipv4_l3fwd_lookup_struct(__rte_unused const int socketid)
diff --git a/examples/l3fwd/l3fwd_acl_event.h b/examples/l3fwd/l3fwd_acl_event.h
new file mode 100644
index 0000000000..240dd3fb03
--- /dev/null
+++ b/examples/l3fwd/l3fwd_acl_event.h
@@ -0,0 +1,258 @@
+#include "l3fwd_event.h"
+
+/* One eventdev loop for single and burst using acl. */
+static __rte_always_inline void
+acl_event_loop(struct l3fwd_event_resources *evt_rsrc,
+		const uint8_t flags)
+{
+	uint32_t i, lcore_id, nb_deq, nb_enq;
+	int32_t socketid;
+	uint16_t hops[MAX_PKT_BURST];
+	struct rte_mbuf *pkts[MAX_PKT_BURST];
+	struct rte_event events[MAX_PKT_BURST];
+
+	const int event_p_id = l3fwd_get_free_event_port(evt_rsrc);
+	const uint8_t tx_q_id = evt_rsrc->evq.event_q_id[
+			evt_rsrc->evq.nb_queues - 1];
+	const uint8_t event_d_id = evt_rsrc->event_d_id;
+	const uint16_t deq_len = RTE_MIN(evt_rsrc->deq_depth, MAX_PKT_BURST);
+
+	if (event_p_id < 0)
+		return;
+
+	lcore_id = rte_lcore_id();
+	socketid = rte_lcore_to_socket_id(lcore_id);
+
+	RTE_LOG(INFO, L3FWD, "entering %s on lcore %u\n", __func__, lcore_id);
+
+	nb_deq = 0;
+	nb_enq = 0;
+
+	while (!force_quit) {
+		/* Read events from RX queues. */
+		nb_deq = rte_event_dequeue_burst(event_d_id, event_p_id,
+				events, deq_len, 0);
+		if (nb_deq == 0) {
+			rte_pause();
+			continue;
+		}
+
+		for (i = 0; i != nb_deq; i++) {
+			pkts[i] = events[i].mbuf;
+			if (flags & L3FWD_EVENT_TX_ENQ) {
+				events[i].queue_id = tx_q_id;
+				events[i].op = RTE_EVENT_OP_FORWARD;
+			}
+			rte_event_eth_tx_adapter_txq_set(pkts[i], 0);
+		}
+
+		acl_process_pkts(pkts, hops, nb_deq, socketid);
+
+		for (i = 0; i != nb_deq; i++) {
+			process_packet(pkts[i], &hops[i]);
+			pkts[i]->port = (hops[i] != BAD_PORT) ?
+				       hops[i] : pkts[i]->port;
+		}
+
+		if (flags & L3FWD_EVENT_TX_ENQ) {
+			nb_enq = rte_event_enqueue_burst(event_d_id, event_p_id,
+					events, nb_deq);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_enqueue_burst(event_d_id,
+						event_p_id, events + nb_enq,
+						nb_deq - nb_enq);
+		}
+
+		if (flags & L3FWD_EVENT_TX_DIRECT) {
+			nb_enq = rte_event_eth_tx_adapter_enqueue(event_d_id,
+					event_p_id, events, nb_deq, 0);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_eth_tx_adapter_enqueue(
+						event_d_id, event_p_id,
+						events + nb_enq,
+						nb_deq - nb_enq, 0);
+		}
+	}
+
+	l3fwd_event_worker_cleanup(event_d_id, event_p_id, events, nb_enq,
+				   nb_deq, 0);
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d_burst(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q_burst(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
+
+static __rte_always_inline void
+acl_process_event_vector(struct rte_event_vector *vec, uint16_t *hops,
+	int32_t socketid)
+{
+	uint32_t i, k;
+
+	for (i = 0; i != vec->nb_elem; i += k) {
+		k = RTE_MIN(vec->nb_elem - i, (uint32_t)MAX_PKT_BURST);
+		acl_process_pkts(vec->mbufs + i, hops + i, k, socketid);
+	}
+
+#if defined ACL_SEND_MULTI
+	k = RTE_ALIGN_FLOOR(vec->nb_elem, FWDSTEP);
+
+	for (i = 0; i != k; i += FWDSTEP)
+		processx4_step3(&vec->mbufs[i], &hops[i]);
+	for (; i < vec->nb_elem; i++)
+		process_packet(vec->mbufs[i], &hops[i]);
+#else
+	for (i = 0; i < vec->nb_elem; i++)
+		process_packet(vec->mbufs[i], &hops[i]);
+#endif
+
+	process_event_vector(vec, hops);
+}
+
+static __rte_always_inline void
+acl_event_loop_vector(struct l3fwd_event_resources *evt_rsrc,
+		      const uint8_t flags)
+{
+	uint16_t *hops;
+	int32_t socketid;
+	uint32_t i, lcore_id, nb_deq, nb_enq;
+	struct rte_event events[MAX_PKT_BURST];
+
+	const int event_p_id = l3fwd_get_free_event_port(evt_rsrc);
+	const uint8_t tx_q_id =
+		evt_rsrc->evq.event_q_id[evt_rsrc->evq.nb_queues - 1];
+	const uint8_t event_d_id = evt_rsrc->event_d_id;
+	const uint16_t deq_len = evt_rsrc->deq_depth;
+
+	if (event_p_id < 0)
+		return;
+
+	lcore_id = rte_lcore_id();
+	socketid = rte_lcore_to_socket_id(lcore_id);
+
+	hops = rte_zmalloc_socket(NULL, sizeof(hops[0]) * evt_rsrc->vector_size,
+			RTE_CACHE_LINE_SIZE, socketid);
+	if (hops == NULL) {
+		RTE_LOG(ERR, L3FWD,
+			"%s: failed to alloc internal buffers on lcore %u\n",
+			__func__, lcore_id);
+		return;
+	}
+
+	RTE_LOG(INFO, L3FWD, "entering %s on lcore %u\n", __func__, lcore_id);
+
+	nb_deq = 0;
+	nb_enq = 0;
+
+	while (!force_quit) {
+		/* Read events from RX queues. */
+		nb_deq = rte_event_dequeue_burst(event_d_id, event_p_id, events,
+						 deq_len, 0);
+		if (nb_deq == 0) {
+			rte_pause();
+			continue;
+		}
+
+		for (i = 0; i < nb_deq; i++) {
+			if (flags & L3FWD_EVENT_TX_ENQ) {
+				events[i].queue_id = tx_q_id;
+				events[i].op = RTE_EVENT_OP_FORWARD;
+			}
+
+			acl_process_event_vector(events[i].vec, hops, socketid);
+		}
+
+		if (flags & L3FWD_EVENT_TX_ENQ) {
+			nb_enq = rte_event_enqueue_burst(event_d_id, event_p_id,
+							 events, nb_deq);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_enqueue_burst(
+					event_d_id, event_p_id, events + nb_enq,
+					nb_deq - nb_enq);
+		}
+
+		if (flags & L3FWD_EVENT_TX_DIRECT) {
+			nb_enq = rte_event_eth_tx_adapter_enqueue(
+				event_d_id, event_p_id, events, nb_deq, 0);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_eth_tx_adapter_enqueue(
+					event_d_id, event_p_id, events + nb_enq,
+					nb_deq - nb_enq, 0);
+		}
+	}
+
+	l3fwd_event_worker_cleanup(event_d_id, event_p_id, events, nb_enq,
+				   nb_deq, 1);
+	rte_free(hops);
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d_burst_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q_burst_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
diff --git a/examples/l3fwd/l3fwd_event.c b/examples/l3fwd/l3fwd_event.c
index c698ed27d2..cd8466a605 100644
--- a/examples/l3fwd/l3fwd_event.c
+++ b/examples/l3fwd/l3fwd_event.c
@@ -252,6 +252,16 @@ l3fwd_event_resource_setup(struct rte_eth_conf *port_conf)
 		[1][1][0] = fib_event_main_loop_tx_q_vector,
 		[1][1][1] = fib_event_main_loop_tx_q_burst_vector,
 	};
+	const event_loop_cb acl_event_loop[2][2][2] = {
+		[0][0][0] = acl_event_main_loop_tx_d,
+		[0][0][1] = acl_event_main_loop_tx_d_burst,
+		[0][1][0] = acl_event_main_loop_tx_q,
+		[0][1][1] = acl_event_main_loop_tx_q_burst,
+		[1][0][0] = acl_event_main_loop_tx_d_vector,
+		[1][0][1] = acl_event_main_loop_tx_d_burst_vector,
+		[1][1][0] = acl_event_main_loop_tx_q_vector,
+		[1][1][1] = acl_event_main_loop_tx_q_burst_vector,
+	};
 	uint32_t event_queue_cfg;
 	int ret;
 
@@ -295,6 +305,10 @@ l3fwd_event_resource_setup(struct rte_eth_conf *port_conf)
 	evt_rsrc->ops.fib_event_loop =
 		fib_event_loop[evt_rsrc->vector_enabled][evt_rsrc->tx_mode_q]
 			      [evt_rsrc->has_burst];
+
+	evt_rsrc->ops.acl_event_loop =
+		acl_event_loop[evt_rsrc->vector_enabled][evt_rsrc->tx_mode_q]
+			      [evt_rsrc->has_burst];
 }
 
 static void
diff --git a/examples/l3fwd/l3fwd_event.h b/examples/l3fwd/l3fwd_event.h
index c6a4a89127..21aa24c396 100644
--- a/examples/l3fwd/l3fwd_event.h
+++ b/examples/l3fwd/l3fwd_event.h
@@ -58,6 +58,7 @@ struct l3fwd_event_setup_ops {
 	event_loop_cb lpm_event_loop;
 	event_loop_cb em_event_loop;
 	event_loop_cb fib_event_loop;
+	event_loop_cb acl_event_loop;
 };
 
 struct l3fwd_event_resources {
diff --git a/examples/l3fwd/l3fwd_sse.h b/examples/l3fwd/l3fwd_sse.h
index 083729cdef..29c5c7c014 100644
--- a/examples/l3fwd/l3fwd_sse.h
+++ b/examples/l3fwd/l3fwd_sse.h
@@ -86,12 +86,35 @@ process_packet(struct rte_mbuf *pkt, uint16_t *dst_port)
 	_mm_storeu_si128((__m128i *)eth_hdr, te);
 }
 
+static inline void
+process_step3_burst(struct rte_mbuf *pkt[], uint16_t dst_port[], uint32_t num)
+{
+	uint32_t i, k;
+
+	k = RTE_ALIGN_FLOOR(num, FWDSTEP);
+
+	for (i = 0; i != k; i += FWDSTEP)
+		processx4_step3(pkt + i, dst_port + i);
+
+	/* Process up to last 3 packets one by one. */
+	switch (num % FWDSTEP) {
+	case 3:
+		process_packet(pkt[i + 2], dst_port + i + 2);
+		/* fall-through */
+	case 2:
+		process_packet(pkt[i + 1], dst_port + i + 1);
+		/* fall-through */
+	case 1:
+		process_packet(pkt[i], dst_port + i);
+	}
+}
+
 /**
  * Send packets burst from pkts_burst to the ports in dst_port array
  */
 static __rte_always_inline void
-send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
-		uint16_t dst_port[MAX_PKT_BURST], int nb_rx)
+__send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
+		uint16_t dst_port[MAX_PKT_BURST], int nb_rx, int step3)
 {
 	int32_t k;
 	int j = 0;
@@ -110,13 +133,15 @@ send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
 		lp = pnum;
 		lp[0] = 1;
 
-		processx4_step3(pkts_burst, dst_port);
+		if (step3 != 0)
+			processx4_step3(pkts_burst, dst_port);
 
 		/* dp1: <d[0], d[1], d[2], d[3], ... > */
 		dp1 = _mm_loadu_si128((__m128i *)dst_port);
 
 		for (j = FWDSTEP; j != k; j += FWDSTEP) {
-			processx4_step3(&pkts_burst[j], &dst_port[j]);
+			if (step3 != 0)
+				processx4_step3(&pkts_burst[j], &dst_port[j]);
 
 			/*
 			 * dp2:
@@ -155,17 +180,20 @@ send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
 	/* Process up to last 3 packets one by one. */
 	switch (nb_rx % FWDSTEP) {
 	case 3:
-		process_packet(pkts_burst[j], dst_port + j);
+		if (step3 != 0)
+			process_packet(pkts_burst[j], dst_port + j);
 		GROUP_PORT_STEP(dlp, dst_port, lp, pnum, j);
 		j++;
 		/* fall-through */
 	case 2:
-		process_packet(pkts_burst[j], dst_port + j);
+		if (step3 != 0)
+			process_packet(pkts_burst[j], dst_port + j);
 		GROUP_PORT_STEP(dlp, dst_port, lp, pnum, j);
 		j++;
 		/* fall-through */
 	case 1:
-		process_packet(pkts_burst[j], dst_port + j);
+		if (step3 != 0)
+			process_packet(pkts_burst[j], dst_port + j);
 		GROUP_PORT_STEP(dlp, dst_port, lp, pnum, j);
 		j++;
 	}
@@ -194,6 +222,13 @@ send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
 	}
 }
 
+static __rte_always_inline void
+send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
+		uint16_t dst_port[MAX_PKT_BURST], int nb_rx)
+{
+	__send_packets_multi(qconf, pkts_burst, dst_port, nb_rx, 1);
+}
+
 static __rte_always_inline uint16_t
 process_dst_port(uint16_t *dst_ports, uint16_t nb_elem)
 {
diff --git a/examples/l3fwd/l3fwd_wqp.c b/examples/l3fwd/l3fwd_wqp.c
new file mode 100644
index 0000000000..eb5b80e358
--- /dev/null
+++ b/examples/l3fwd/l3fwd_wqp.c
@@ -0,0 +1,274 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "l3fwd.h"
+#include "l3fwd_wqp.h"
+
+static int
+wqp_ring_init(struct rte_ring **r, uint32_t num, size_t sz, int32_t sid,
+	uint32_t flags)
+{
+	char name[RTE_RING_NAMESIZE];
+
+	*r = rte_zmalloc_socket(NULL, sz, RTE_CACHE_LINE_SIZE, sid);
+	if (*r == NULL)
+		return -ENOMEM;
+
+	snprintf(name, sizeof(name), "%p", *r);
+	return rte_ring_init(*r, name, num, flags);
+}
+
+static int
+wqp_soring_init(struct rte_soring **r, struct rte_soring_param *prm, size_t sz,
+	int32_t sid)
+{
+	char name[RTE_RING_NAMESIZE];
+
+	*r = rte_zmalloc_socket(NULL, sz, RTE_CACHE_LINE_SIZE, sid);
+	if (*r == NULL)
+		return -ENOMEM;
+
+	snprintf(name, sizeof(name), "%p", *r);
+	prm->name = name;
+	return rte_soring_init(*r, prm);
+}
+
+static void
+wqp_fini(struct lcore_conf *lc)
+{
+	uint32_t i, j;
+
+	if (lc->n_rx_queue == 0)
+		return;
+
+	for (i = 0; i != lc->wqpool.nb_queue; i++) {
+		for (j = 0; j != RTE_DIM(lc->wqpool.queue[i].r); j++)
+			rte_free(lc->wqpool.queue[i].r[j]);
+	}
+
+	memset(&lc->wqpool, 0, sizeof(lc->wqpool));
+}
+
+static int
+l3fwd_wqp_unque_init(struct lcore_conf *lc, const struct l3fwd_wqp_param *prm,
+	uint32_t lcid)
+{
+	int32_t rc, sid;
+	uint32_t i, n, nq;
+	size_t szi, szo;
+
+	sid = rte_lcore_to_socket_id(lcid);
+
+	n = rte_align32pow2(prm->qsize);
+
+	szi = rte_ring_get_memsize(n);
+	szo = rte_ring_get_memsize_elem(sizeof(struct wqelm), n);
+
+	nq = (prm->single == 0) ? lc->n_rx_queue : 1;
+	lc->wqpool.nb_queue = nq;
+	lc->wqpool.qmask = (prm->single != 0) ? 0 : UINT32_MAX;
+
+	rc = 0;
+	for (i = 0; i != nq; i++) {
+
+		rc = wqp_ring_init(&lc->wqpool.queue[i].r[LCORE_WQ_IN], n, szi,
+			sid, RING_F_SP_ENQ);
+		if (rc != 0)
+			break;
+
+		rc = wqp_ring_init(&lc->wqpool.queue[i].r[LCORE_WQ_OUT], n, szo,
+			sid, RING_F_SC_DEQ);
+		if (rc != 0)
+			break;
+	}
+
+	if (i != nq) {
+		printf("error: %s failed at %u-th queue, error code: %d\n",
+			__func__, i, rc);
+		wqp_fini(lc);
+	}
+
+	lc->wqpool.prm = *prm;
+	return rc;
+}
+
+static int
+l3fwd_wqp_orque_init(struct lcore_conf *lc, const struct l3fwd_wqp_param *qprm,
+	uint32_t lcid)
+{
+	int32_t rc, sid;
+	uint32_t i, n, nq;
+	ssize_t sz;
+	struct rte_soring_param prm;
+
+	sid = rte_lcore_to_socket_id(lcid);
+
+	memset(&prm, 0, sizeof(prm));
+
+	n = 2 * qprm->qsize;
+	prm.elems = n;
+	prm.esize = sizeof(uintptr_t);
+	prm.stsize = sizeof(uint32_t);
+	prm.stages = 1;
+	prm.prod_synt = RTE_RING_SYNC_ST;
+	prm.cons_synt = RTE_RING_SYNC_ST;
+
+	sz = rte_soring_get_memsize(&prm);
+	if (sz < 0)
+		return sz;
+
+	nq = (qprm->single == 0) ? lc->n_rx_queue : 1;
+	lc->wqpool.nb_queue = nq;
+	lc->wqpool.qmask = (qprm->single != 0) ? 0 : UINT32_MAX;
+
+	rc = 0;
+	for (i = 0; i != nq; i++) {
+
+		rc = wqp_soring_init(&lc->wqpool.queue[i].sor, &prm, sz, sid);
+		if (rc != 0)
+			break;
+	}
+
+	if (i != nq) {
+		printf("error: %s failed at %u-th queue, error code: %d\n",
+			__func__, i, rc);
+		wqp_fini(lc);
+	}
+
+	lc->wqpool.prm = *qprm;
+	return rc;
+}
+
+static int
+wqp_init(struct lcore_conf *lc, const struct l3fwd_wqp_param *prm,
+	uint32_t lcid)
+{
+	/* this is I/O poll lcore */
+	if (lc->n_rx_queue != 0) {
+		if (prm->mode == L3FWD_WORKER_UNQUE)
+			return l3fwd_wqp_unque_init(lc, prm, lcid);
+		else if (prm->mode == L3FWD_WORKER_ORQUE)
+			return l3fwd_wqp_orque_init(lc, prm, lcid);
+		else
+			return -ENOTSUP;
+	}
+
+	return -ENOTSUP;
+}
+
+void
+l3fwd_wqp_fini(struct lcore_conf lc[RTE_MAX_LCORE])
+{
+	uint32_t lcid;
+
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++)
+		wqp_fini(lc + lcid);
+}
+
+static int
+check_set_wqp_param(struct l3fwd_wqp_param *prm)
+{
+	uint32_t n;
+
+	if (prm->qsize == 0) {
+		n = RTE_MAX(nb_rxd, nb_txd);
+		n = n + n / 2;
+		prm->qsize = n;
+	}
+
+	return 0;
+}
+
+static void print_wqp_param(const struct l3fwd_wqp_param *prm)
+{
+	printf("%s(%p): mode=%d, qsize=%u, single=%d\n",
+		__func__, prm, prm->mode, prm->qsize, prm->single);
+}
+
+int
+l3fwd_wqp_init(struct lcore_conf lc[RTE_MAX_LCORE],
+	const struct l3fwd_wqp_param *qprm)
+{
+	int32_t rc;
+	uint32_t i, j, k, lcid, m, n, nrxq, nwqt;
+	union lcore_wq *wqp;
+	struct l3fwd_wqp_param prm;
+
+	if (qprm->mode == L3FWD_WORKER_POLL)
+		return 0;
+
+	prm = *qprm;
+	rc = check_set_wqp_param(&prm);
+	print_wqp_param(&prm);
+	if (rc < 0) {
+		printf("error: %s invalid paramer values\n", __func__);
+		return rc;
+	}
+
+	nrxq = 0;
+	nwqt = 0;
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		if (rte_lcore_is_enabled(lcid) == 0)
+			continue;
+		if (lc[lcid].n_rx_queue != 0)
+			nrxq += (prm.single != 0) ? 1 : lc[lcid].n_rx_queue;
+		nwqt += (lc[lcid].n_rx_queue == 0);
+	}
+
+	printf("%s: total worker queues: %u, total WQ threads: %u\n",
+			__func__, nrxq, nwqt);
+	if (nrxq == 0)
+		return 0;
+
+	if (nrxq > nwqt * MAX_RX_QUEUE_PER_LCORE) {
+		printf("error: %s not enough WQ threads to handle all RXQs\n",
+			__func__);
+		return -EINVAL;
+	}
+
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		if (rte_lcore_is_enabled(lcid) == 0 || lc[lcid].n_rx_queue == 0)
+			continue;
+		rc = wqp_init(lc + lcid, &prm, lcid);
+		if (rc != 0)
+			break;
+	}
+	if (rc != 0)
+		return rc;
+
+	/* create a temp pool of all RX queues */
+	wqp = malloc(sizeof(wqp[0]) * nrxq);
+	if (wqp == NULL) {
+		l3fwd_wqp_fini(lc);
+		return -ENOMEM;
+	}
+
+	n = 0;
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		memcpy(wqp + n, lc[lcid].wqpool.queue,
+			lc[lcid].wqpool.nb_queue * sizeof(wqp[0]));
+		n += lc[lcid].wqpool.nb_queue;
+	}
+
+	/* distribute them across all worker threads */
+	k = 0;
+	m = RTE_MIN(RTE_DIM(lc[lcid].wqpool.queue), n);
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		if (rte_lcore_is_enabled(lcid) == 0 ||
+				lc[lcid].wqpool.nb_queue != 0)
+			continue;
+		j = k;
+		for (i = 0; i != m; i++) {
+			lc[lcid].wqpool.queue[i] = wqp[j];
+			j = (j + 1) % n;
+		}
+		lc[lcid].wqpool.nb_queue = i;
+		lc[lcid].wqpool.qmask = UINT32_MAX;
+		lc[lcid].wqpool.prm = prm;
+		k = j;
+	}
+
+	free(wqp);
+	return rc;
+}
diff --git a/examples/l3fwd/l3fwd_wqp.h b/examples/l3fwd/l3fwd_wqp.h
new file mode 100644
index 0000000000..9a6a7eca6c
--- /dev/null
+++ b/examples/l3fwd/l3fwd_wqp.h
@@ -0,0 +1,132 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef L3FWD_WQP_H
+#define L3FWD_WQP_H
+
+#include <rte_soring.h>
+
+struct wqelm {
+	struct rte_mbuf *mb;
+	uint32_t rc;
+}  __rte_packed;
+
+
+void l3fwd_wqp_fini(struct lcore_conf lc[RTE_MAX_LCORE]);
+int l3fwd_wqp_init(struct lcore_conf lc[RTE_MAX_LCORE],
+	const struct l3fwd_wqp_param *prm);
+
+static inline uint32_t
+lcore_wq_submit(const struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf * const pkts[MAX_PKT_BURST], uint32_t num)
+{
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_IN];
+		return rte_ring_enqueue_burst(r, (void * const *)pkts, num,
+				NULL);
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+		return rte_soring_enqueue(sor, pkts, NULL, num,
+			RTE_RING_QUEUE_VARIABLE, NULL);
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+static inline uint32_t
+lcore_wq_receive(const struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf *pkts[MAX_PKT_BURST], uint16_t hops[MAX_PKT_BURST],
+	uint32_t num)
+{
+	uint32_t i, n;
+	uint32_t rcs[MAX_PKT_BURST];
+	struct wqelm elm[MAX_PKT_BURST];
+
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_OUT];
+
+		n = rte_ring_dequeue_burst_elem(r, elm, sizeof(elm[0]), num,
+				NULL);
+		for (i = 0; i != n; i++) {
+			pkts[i] = elm[i].mb;
+			hops[i] = elm[i].rc;
+		}
+
+		return n;
+
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+
+		n = rte_soring_dequeue(sor, pkts, rcs, num,
+				RTE_RING_QUEUE_VARIABLE, NULL);
+		for (i = 0; i != n; i++)
+			hops[i] = rcs[i];
+
+		return n;
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+static inline uint32_t
+lcore_wq_pull(struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf *pkts[MAX_PKT_BURST], uint32_t num)
+{
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_IN];
+		return rte_ring_dequeue_burst(r, (void **)pkts, num, NULL);
+
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+		return rte_soring_acquire(sor, pkts, NULL, 0, num,
+			RTE_RING_QUEUE_VARIABLE, &wqp->queue[idx].ftoken, NULL);
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+static inline uint32_t
+lcore_wq_push(const struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf * const pkts[MAX_PKT_BURST],
+	const uint16_t hops[MAX_PKT_BURST], uint32_t num)
+{
+	uint32_t i;
+	uint32_t rcs[MAX_PKT_BURST];
+	struct wqelm elm[MAX_PKT_BURST];
+
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_OUT];
+
+		for (i = 0; i != num; i++) {
+			elm[i].mb = pkts[i];
+			elm[i].rc = hops[i];
+		}
+		return rte_ring_enqueue_burst_elem(r, elm, sizeof(elm[0]), num,
+				NULL);
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+
+		for (i = 0; i != num; i++)
+			rcs[i] = hops[i];
+		rte_soring_release(sor, NULL, rcs, 0, num,
+			wqp->queue[idx].ftoken);
+		return num;
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+#endif /* L3FWD_WQP_H */
diff --git a/examples/l3fwd/main.c b/examples/l3fwd/main.c
index 01b763e5ba..5d2bc5c611 100644
--- a/examples/l3fwd/main.c
+++ b/examples/l3fwd/main.c
@@ -47,6 +47,7 @@
 #include "l3fwd.h"
 #include "l3fwd_event.h"
 #include "l3fwd_route.h"
+#include "l3fwd_wqp.h"
 
 #define MAX_TX_QUEUE_PER_PORT RTE_MAX_LCORE
 #define MAX_RX_QUEUE_PER_PORT 128
@@ -69,6 +70,10 @@ enum L3FWD_LOOKUP_MODE {
 };
 static enum L3FWD_LOOKUP_MODE lookup_mode;
 
+struct l3fwd_wqp_param l3fwd_wqp_param = {
+	.mode = L3FWD_WORKER_POLL,
+};
+
 /* Global variables. */
 static int numa_on = 1; /**< NUMA is enabled by default. */
 static int parse_ptype; /**< Parse packet type using rx callback, and */
@@ -246,6 +251,8 @@ const struct ipv6_l3fwd_route ipv6_l3fwd_route_array[] = {
 	{{32, 1, 2, 0, 0, 0, 0, 15, 0, 0, 0, 0, 0, 0, 0, 0}, 64, 15},
 };
 
+uint32_t l3fwd_lookup_iter_num = 1;
+
 /*
  * API's called during initialization to setup ACL/EM/LPM rules.
  */
@@ -453,6 +460,23 @@ print_usage(const char *prgname)
 		ACL_LEAD_CHAR, ROUTE_LEAD_CHAR, alg);
 }
 
+static int
+parse_uint_val(const char *str, uint32_t *val, uint32_t min, uint32_t max)
+{
+	char *end = NULL;
+	unsigned long v;
+
+	errno = 0;
+	v = strtoul(str, &end, 0);
+	if (errno != 0 || end == NULL || *end != '\0')
+		return -1;
+	if (v < min || v > max)
+		return -1;
+
+	*val = v;
+	return 0;
+}
+
 static int
 parse_max_pkt_len(const char *pktlen)
 {
@@ -572,16 +596,35 @@ parse_eth_dest(const char *optarg)
 }
 
 static void
-parse_mode(const char *optarg __rte_unused)
+parse_mode(const char *optarg)
 {
+	l3fwd_wqp_param.mode = L3FWD_WORKER_POLL;
+
 #ifdef RTE_LIB_EVENTDEV
 	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
 
+	evt_rsrc->enabled = false;
+
 	if (!strcmp(optarg, "poll"))
 		evt_rsrc->enabled = false;
 	else if (!strcmp(optarg, "eventdev"))
 		evt_rsrc->enabled = true;
+	else
 #endif
+	if (strcmp(optarg, "wqorder") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_ORQUE;
+		l3fwd_wqp_param.single = 0;
+	} else if (strcmp(optarg, "wqunorder") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_UNQUE;
+		l3fwd_wqp_param.single = 0;
+	} else if (strcmp(optarg, "wqorderS") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_ORQUE;
+		l3fwd_wqp_param.single = 1;
+	} else if (strcmp(optarg, "wqunorderS") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_UNQUE;
+		l3fwd_wqp_param.single = 1;
+	} else
+		rte_exit(EXIT_FAILURE, "unknown mode: %s\n", optarg);
 }
 
 static void
@@ -698,6 +741,8 @@ static const char short_options[] =
 #define CMD_LINE_OPT_RULE_IPV4 "rule_ipv4"
 #define CMD_LINE_OPT_RULE_IPV6 "rule_ipv6"
 #define CMD_LINE_OPT_ALG "alg"
+#define CMD_LINE_OPT_WQSIZE "wqsize"
+#define CMD_LINE_OPT_LOOKUP_ITER "lookup-iter"
 
 enum {
 	/* long options mapped to a short option */
@@ -726,7 +771,9 @@ enum {
 	CMD_LINE_OPT_LOOKUP_NUM,
 	CMD_LINE_OPT_ENABLE_VECTOR_NUM,
 	CMD_LINE_OPT_VECTOR_SIZE_NUM,
-	CMD_LINE_OPT_VECTOR_TMO_NS_NUM
+	CMD_LINE_OPT_VECTOR_TMO_NS_NUM,
+	CMD_LINE_OPT_WQSIZE_NUM,
+	CMD_LINE_OPT_LOOKUP_ITER_NUM,
 };
 
 static const struct option lgopts[] = {
@@ -753,6 +800,8 @@ static const struct option lgopts[] = {
 	{CMD_LINE_OPT_RULE_IPV4,   1, 0, CMD_LINE_OPT_RULE_IPV4_NUM},
 	{CMD_LINE_OPT_RULE_IPV6,   1, 0, CMD_LINE_OPT_RULE_IPV6_NUM},
 	{CMD_LINE_OPT_ALG,   1, 0, CMD_LINE_OPT_ALG_NUM},
+	{CMD_LINE_OPT_WQSIZE, 1, 0, CMD_LINE_OPT_WQSIZE_NUM},
+	{CMD_LINE_OPT_LOOKUP_ITER, 1, 0, CMD_LINE_OPT_LOOKUP_ITER_NUM},
 	{NULL, 0, 0, 0}
 };
 
@@ -934,6 +983,18 @@ parse_args(int argc, char **argv)
 		case CMD_LINE_OPT_ALG_NUM:
 			l3fwd_set_alg(optarg);
 			break;
+		case CMD_LINE_OPT_WQSIZE_NUM:
+			ret = parse_uint_val(optarg, &l3fwd_wqp_param.qsize,
+				RX_DESC_DEFAULT, UINT16_MAX);
+			if (ret < 0)
+				return ret;
+			break;
+		case CMD_LINE_OPT_LOOKUP_ITER_NUM:
+			ret = parse_uint_val(optarg, &l3fwd_lookup_iter_num,
+				1, UINT16_MAX);
+			if (ret < 0)
+				return ret;
+			break;
 		default:
 			print_usage(prgname);
 			return -1;
@@ -1588,6 +1649,8 @@ main(int argc, char **argv)
 			l3fwd_lkp.main_loop = evt_rsrc->ops.em_event_loop;
 		else if (lookup_mode == L3FWD_LOOKUP_FIB)
 			l3fwd_lkp.main_loop = evt_rsrc->ops.fib_event_loop;
+		else if (lookup_mode == L3FWD_LOOKUP_ACL)
+			l3fwd_lkp.main_loop = evt_rsrc->ops.acl_event_loop;
 		else
 			l3fwd_lkp.main_loop = evt_rsrc->ops.lpm_event_loop;
 	} else
@@ -1640,6 +1703,12 @@ main(int argc, char **argv)
 		}
 	}
 
+	/* init worker queues for lcores (if any) */
+	ret = l3fwd_wqp_init(lcore_conf, &l3fwd_wqp_param);
+	if (ret != 0)
+		rte_exit(EXIT_FAILURE, "l3fwd_wqp_init: err=%d, lcore=%u\n",
+			ret, lcore_id);
+
 	check_all_ports_link_status(enabled_port_mask);
 
 	ret = 0;
@@ -1695,6 +1764,8 @@ main(int argc, char **argv)
 	/* clean up config file routes */
 	l3fwd_lkp.free_routes();
 
+	l3fwd_wqp_fini(lcore_conf);
+
 	/* clean up the EAL */
 	rte_eal_cleanup();
 
diff --git a/examples/l3fwd/meson.build b/examples/l3fwd/meson.build
index c25de77bba..a024492fb1 100644
--- a/examples/l3fwd/meson.build
+++ b/examples/l3fwd/meson.build
@@ -16,6 +16,7 @@ sources = files(
         'l3fwd_event_generic.c',
         'l3fwd_fib.c',
         'l3fwd_lpm.c',
+        'l3fwd_wqp.c',
         'main.c',
 )
 if dpdk_conf.has('RTE_LIB_EVENTDEV')
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFC 6/6] ring: minimize reads of the counterpart cache-line
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                   ` (4 preceding siblings ...)
  2024-08-15  8:53 ` [RFC 5/6] examples/l3fwd: make ACL work in pipeline and eventdev modes Konstantin Ananyev
@ 2024-08-15  8:53 ` Konstantin Ananyev
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15  8:53 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: this change shouldn't affect rte_ring public API.
Though as layout of public structures have changed - it is an ABI
breakage.

This is an attempt to implement rte_ring optimization
that was suggested by Morten and discussed on this mailing
list a while ago.
The idea is to optimize MP/SP & MC/SC ring enqueue/dequeue ops
by storing along with the head its Cached Foreign Tail (CFT) value.
I.E.: for producer we cache consumer tail value and visa-versa.
To avoid races head and CFT values are read/written using atomic
64-bit ops.
In theory that might help by reducing number of times producer
needs to access consumer's cache-line and visa-versa.
In practice, I didn't see any impressive boost so far:
-  ring_per_autotest micro-bench - results are a mixed bag,
   Some are a bit better, some are worse.
 - [so]ring_stress_autotest  micro-benchs: ~10-15% improvement
 - l3fwd in wqorder/wqundorder (see previous patch for details):
   no real difference.

Though so far my testing scope was quite limited, I tried it only
on x86 machines. So can I ask all interested parties:
different platform vendors (ARM, PPC, etc.)
and people who do use rte_ring extensively to give it a try and come up
with the feedback.

If there would be no real performance improvements on
any platform we support, or some problems will be encountered -
I am ok to drop that patch.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 drivers/net/mlx5/mlx5_hws_cnt.h   |  5 ++--
 drivers/net/ring/rte_eth_ring.c   |  2 +-
 lib/ring/rte_ring.c               |  6 ++--
 lib/ring/rte_ring_core.h          | 12 +++++++-
 lib/ring/rte_ring_generic_pvt.h   | 46 +++++++++++++++++++++----------
 lib/ring/rte_ring_peek_elem_pvt.h |  4 +--
 lib/ring/soring.c                 | 31 +++++++++++++++------
 lib/ring/soring.h                 |  4 +--
 8 files changed, 77 insertions(+), 33 deletions(-)

diff --git a/drivers/net/mlx5/mlx5_hws_cnt.h b/drivers/net/mlx5/mlx5_hws_cnt.h
index 996ac8dd9a..663146563c 100644
--- a/drivers/net/mlx5/mlx5_hws_cnt.h
+++ b/drivers/net/mlx5/mlx5_hws_cnt.h
@@ -388,11 +388,12 @@ __mlx5_hws_cnt_pool_enqueue_revert(struct rte_ring *r, unsigned int n,
 
 	MLX5_ASSERT(r->prod.sync_type == RTE_RING_SYNC_ST);
 	MLX5_ASSERT(r->cons.sync_type == RTE_RING_SYNC_ST);
-	current_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	current_head = rte_atomic_load_explicit(&r->prod.head.val.pos,
+			rte_memory_order_relaxed);
 	MLX5_ASSERT(n <= r->capacity);
 	MLX5_ASSERT(n <= rte_ring_count(r));
 	revert2head = current_head - n;
-	r->prod.head = revert2head; /* This ring should be SP. */
+	r->prod.head.val.pos = revert2head; /* This ring should be SP. */
 	__rte_ring_get_elem_addr(r, revert2head, sizeof(cnt_id_t), n,
 			&zcd->ptr1, &zcd->n1, &zcd->ptr2);
 	/* Update tail */
diff --git a/drivers/net/ring/rte_eth_ring.c b/drivers/net/ring/rte_eth_ring.c
index 1346a0dba3..31009e90d2 100644
--- a/drivers/net/ring/rte_eth_ring.c
+++ b/drivers/net/ring/rte_eth_ring.c
@@ -325,7 +325,7 @@ eth_get_monitor_addr(void *rx_queue, struct rte_power_monitor_cond *pmc)
 	 */
 	pmc->addr = &rng->prod.head;
 	pmc->size = sizeof(rng->prod.head);
-	pmc->opaque[0] = rng->prod.head;
+	pmc->opaque[0] = rng->prod.head.val.pos;
 	pmc->fn = ring_monitor_callback;
 	return 0;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..cb2c39c7ad 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -102,7 +102,7 @@ reset_headtail(void *p)
 	switch (ht->sync_type) {
 	case RTE_RING_SYNC_MT:
 	case RTE_RING_SYNC_ST:
-		ht->head = 0;
+		ht->head.raw = 0;
 		ht->tail = 0;
 		break;
 	case RTE_RING_SYNC_MT_RTS:
@@ -373,9 +373,9 @@ rte_ring_dump(FILE *f, const struct rte_ring *r)
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
 	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
+	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head.val.pos);
 	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
+	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head.val.pos);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
 }
diff --git a/lib/ring/rte_ring_core.h b/lib/ring/rte_ring_core.h
index 270869d214..b88a1bc352 100644
--- a/lib/ring/rte_ring_core.h
+++ b/lib/ring/rte_ring_core.h
@@ -66,8 +66,17 @@ enum rte_ring_sync_type {
  * Depending on sync_type format of that structure might be different,
  * but offset for *sync_type* and *tail* values should remain the same.
  */
+union __rte_ring_head_cft {
+	/** raw 8B value to read/write *cnt* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		uint32_t pos; /**< head position */
+		uint32_t cft; /**< cached foreign tail value*/
+	} val;
+};
+
 struct rte_ring_headtail {
-	volatile RTE_ATOMIC(uint32_t) head;      /**< prod/consumer head. */
+	uint32_t __unused;
 	volatile RTE_ATOMIC(uint32_t) tail;      /**< prod/consumer tail. */
 	union {
 		/** sync type of prod/cons */
@@ -75,6 +84,7 @@ struct rte_ring_headtail {
 		/** deprecated -  True if single prod/cons */
 		uint32_t single;
 	};
+	union __rte_ring_head_cft head;
 };
 
 union __rte_ring_rts_poscnt {
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 12f3595926..e70f4ff32c 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -38,17 +38,18 @@ __rte_ring_headtail_move_head(struct rte_ring_headtail *d,
 {
 	unsigned int max = n;
 	int success;
+	uint32_t tail;
+	union __rte_ring_head_cft nh, oh;
+
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = d->head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
+		*old_head = oh.val.pos;
+		tail = oh.val.cft;
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -56,24 +57,41 @@ __rte_ring_headtail_move_head(struct rte_ring_headtail *d,
 		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*entries = (capacity + s->tail - *old_head);
+		*entries = (capacity + tail - *old_head);
 
-		/* check that we have enough room in ring */
-		if (unlikely(n > *entries))
-			n = (behavior == RTE_RING_QUEUE_FIXED) ?
+		/* attempt #1: check that we have enough room with
+		 * cached-foreign-tail value.
+		 * Note that actual tail value can go forward till we cached
+		 * it, in that case we might have to update our cached value.
+		 */
+		if (unlikely(n > *entries)) {
+
+			tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_relaxed);
+			*entries = (capacity + tail - *old_head);
+
+			/* attempt #2: check that we have enough room in ring */
+			if (unlikely(n > *entries))
+				n = (behavior == RTE_RING_QUEUE_FIXED) ?
 					0 : *entries;
+		}
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
+		nh.val.pos = *new_head;
+		nh.val.cft = tail;
+
 		if (is_st) {
-			d->head = *new_head;
+			d->head.raw = nh.raw;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset(
-					(uint32_t *)(uintptr_t)&d->head,
-					*old_head, *new_head);
+			success = rte_atomic_compare_exchange_strong_explicit(
+				&d->head.raw, (uint64_t *)(uintptr_t)&oh.raw,
+				nh.raw, rte_memory_order_acquire,
+				rte_memory_order_acquire);
+
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_peek_elem_pvt.h b/lib/ring/rte_ring_peek_elem_pvt.h
index b5f0822b7e..e4dd0ae094 100644
--- a/lib/ring/rte_ring_peek_elem_pvt.h
+++ b/lib/ring/rte_ring_peek_elem_pvt.h
@@ -33,7 +33,7 @@ __rte_ring_st_get_tail(struct rte_ring_headtail *ht, uint32_t *tail,
 {
 	uint32_t h, n, t;
 
-	h = ht->head;
+	h = ht->head.val.pos;
 	t = ht->tail;
 	n = h - t;
 
@@ -58,7 +58,7 @@ __rte_ring_st_set_head_tail(struct rte_ring_headtail *ht, uint32_t tail,
 	RTE_SET_USED(enqueue);
 
 	pos = tail + num;
-	ht->head = pos;
+	ht->head.val.pos = pos;
 	rte_atomic_store_explicit(&ht->tail, pos, rte_memory_order_release);
 }
 
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
index 929bde9697..baa449c872 100644
--- a/lib/ring/soring.c
+++ b/lib/ring/soring.c
@@ -90,7 +90,8 @@ __rte_soring_stage_finalize(struct soring_stage_headtail *sht,
 	 * already finished.
 	 */
 
-	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	head = rte_atomic_load_explicit(&sht->head.val.pos,
+			rte_memory_order_relaxed);
 	n = RTE_MIN(head - ot.pos, maxn);
 	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
 
@@ -213,22 +214,36 @@ __rte_soring_stage_move_head(struct soring_stage_headtail *d,
 	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
 {
 	uint32_t n, tail;
+	union __rte_ring_head_cft nh, oh;
 
-	*old_head = rte_atomic_load_explicit(&d->head,
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
 			rte_memory_order_acquire);
 
 	do {
 		n = num;
-		tail = rte_atomic_load_explicit(&s->tail,
-				rte_memory_order_relaxed);
+		*old_head = oh.val.pos;
+		tail = oh.val.cft;
 		*avail = capacity + tail - *old_head;
-		if (n > *avail)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+
+		if (n > *avail) {
+			tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_relaxed);
+			*avail = capacity + tail - *old_head;
+
+			if (n > *avail)
+				n = (behavior == RTE_RING_QUEUE_FIXED) ?
+					0 : *avail;
+		}
+
 		if (n == 0)
 			return 0;
+
 		*new_head = *old_head + n;
-	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
-			old_head, *new_head, rte_memory_order_acquire,
+		nh.val.pos = *new_head;
+		nh.val.cft = tail;
+
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
+			&oh.raw, nh.raw, rte_memory_order_acquire,
 			rte_memory_order_acquire) == 0);
 
 	return n;
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
index 3a3f6efa76..0fb333aa71 100644
--- a/lib/ring/soring.h
+++ b/lib/ring/soring.h
@@ -60,8 +60,8 @@ union soring_stage_tail {
 
 struct soring_stage_headtail {
 	volatile union soring_stage_tail tail;
-	enum rte_ring_sync_type unused;  /**< unused */
-	volatile RTE_ATOMIC(uint32_t) head;
+	enum rte_ring_sync_type __unused;  /**< unused */
+	union __rte_ring_head_cft head;
 };
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [RFC 3/6] ring/soring: introduce Staged Ordered Ring
  2024-08-15  8:53 ` [RFC 3/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-08-15 11:11   ` Morten Brørup
  2024-08-15 12:41     ` Konstantin Ananyev
  2024-08-26 19:04   ` Mattias Rönnblom
  1 sibling, 1 reply; 101+ messages in thread
From: Morten Brørup @ 2024-08-15 11:11 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, Konstantin Ananyev

> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> 
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'.
> It is based on conventional DPDK rte_ring, re-uses many of its concepts,
> and even substantial part of its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in the middle.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages, i.e.:
>   at the same stage multiple threads can process objects from the ring in
>   any order, but for the next stage objects will always appear in the
>   original order.
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> - number of stages, size and number of objects in the ring are
>   configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation.
>   after some objects are 'acquired' - given thread can safely assume that
>   it has exclusive possession of these objects till 'release' for them is
>   invoked.
>   Note that right now user has to release exactly the same number of
>   objects that was acquired before.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important. I.E.: IPsec processing, etc.
> 
> Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> ---

The existing RING library is for a ring of objects.

It is very confusing that the new SORING library is for a ring of object pairs (obj, objst).

The new SORING library should be for a ring of objects, like the existing RING library. Please get rid of all the objst stuff.

This might also improve performance when not using the optional secondary object.


With that in place, you can extend the SORING library with additional APIs for object pairs.

I suggest calling the secondary object "metadata" instead of "status" or "state" or "ret-value".

I agree that data passed as {obj[num], meta[num]} is more efficient than {obj, meta}[num] in some use cases, which is why your API uses two vector pointers instead of one.


Furthermore, you should consider semi-zero-copy APIs for the "acquire"/"release" functions:

The "acquire" function can use a concept similar to rte_pktmbuf_read(), where a vector is provided for copying (if the ring wraps), and the return value either points directly to the objects in the ring (zero-copy), or to the vector where the objects were copied to.

And the "release" function does not need to copy the object vector back if the "acquire" function returned a zero-copy pointer.


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [RFC 3/6] ring/soring: introduce Staged Ordered Ring
  2024-08-15 11:11   ` Morten Brørup
@ 2024-08-15 12:41     ` Konstantin Ananyev
  2024-08-15 13:22       ` Morten Brørup
  0 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-08-15 12:41 UTC (permalink / raw)
  To: Morten Brørup, Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang



> > From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> >
> > Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> > with multiple processing 'stages'.
> > It is based on conventional DPDK rte_ring, re-uses many of its concepts,
> > and even substantial part of its code.
> > It can be viewed as an 'extension' of rte_ring functionality.
> > In particular, main SORING properties:
> > - circular ring buffer with fixed size objects
> > - producer, consumer plus multiple processing stages in the middle.
> > - allows to split objects processing into multiple stages.
> > - objects remain in the same ring while moving from one stage to the other,
> >   initial order is preserved, no extra copying needed.
> > - preserves the ingress order of objects within the queue across multiple
> >   stages, i.e.:
> >   at the same stage multiple threads can process objects from the ring in
> >   any order, but for the next stage objects will always appear in the
> >   original order.
> > - each stage (and producer/consumer) can be served by single and/or
> >   multiple threads.
> > - number of stages, size and number of objects in the ring are
> >   configurable at ring initialization time.
> >
> > Data-path API provides four main operations:
> > - enqueue/dequeue works in the same manner as for conventional rte_ring,
> >   all rte_ring synchronization types are supported.
> > - acquire/release - for each stage there is an acquire (start) and
> >   release (finish) operation.
> >   after some objects are 'acquired' - given thread can safely assume that
> >   it has exclusive possession of these objects till 'release' for them is
> >   invoked.
> >   Note that right now user has to release exactly the same number of
> >   objects that was acquired before.
> >   After 'release', objects can be 'acquired' by next stage and/or dequeued
> >   by the consumer (in case of last stage).
> >
> > Expected use-case: applications that uses pipeline model
> > (probably with multiple stages) for packet processing, when preserving
> > incoming packet order is important. I.E.: IPsec processing, etc.
> >
> > Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> > ---
> 
> The existing RING library is for a ring of objects.
> 
> It is very confusing that the new SORING library is for a ring of object pairs (obj, objst).
> 
> The new SORING library should be for a ring of objects, like the existing RING library. Please get rid of all the objst stuff.
> 
> This might also improve performance when not using the optional secondary object.
> 
> 
> With that in place, you can extend the SORING library with additional APIs for object pairs.
> 
> I suggest calling the secondary object "metadata" instead of "status" or "state" or "ret-value".
> I agree that data passed as {obj[num], meta[num]} is more efficient than {obj, meta}[num] in some use cases, which is why your API
> uses two vector pointers instead of one.

I suppose what you suggest is to have 2 set of functions: one that takes both objs[] and meta[] and second that takes just objs[]?
If so, yes I can do that - in fact I was thinking about same thing.
BTW, right now meta[] is an optional one anyway.
Also will probably get rid of explicit 'behavior' and will have '_burst_' and '_bulk_' versions instead,
same as rte_ring. 

> 
> Furthermore, you should consider semi-zero-copy APIs for the "acquire"/"release" functions:
> 
> The "acquire" function can use a concept similar to rte_pktmbuf_read(), where a vector is provided for copying (if the ring wraps), and
> the return value either points directly to the objects in the ring (zero-copy), or to the vector where the objects were copied to.

You mean to introduce analog of rte_ring '_zc_' functions?
Yes, I considered that, but decided to leave it for the future.
First, because we do need a generic and simple function with copying things anyway.
Second I am not so convinced that this _zc_ will give much performance gain,
while it definitely makes API not that straightforward.  

> And the "release" function does not need to copy the object vector back if the "acquire" function returned a zero-copy pointer.

For "release" you don't need to *always* copy objs[] and meta[].
It is optional and is left for the user to decide based on the use-case.
If he doesn't need to update objs[] or meta[] he can just pass a NULL ptr here.

 


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [RFC 3/6] ring/soring: introduce Staged Ordered Ring
  2024-08-15 12:41     ` Konstantin Ananyev
@ 2024-08-15 13:22       ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-08-15 13:22 UTC (permalink / raw)
  To: Konstantin Ananyev, Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang

> From: Konstantin Ananyev [mailto:konstantin.ananyev@huawei.com]
> 
> > > From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> > >
> > > Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered'
> queues
> > > with multiple processing 'stages'.
> > > It is based on conventional DPDK rte_ring, re-uses many of its concepts,
> > > and even substantial part of its code.
> > > It can be viewed as an 'extension' of rte_ring functionality.
> > > In particular, main SORING properties:
> > > - circular ring buffer with fixed size objects
> > > - producer, consumer plus multiple processing stages in the middle.
> > > - allows to split objects processing into multiple stages.
> > > - objects remain in the same ring while moving from one stage to the
> other,
> > >   initial order is preserved, no extra copying needed.
> > > - preserves the ingress order of objects within the queue across multiple
> > >   stages, i.e.:
> > >   at the same stage multiple threads can process objects from the ring in
> > >   any order, but for the next stage objects will always appear in the
> > >   original order.
> > > - each stage (and producer/consumer) can be served by single and/or
> > >   multiple threads.
> > > - number of stages, size and number of objects in the ring are
> > >   configurable at ring initialization time.
> > >
> > > Data-path API provides four main operations:
> > > - enqueue/dequeue works in the same manner as for conventional rte_ring,
> > >   all rte_ring synchronization types are supported.
> > > - acquire/release - for each stage there is an acquire (start) and
> > >   release (finish) operation.
> > >   after some objects are 'acquired' - given thread can safely assume that
> > >   it has exclusive possession of these objects till 'release' for them is
> > >   invoked.
> > >   Note that right now user has to release exactly the same number of
> > >   objects that was acquired before.
> > >   After 'release', objects can be 'acquired' by next stage and/or dequeued
> > >   by the consumer (in case of last stage).
> > >
> > > Expected use-case: applications that uses pipeline model
> > > (probably with multiple stages) for packet processing, when preserving
> > > incoming packet order is important. I.E.: IPsec processing, etc.
> > >
> > > Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> > > ---
> >
> > The existing RING library is for a ring of objects.
> >
> > It is very confusing that the new SORING library is for a ring of object
> pairs (obj, objst).
> >
> > The new SORING library should be for a ring of objects, like the existing
> RING library. Please get rid of all the objst stuff.
> >
> > This might also improve performance when not using the optional secondary
> object.
> >
> >
> > With that in place, you can extend the SORING library with additional APIs
> for object pairs.
> >
> > I suggest calling the secondary object "metadata" instead of "status" or
> "state" or "ret-value".
> > I agree that data passed as {obj[num], meta[num]} is more efficient than
> {obj, meta}[num] in some use cases, which is why your API
> > uses two vector pointers instead of one.
> 
> I suppose what you suggest is to have 2 set of functions: one that takes both
> objs[] and meta[] and second that takes just objs[]?
> If so, yes I can do that - in fact I was thinking about same thing.

Yes, please.
Mainly for readability/familiarity; it makes the API much more similar to the Ring API.

> BTW, right now meta[] is an optional one anyway.

I noticed that meta[] is optional, but it is confusing that the APIs are so much different than the Ring APIs.

With two sets of functions, the basic set will resemble the Ring APIs much more.

> Also will probably get rid of explicit 'behavior' and will have '_burst_' and
> '_bulk_' versions instead,
> same as rte_ring.

+1

> 
> >
> > Furthermore, you should consider semi-zero-copy APIs for the
> "acquire"/"release" functions:
> >
> > The "acquire" function can use a concept similar to rte_pktmbuf_read(),
> where a vector is provided for copying (if the ring wraps), and
> > the return value either points directly to the objects in the ring (zero-
> copy), or to the vector where the objects were copied to.
> 
> You mean to introduce analog of rte_ring '_zc_' functions?
> Yes, I considered that, but decided to leave it for the future.

Somewhat similar, but I think the (semi-)zero-copy "acquire"/"release" APIs will be simpler than the rte_ring's _zc_ functions because we know that no other thread can dequeue the objects out of the ring before the processing stage has released them, i.e. no additional locking is required.

Anyway, leave it for the future.
I don't think it will require changes to the underlying implementation, so we don't need to consider it in advance.

> First, because we do need a generic and simple function with copying things
> anyway.
> Second I am not so convinced that this _zc_ will give much performance gain,
> while it definitely makes API not that straightforward.
> 
> > And the "release" function does not need to copy the object vector back if
> the "acquire" function returned a zero-copy pointer.
> 
> For "release" you don't need to *always* copy objs[] and meta[].
> It is optional and is left for the user to decide based on the use-case.
> If he doesn't need to update objs[] or meta[] he can just pass a NULL ptr
> here.

Yes, I noticed.
I'm mentioning that zero-copy adds a "release" case where copying back the modified object vector is not required; when the processing stage has modified the object vector (e.g. replaced some objects) directly in the ring, and not in a copy returned by "acquire".

> 
> 


^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [RFC 3/6] ring/soring: introduce Staged Ordered Ring
  2024-08-15  8:53 ` [RFC 3/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
  2024-08-15 11:11   ` Morten Brørup
@ 2024-08-26 19:04   ` Mattias Rönnblom
  2024-09-03 13:55     ` Konstantin Ananyev
  1 sibling, 1 reply; 101+ messages in thread
From: Mattias Rönnblom @ 2024-08-26 19:04 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

On 2024-08-15 10:53, Konstantin Ananyev wrote:
> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> 
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'.
> It is based on conventional DPDK rte_ring, re-uses many of its concepts,
> and even substantial part of its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in the middle.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>    initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>    stages, i.e.:
>    at the same stage multiple threads can process objects from the ring in
>    any order, but for the next stage objects will always appear in the
>    original order.
> - each stage (and producer/consumer) can be served by single and/or
>    multiple threads.
> - number of stages, size and number of objects in the ring are
>    configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>    all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>    release (finish) operation.
>    after some objects are 'acquired' - given thread can safely assume that
>    it has exclusive possession of these objects till 'release' for them is
>    invoked.
>    Note that right now user has to release exactly the same number of
>    objects that was acquired before.
>    After 'release', objects can be 'acquired' by next stage and/or dequeued
>    by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important. I.E.: IPsec processing, etc.
> 

How does SORING related to Eventdev? Would it be feasible to reshape 
this into a SW event device?

> Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> ---
>   lib/ring/meson.build  |   4 +-
>   lib/ring/rte_soring.c | 144 ++++++++++++++
>   lib/ring/rte_soring.h | 270 ++++++++++++++++++++++++++
>   lib/ring/soring.c     | 431 ++++++++++++++++++++++++++++++++++++++++++
>   lib/ring/soring.h     | 124 ++++++++++++
>   lib/ring/version.map  |  13 ++
>   6 files changed, 984 insertions(+), 2 deletions(-)
>   create mode 100644 lib/ring/rte_soring.c
>   create mode 100644 lib/ring/rte_soring.h
>   create mode 100644 lib/ring/soring.c
>   create mode 100644 lib/ring/soring.h
> 
> diff --git a/lib/ring/meson.build b/lib/ring/meson.build
> index 7fca958ed7..21f2c12989 100644
> --- a/lib/ring/meson.build
> +++ b/lib/ring/meson.build
> @@ -1,8 +1,8 @@
>   # SPDX-License-Identifier: BSD-3-Clause
>   # Copyright(c) 2017 Intel Corporation
>   
> -sources = files('rte_ring.c')
> -headers = files('rte_ring.h')
> +sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
> +headers = files('rte_ring.h', 'rte_soring.h')
>   # most sub-headers are not for direct inclusion
>   indirect_headers += files (
>           'rte_ring_core.h',
> diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
> new file mode 100644
> index 0000000000..17b1b73a42
> --- /dev/null
> +++ b/lib/ring/rte_soring.c
> @@ -0,0 +1,144 @@
> +/* SPDX-License-Identifier: BSD-3-Clause
> + * Copyright(c) 2024 Huawei Technologies Co., Ltd
> + */
> +
> +#include "soring.h"
> +#include <rte_string_fns.h>
> +
> +RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
> +#define RTE_LOGTYPE_SORING soring_logtype
> +#define SORING_LOG(level, ...) \
> +	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
> +
> +static uint32_t
> +soring_calc_elem_num(uint32_t count)
> +{
> +	return rte_align32pow2(count + 1);
> +}
> +
> +static int
> +soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
> +	uint32_t stages)
> +{
> +	if (stages == 0) {
> +		SORING_LOG(ERR, "invalid number of stages: %u", stages);
> +		return -EINVAL;
> +	}
> +
> +	/* Check if element size is a multiple of 4B */
> +	if (esize == 0 || esize % 4 != 0) {
> +		SORING_LOG(ERR, "invalid element size: %u", esize);
> +		return -EINVAL;
> +	}
> +
> +	/* Check if ret-code size is a multiple of 4B */
> +	if (stsize % 4 != 0) {
> +		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
> +		return -EINVAL;
> +	}
> +
> +	 /* count must be a power of 2 */
> +	if (rte_is_power_of_2(count) == 0 ||
> +			(count > RTE_SORING_ELEM_MAX + 1)) {
> +		SORING_LOG(ERR, "invalid number of elements: %u", count);
> +		return -EINVAL;
> +	}
> +
> +	return 0;
> +}
> +
> +/*
> + * Calculate size offsets for SORING internal data layout.
> + */
> +static size_t
> +soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
> +	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
> +	size_t *stage_ofs)
> +{
> +	size_t sz;
> +	const struct rte_soring * const r = NULL;
> +
> +	sz = sizeof(r[0]) + (size_t)count * esize;
> +	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
> +
> +	if (elst_ofs != NULL)
> +		*elst_ofs = sz;
> +
> +	sz = sz + (size_t)count * stsize;
> +	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
> +
> +	if (state_ofs != NULL)
> +		*state_ofs = sz;
> +
> +	sz += sizeof(r->state[0]) * count;
> +	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
> +
> +	if (stage_ofs != NULL)
> +		*stage_ofs = sz;
> +
> +	sz += sizeof(r->stage[0]) * stages;
> +	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
> +
> +	return sz;
> +}
> +
> +
> +ssize_t
> +rte_soring_get_memsize(const struct rte_soring_param *prm)
> +{
> +	int32_t rc;
> +	uint32_t count;
> +
> +	count = soring_calc_elem_num(prm->elems);
> +	rc = soring_check_param(prm->esize, prm->stsize, count, prm->stages);
> +	if (rc != 0)
> +		return rc;
> +
> +	return soring_get_szofs(prm->esize, prm->stsize, count, prm->stages,
> +			NULL, NULL, NULL);
> +}
> +
> +int
> +rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
> +{
> +	int32_t rc;
> +	uint32_t n;
> +	size_t elst_ofs, stage_ofs, state_ofs;
> +
> +	if (r == NULL || prm == NULL)
> +		return -EINVAL;
> +
> +	n = soring_calc_elem_num(prm->elems);
> +	rc = soring_check_param(prm->esize, prm->stsize, n, prm->stages);
> +	if (rc != 0)
> +		return rc;
> +
> +	soring_get_szofs(prm->esize, prm->stsize, n, prm->stages, &elst_ofs,
> +			&state_ofs, &stage_ofs);
> +
> +	memset(r, 0, sizeof(*r));
> +	rc = strlcpy(r->name, prm->name, sizeof(r->name));
> +	if (rc < 0 || rc >= (int)sizeof(r->name))
> +		return -ENAMETOOLONG;
> +
> +	r->size = n;
> +	r->mask = r->size - 1;
> +	r->capacity = prm->elems;
> +	r->esize = prm->esize;
> +	r->stsize = prm->stsize;
> +
> +	r->prod.ht.sync_type = prm->prod_synt;
> +	r->cons.ht.sync_type = prm->cons_synt;
> +
> +	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
> +	memset(r->state, 0, sizeof(r->state[0]) * r->size);
> +
> +	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
> +	r->nb_stage = prm->stages;
> +	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
> +
> +	if (r->stsize != 0)
> +		r->elemst = (void *)((uintptr_t)r + elst_ofs);
> +
> +	return 0;
> +}
> diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
> new file mode 100644
> index 0000000000..fb0e75b39a
> --- /dev/null
> +++ b/lib/ring/rte_soring.h
> @@ -0,0 +1,270 @@
> +/* SPDX-License-Identifier: BSD-3-Clause
> + * Copyright(c) 2024 Huawei Technologies Co., Ltd
> + */
> +
> +#ifndef _RTE_SORING_H_
> +#define _RTE_SORING_H_
> +
> +/**
> + * @file
> + * This file contains definition of RTE soring (Staged Ordered Ring) public API.
> + * Brief description:
> + * enqueue/dequeue works the same as for conventional rte_ring:
> + * any rte_ring sync types can be used, etc.
> + * Plus there could be multiple 'stages'.
> + * For each stage there is an acquire (start) and release (finish) operation.
> + * after some elems are 'acquired' - user  can safely assume that he has
> + * exclusive possession of these elems till 'release' for them is done.
> + * Note that right now user has to release exactly the same number of elems
> + * he acquired before.
> + * After 'release', elems can be 'acquired' by next stage and/or dequeued
> + * (in case of last stage).
> + */
> +
> +#ifdef __cplusplus
> +extern "C" {
> +#endif
> +
> +#include <rte_ring.h>
> +
> +/* upper 2 bits are used for status */
> +#define RTE_SORING_ST_BIT       30
> +
> +/* max possible number of elements in the soring */
> +#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
> +
> +struct rte_soring_param {
> +	/** expected name of the ring */
> +	const char *name;
> +	/** number of elemnts in the ring */
> +	uint32_t elems;
> +	/** size of elements in the ring, must be a multiple of 4 */
> +	uint32_t esize;
> +	/**
> +	 * size of retcode for each elem, must be a multiple of 4.
> +	 * This parameter defines a size of supplementary and optional
> +	 * array of ret-codes associated with each object in the soring.
> +	 * While element size is configurable (see @esize parameter above),
> +	 * so user can specify it big enough to hold both object and its
> +	 * ret-code together, for performance reasons it might be plausible
> +	 * to access them as separate arrays.
> +	 * Common usage scenario when such separation helps:
> +	 * enqueue() - writes to objects array
> +	 * acquire() - reads from objects array
> +	 * release() - writes to ret-code array
> +	 * dequeue() - reads both objects and ret-code array
> +	 */
> +	uint32_t stsize;
> +	/** number of stages in the ring */
> +	uint32_t stages;
> +	/** sync type for producer */
> +	enum rte_ring_sync_type prod_synt;
> +	/** sync type for consumer */
> +	enum rte_ring_sync_type cons_synt;
> +};
> +
> +struct rte_soring;
> +
> +/**
> + * Calculate the memory size needed for a soring
> + *
> + * This function returns the number of bytes needed for a ring, given
> + * the expected parameters for it. This value is the sum of the size of
> + * the internal metadata and the size of the memory needed by the
> + * actual ring elements and theri rec-codes. The value is aligned to a cache
> + * line size.
> + *
> + * @param prm
> + *   Pointer to the structure that contains soring creation paramers.
> + * @return
> + *   - The memory size needed for the soring on success.
> + *   - -EINVAL if provided paramer values are invalid.
> + */
> +__rte_experimental
> +ssize_t
> +rte_soring_get_memsize(const struct rte_soring_param *prm);
> +
> +/**
> + * Initialize a soring structure.
> + *
> + * Initialize a soring structure in memory pointed by "r".
> + * The size of the memory area must be large enough to store the soring
> + * internal structures plus the objects and ret-code tables.
> + * It is strongly advised to use rte_ring_get_memsize() to get the
> + * appropriate size.
> + *
> + * @param r
> + *   Pointer to the soring structure.
> + * @param prm
> + *   Pointer to the structure that contains soring creation paramers.
> + * @return
> + *   - 0 on success, or a negative error code.
> + */
> +__rte_experimental
> +int
> +rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
> +
> +/**
> + * Return the total number of filled entries in a ring.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @return
> + *   The number of entries in the ring.
> + */
> +__rte_experimental
> +unsigned int
> +rte_soring_count(const struct rte_soring *r);
> +
> +/**
> + * Enqueue several objects on the ring.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param obj_table
> + *   A pointer to an array of objects to enqueue.
> + *   Size of objects to enqueue must be the same value as @esize parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param objst
> + *   A pointer to an array of status values for each object to enqueue.
> + *   Note that if user not using object status values, then this parameter
> + *   can be NULL.
> + *   Size of elements in this array must be the same value as @stsize parameter
> + *   used while creating the ring. If user created the soring with
> + *   @stsize value equals zero, then @objst parameter should be NULL.
> + *   Otherwise the results are undefined.
> + * @param n
> + *   The number of objects to add in the ring from the obj_table.
> + * @param behavior
> + *   Behavior type, one of:
> + *   - RTE_RING_QUEUE_FIXED: enqueue either exactly @n objects or none.
> + *   - RTE_RING_QUEUE_VARIABLE: enqueue up to @n objects.
> + * @param free_space
> + *   if non-NULL, returns the amount of space in the ring after the
> + *   enqueue operation has finished.
> + * @return
> + *   - Actual number of objects enqueued.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_enqueue(struct rte_soring *r, const void *obj_table,
> +	const void *objst, uint32_t n, enum rte_ring_queue_behavior behavior,
> +	uint32_t *free_space);
> +
> +/**
> + * Dequeue several objects from the ring.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param obj_table
> + *   A pointer to an array of objects to dequeue.
> + *   Size of objects to enqueue must be the same value as @esize parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param objst
> + *   A pointer to array of status values for each object to dequeue.
> + *   Note that if user not using object status values, then this parameter
> + *   can be NULL.
> + *   Size of elements in this array must be the same value as @stsize parameter
> + *   used while creating the ring. If user created the soring with
> + *   @stsize value equals zero, then @objst parameter should be NULL.
> + *   Otherwise the results are undefined.
> + * @param num
> + *   The number of objects to dequeue from the ring into the obj_table.
> + * @param behavior
> + *   Behavior type, one of:
> + *   - RTE_RING_QUEUE_FIXED: dequeue either exactly @n objects or none.
> + *   - RTE_RING_QUEUE_VARIABLE: dequeue up to @n objects.
> + * @param available
> + *   If non-NULL, returns the number of remaining ring entries after the
> + *   dequeue has finished.
> + * @return
> + *   - Actual number of objects dequeued.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_dequeue(struct rte_soring *r, void *obj_table, void *objst,
> +	uint32_t num, enum rte_ring_queue_behavior behavior,
> +	uint32_t *available);
> +
> +/**
> + * Acquire several objects from the ring for given stage.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param objs
> + *   A pointer to an array of objects to acquire.
> + *   Size of objects must be the same value as @esize parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param objst
> + *   A pointer to an array of status values for each for each acquired object.
> + *   Note that if user not using object status values, then this parameter
> + *   can be NULL.
> + *   Size of elements in this array must be the same value as @stsize parameter
> + *   used while creating the ring. If user created the soring with
> + *   @stsize value equals zero, then @objst parameter should be NULL.
> + *   Otherwise the results are undefined.
> + * @param stage
> + *   Stage to acquire objects for.
> + * @param num
> + *   The number of objects to acquire.
> + * @param behavior
> + *   Behavior type, one of:
> + *   - RTE_RING_QUEUE_FIXED: acquire either exactly @n objects or none.
> + *   - RTE_RING_QUEUE_VARIABLE: acquire up to @n objects.
> + * @param ftoken
> + *   Pointer to the opaque 'token' value used by release() op.
> + *   User has to store this value somewhere, and later provide to the
> + *   release().
> + * @param available
> + *   If non-NULL, returns the number of remaining ring entries for given stage
> + *   after the acquire has finished.
> + * @return
> + *   - Actual number of objects acquired.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_acquire(struct rte_soring *r, void *objs, void *objst,
> +	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
> +	uint32_t *ftoken, uint32_t *available);
> +
> +/**
> + * Release several objects for given stage back to the ring.
> + * Note that it means these objects become avaialble for next stage or
> + * dequeue.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param objs
> + *   A pointer to an array of objects to relase.
> + *   Note that unless user needs to overwrite ring objects this parameter
> + *   can be NULL.
> + *   Size of objects must be the same value as @esize parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param objst
> + *   A pointer to an array of status values for each object to release.
> + *   Note that if user not using object status values, then this parameter
> + *   can be NULL.
> + *   Size of elements in this array must be the same value as @stsize parameter
> + *   used while creating the ring. If user created the soring with
> + *   @stsize value equals zero, then objst parameter should be NULL.
> + *   Otherwise the results are undefined.
> + * @param stage
> + *   Current stage.
> + * @param n
> + *   The number of objects to release.
> + *   Has to be the same value as returned by acquire() op.
> + * @param ftoken
> + *   Opaque 'token' value obtained from acquire() op.
> + * @return
> + *   - None.
> + */
> +__rte_experimental
> +void
> +rte_soring_release(struct rte_soring *r, const void *objs,
> +	const void *objst, uint32_t stage, uint32_t n, uint32_t ftoken);
> +
> +#ifdef __cplusplus
> +}
> +#endif
> +
> +#endif /* _RTE_SORING_H_ */
> diff --git a/lib/ring/soring.c b/lib/ring/soring.c
> new file mode 100644
> index 0000000000..929bde9697
> --- /dev/null
> +++ b/lib/ring/soring.c
> @@ -0,0 +1,431 @@
> +/* SPDX-License-Identifier: BSD-3-Clause
> + * Copyright(c) 2024 Huawei Technologies Co., Ltd
> + */
> +
> +/**
> + * @file
> + * This file contains implementation of SORING 'datapath' functions.
> + * Brief description:
> + * ==================
> + * enqueue/dequeue works the same as for conventional rte_ring:
> + * any rte_ring sync types can be used, etc.
> + * Plus there could be multiple 'stages'.
> + * For each stage there is an acquire (start) and release (finish) operation.
> + * After some elems are 'acquired' - user can safely assume that he has
> + * exclusive possession of these elems till 'release' for them is done.
> + * Note that right now user has to release exactly the same number of elems
> + * he acquired before.
> + * After 'release', elems can be 'acquired' by next stage and/or dequeued
> + * (in case of last stage).
> + * Internal structure:
> + * ===================
> + * In addition to 'normal' ring of elems, it also has a ring of states of the
> + * same size. Each state[] corresponds to exactly one elem[].
> + * state[] will be used by acquire/release/dequeue functions to store internal
> + * information and should not be accessed by the user directly.
> + * How it works:
> + * =============
> + * 'acquire()' just moves stage's head (same as rte_ring move_head does),
> + * plus it saves in state[stage.cur_head] information about how many elems
> + * were acquired, current head position and special flag value to indicate
> + * that elems are acquired (RTE_SORING_ST_START).
> + * Note that 'acquire()' returns to the user a special 'ftoken' that user has
> + * to provide for 'release()' (in fact it is just a position for current head).
> + * 'release()' extracts old head value from provided ftoken and checks that
> + * corresponding 'state[]' contains expected values(mostly for sanity
> + * purposes).
> + * * Then it marks this state[] with 'RTE_SORING_ST_FINISH' flag to indicate
> + * that given subset of objects was released.
> + * After that, it checks does old head value equals to current tail value?
> + * If yes, then it performs  'finalize()' operation, otherwise 'release()'
> + * just returns (without spinning on stage tail value).
> + * As updated state[] is shared by all threads, some other thread can do
> + * 'finalize()' for given stage.
> + * That allows 'release()' to avoid excessive waits on the tail value.
> + * Main purpose of 'finalize()' operation is to walk through 'state[]'
> + * from current stage tail up to its head, check state[] and move stage tail
> + * through elements that already are in RTE_SORING_ST_FINISH state.
> + * Along with that, corresponding state[] values are reset to zero.
> + * Note that 'finalize()' for given stage can be done from multiple places:
> + * 'release()' for that stage or from 'acquire()' for next stage
> + * even from consumer's 'dequeue()' - in case given stage is the last one.
> + * So 'finalize()' has to be MT-safe and inside it we have to
> + * guarantee that only one thread will update state[] and stage's tail values.
> + */
> +
> +#include "soring.h"
> +
> +/*
> + * Inline functions (fastpath) start here.
> + */
> +static __rte_always_inline uint32_t
> +__rte_soring_stage_finalize(struct soring_stage_headtail *sht,
> +	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
> +{
> +	int32_t rc;
> +	uint32_t head, i, idx, k, n, tail;
> +	union soring_stage_tail nt, ot;
> +	union soring_state st;
> +
> +	/* try to grab exclusive right to update tail value */
> +	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
> +			rte_memory_order_acquire);
> +
> +	/* other thread already finalizing it for us */
> +	if (ot.sync != 0)
> +		return 0;
> +
> +	nt.pos = ot.pos;
> +	nt.sync = 1;
> +	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
> +		&ot.raw, nt.raw, rte_memory_order_acquire,
> +		rte_memory_order_relaxed);
> +
> +	/* other thread won the race */
> +	if (rc == 0)
> +		return 0;
> +
> +	/*
> +	 * start with current tail and walk through states that are
> +	 * already finished.
> +	 */
> +
> +	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
> +	n = RTE_MIN(head - ot.pos, maxn);
> +	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
> +
> +		idx = tail & rmask;
> +		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
> +			rte_memory_order_relaxed);
> +		if ((st.stnum & RTE_SORING_ST_MASK) != RTE_SORING_ST_FINISH ||
> +				st.ftoken != tail)
> +			break;
> +
> +		k = st.stnum & ~RTE_SORING_ST_MASK;
> +		rte_atomic_store_explicit(&rstate[idx].raw, 0,
> +				rte_memory_order_relaxed);
> +	}
> +
> +
> +	/* release exclusive right to update along with new tail value */
> +	ot.pos = tail;
> +	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
> +			rte_memory_order_release);
> +
> +	return i;
> +}
> +
> +static __rte_always_inline uint32_t
> +__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
> +	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
> +	uint32_t *head, uint32_t *next, uint32_t *free)
> +{
> +	uint32_t n;
> +
> +	switch (st) {
> +	case RTE_RING_SYNC_ST:
> +	case RTE_RING_SYNC_MT:
> +		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
> +			r->capacity, st, num, behavior, head, next, free);
> +		break;
> +	case RTE_RING_SYNC_MT_RTS:
> +		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
> +			r->capacity, num, behavior, head, free);
> +		*next = *head + n;
> +		break;
> +	case RTE_RING_SYNC_MT_HTS:
> +		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
> +			r->capacity, num, behavior, head, free);
> +		*next = *head + n;
> +		break;
> +	default:
> +		/* unsupported mode, shouldn't be here */
> +		RTE_ASSERT(0);
> +		*free = 0;
> +		n = 0;
> +	}
> +
> +	return n;
> +}
> +
> +static __rte_always_inline uint32_t
> +__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
> +	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
> +	uint32_t *head, uint32_t *next, uint32_t *avail)
> +{
> +	uint32_t n;
> +
> +	switch (st) {
> +	case RTE_RING_SYNC_ST:
> +	case RTE_RING_SYNC_MT:
> +		n = __rte_ring_headtail_move_head(&r->cons.ht,
> +			&r->stage[stage].ht, 0, st, num, behavior,
> +			head, next, avail);
> +		break;
> +	case RTE_RING_SYNC_MT_RTS:
> +		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
> +			0, num, behavior, head, avail);
> +		*next = *head + n;
> +		break;
> +	case RTE_RING_SYNC_MT_HTS:
> +		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
> +			0, num, behavior, head, avail);
> +		*next = *head + n;
> +		break;
> +	default:
> +		/* unsupported mode, shouldn't be here */
> +		RTE_ASSERT(0);
> +		*avail = 0;
> +		n = 0;
> +	}
> +
> +	return n;
> +}
> +
> +static __rte_always_inline void
> +__rte_soring_update_tail(struct __rte_ring_headtail *rht,
> +	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
> +{
> +	uint32_t n;
> +
> +	switch (st) {
> +	case RTE_RING_SYNC_ST:
> +	case RTE_RING_SYNC_MT:
> +		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
> +		break;
> +	case RTE_RING_SYNC_MT_RTS:
> +		__rte_ring_rts_update_tail(&rht->rts);
> +		break;
> +	case RTE_RING_SYNC_MT_HTS:
> +		n = next - head;
> +		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
> +		break;
> +	default:
> +		/* unsupported mode, shouldn't be here */
> +		RTE_ASSERT(0);
> +	}
> +}
> +
> +static __rte_always_inline uint32_t
> +__rte_soring_stage_move_head(struct soring_stage_headtail *d,
> +	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
> +	enum rte_ring_queue_behavior behavior,
> +	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
> +{
> +	uint32_t n, tail;
> +
> +	*old_head = rte_atomic_load_explicit(&d->head,
> +			rte_memory_order_acquire);
> +
> +	do {
> +		n = num;
> +		tail = rte_atomic_load_explicit(&s->tail,
> +				rte_memory_order_relaxed);
> +		*avail = capacity + tail - *old_head;
> +		if (n > *avail)
> +			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
> +		if (n == 0)
> +			return 0;
> +		*new_head = *old_head + n;
> +	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
> +			old_head, *new_head, rte_memory_order_acquire,
> +			rte_memory_order_acquire) == 0);
> +
> +	return n;
> +}
> +
> +/*
> + * Public functions (data-path) start here.
> + */
> +
> +unsigned int
> +rte_soring_count(const struct rte_soring *r)
> +{
> +	uint32_t prod_tail = r->prod.ht.tail;
> +	uint32_t cons_tail = r->cons.ht.tail;
> +	uint32_t count = (prod_tail - cons_tail) & r->mask;
> +	return (count > r->capacity) ? r->capacity : count;
> +}
> +
> +uint32_t
> +rte_soring_enqueue(struct rte_soring *r, const void *obj_table,
> +	const void *objst, uint32_t n, enum rte_ring_queue_behavior behavior,
> +	uint32_t *free_space)
> +{
> +	enum rte_ring_sync_type st;
> +	uint32_t nb_free, prod_head, prod_next;
> +
> +	RTE_ASSERT(r != NULL && r->nb_stage > 0);
> +	RTE_ASSERT(objst == NULL || r->elemst != NULL);
> +
> +	st = r->prod.ht.sync_type;
> +
> +	n = __rte_soring_move_prod_head(r, n, behavior, st,
> +			&prod_head, &prod_next, &nb_free);
> +	if (n != 0) {
> +		__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
> +			prod_head & r->mask, r->esize, n);
> +		if (objst != NULL)
> +			__rte_ring_do_enqueue_elems(r->elemst, objst, r->size,
> +				prod_head & r->mask, r->stsize, n);
> +		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
> +	}
> +
> +	if (free_space != NULL)
> +		*free_space = nb_free - n;
> +	return n;
> +}
> +
> +uint32_t
> +rte_soring_dequeue(struct rte_soring *r, void *obj_table, void *objst,
> +	uint32_t num, enum rte_ring_queue_behavior behavior,
> +	uint32_t *available)
> +{
> +	enum rte_ring_sync_type st;
> +	uint32_t entries, cons_head, cons_next, n, ns, reqn;
> +
> +	RTE_ASSERT(r != NULL && r->nb_stage > 0);
> +	RTE_ASSERT(objst == NULL || r->elemst != NULL);
> +
> +	ns = r->nb_stage - 1;
> +	st = r->cons.ht.sync_type;
> +
> +	/* try to grab exactly @num elems first */
> +	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
> +			&cons_head, &cons_next, &entries);
> +	if (n == 0) {
> +		/* try to finalize some elems from previous stage */
> +		n = __rte_soring_stage_finalize(&r->stage[ns].sht,
> +			r->state, r->mask, 2 * num);
> +		entries += n;
> +
> +		/* repeat attempt to grab elems */
> +		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
> +		if (entries >= reqn)
> +			n = __rte_soring_move_cons_head(r, ns, num, behavior,
> +				st, &cons_head, &cons_next, &entries);
> +		else
> +			n = 0;
> +	}
> +
> +	/* we have some elems to consume */
> +	if (n != 0) {
> +		__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
> +			cons_head & r->mask, r->esize, n);
> +		if (objst != NULL)
> +			__rte_ring_do_dequeue_elems(objst, r->elemst, r->size,
> +				cons_head & r->mask, r->stsize, n);
> +		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
> +	}
> +
> +	if (available != NULL)
> +		*available = entries - n;
> +	return n;
> +}
> +
> +uint32_t
> +rte_soring_acquire(struct rte_soring *r, void *objs, void *objst,
> +	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
> +	uint32_t *ftoken, uint32_t *available)
> +{
> +	uint32_t avail, head, idx, n, next, reqn;
> +	union soring_state st;
> +	struct soring_stage *pstg;
> +	struct soring_stage_headtail *cons;
> +
> +	RTE_ASSERT(r != NULL && stage < r->nb_stage);
> +	RTE_ASSERT(objst == NULL || r->elemst != NULL);
> +
> +	cons = &r->stage[stage].sht;
> +
> +	if (stage == 0)
> +		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
> +			behavior, &head, &next, &avail);
> +	else {
> +		pstg = r->stage + stage - 1;
> +
> +		/* try to grab exactly @num elems */
> +		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
> +			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
> +		if (n == 0) {
> +			/* try to finalize some elems from previous stage */
> +			n = __rte_soring_stage_finalize(&pstg->sht, r->state,
> +				r->mask, 2 * num);
> +			avail += n;
> +
> +			/* repeat attempt to grab elems */
> +			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
> +			if (avail >= reqn)
> +				n = __rte_soring_stage_move_head(cons,
> +					&pstg->ht, 0, num, behavior, &head,
> +					&next, &avail);
> +			else
> +				n = 0;
> +		}
> +	}
> +
> +	if (n != 0) {
> +
> +		idx = head & r->mask;
> +
> +		/* copy elems that are ready for given stage */
> +		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
> +				r->esize, n);
> +		if (objst != NULL)
> +			__rte_ring_do_dequeue_elems(objst, r->elemst,
> +				r->size, idx, r->stsize, n);
> +
> +		/* update status ring */
> +		st.ftoken = head;
> +		st.stnum = (RTE_SORING_ST_START | n);
> +
> +		rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
> +				rte_memory_order_relaxed);
> +		*ftoken = head;
> +	}
> +
> +	if (available != NULL)
> +		*available = avail - n;
> +	return n;
> +}
> +
> +void
> +rte_soring_release(struct rte_soring *r, const void *objs,
> +	const void *objst, uint32_t stage, uint32_t n, uint32_t ftoken)
> +{
> +	uint32_t idx, tail;
> +	struct soring_stage *stg;
> +	union soring_state st;
> +
> +	RTE_ASSERT(r != NULL && stage < r->nb_stage);
> +	RTE_ASSERT(objst == NULL || r->elemst != NULL);
> +
> +	stg = r->stage + stage;
> +	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
> +			rte_memory_order_relaxed);
> +
> +	idx = ftoken & r->mask;
> +	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
> +			rte_memory_order_acquire);
> +
> +	/* check state ring contents */
> +	RTE_VERIFY(st.stnum == (RTE_SORING_ST_START | n) &&
> +		st.ftoken == ftoken);
> +
> +	/* update contents of the ring, if necessary */
> +	if (objs != NULL)
> +		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
> +			r->esize, n);
> +	if (objst != NULL)
> +		__rte_ring_do_enqueue_elems(r->elemst, objst, r->size, idx,
> +			r->stsize, n);
> +
> +	/* set state to FINISH, make sure it is not reordered */
> +	st.stnum = RTE_SORING_ST_FINISH | n;
> +	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
> +			rte_memory_order_release);
> +
> +	if (tail == ftoken)
> +		__rte_soring_stage_finalize(&stg->sht, r->state, r->mask,
> +				r->capacity);
> +}
> diff --git a/lib/ring/soring.h b/lib/ring/soring.h
> new file mode 100644
> index 0000000000..3a3f6efa76
> --- /dev/null
> +++ b/lib/ring/soring.h
> @@ -0,0 +1,124 @@
> +/* SPDX-License-Identifier: BSD-3-Clause
> + * Copyright(c) 2024 Huawei Technologies Co., Ltd
> + */
> +
> +#ifndef _SORING_H_
> +#define _SORING_H_
> +
> +/**
> + * @file
> + * This file contains internal strctures of RTE soring: Staged Ordered Ring.
> + * Sort of extension of conventional RTE ring.
> + * Internal structure:
> + * In addition to 'normal' ring of elems, it also has a ring of states of the
> + * same size. Each state[] corresponds to exactly one elem[].
> + * state[] will be used by acquire/release/dequeue functions to store internal
> + * information and should not be accessed by the user directly.
> + * For actual implementation details, please refer to soring.c
> + */
> +
> +#include <rte_soring.h>
> +
> +union soring_state {
> +	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
> +	struct {
> +		RTE_ATOMIC(uint32_t) ftoken;
> +		RTE_ATOMIC(uint32_t) stnum;
> +	};
> +};
> +
> +/* upper 2 bits are used for status */
> +#define RTE_SORING_ST_START	RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
> +#define RTE_SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
> +
> +#define RTE_SORING_ST_MASK	\
> +	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
> +
> +/**
> + * SORING re-uses rte_ring internal structures and implementation
> + * for enqueue/dequeue operations.
> + */
> +struct __rte_ring_headtail {
> +
> +	union __rte_cache_aligned {
> +		struct rte_ring_headtail ht;
> +		struct rte_ring_hts_headtail hts;
> +		struct rte_ring_rts_headtail rts;
> +	};
> +
> +	RTE_CACHE_GUARD;
> +};
> +
> +union soring_stage_tail {
> +	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
> +	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
> +	struct {
> +		RTE_ATOMIC(uint32_t) sync;
> +		RTE_ATOMIC(uint32_t) pos;
> +	};
> +};
> +
> +struct soring_stage_headtail {
> +	volatile union soring_stage_tail tail;
> +	enum rte_ring_sync_type unused;  /**< unused */
> +	volatile RTE_ATOMIC(uint32_t) head;
> +};
> +
> +/**
> + * SORING stage head_tail structure is 'compatible' with rte_ring ones.
> + * rte_ring internal functions for moving producer/consumer head
> + * can work transparently with stage head_tail and visa-versa
> + * (no modifications required).
> + */
> +struct soring_stage {
> +
> +	union __rte_cache_aligned {
> +		struct rte_ring_headtail ht;
> +		struct soring_stage_headtail sht;
> +	};
> +
> +	RTE_CACHE_GUARD;
> +};
> +
> +/**
> + * RTE soring internal structure.
> + * As with rte_ring actual elements array supposed to be located direclty
> + * after the rte_soring structure.
> + */
> +struct  __rte_cache_aligned rte_soring {
> +	uint32_t size;           /**< Size of ring. */
> +	uint32_t mask;           /**< Mask (size-1) of ring. */
> +	uint32_t capacity;       /**< Usable size of ring */
> +	uint32_t esize;
> +	/**< size of elements in the ring, must be a multiple of 4*/
> +	uint32_t stsize;
> +	/**< size of status value for each elem, must be a multiple of 4 */
> +
> +	/** Ring stages */
> +	struct soring_stage *stage;
> +	uint32_t nb_stage;
> +
> +	/** Ring of states (one per element) */
> +	union soring_state *state;
> +
> +	/** Pointer to the buffer where status values for each elements
> +	 * are stored. This is supplementary and optional information that
> +	 * user can attach to each element of the ring.
> +	 * While it is possible to incorporate this information inside
> +	 * user-defined element, in many cases it is plausible to maintain it
> +	 * as a separate array (mainly for performance reasons).
> +	 */
> +	void *elemst;
> +
> +	RTE_CACHE_GUARD;
> +
> +	/** Ring producer status. */
> +	struct __rte_ring_headtail prod;
> +
> +	/** Ring consumer status. */
> +	struct __rte_ring_headtail cons;
> +
> +	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
> +};
> +
> +#endif /* _SORING_H_ */
> diff --git a/lib/ring/version.map b/lib/ring/version.map
> index 8da094a69a..a1f95a500f 100644
> --- a/lib/ring/version.map
> +++ b/lib/ring/version.map
> @@ -14,3 +14,16 @@ DPDK_25 {
>   
>   	local: *;
>   };
> +
> +EXPERIMENTAL {
> +	global:
> +
> +	# added in 24.11
> +	rte_soring_acquire;
> +	rte_soring_count;
> +	rte_soring_dequeue;
> +	rte_soring_enqueue;
> +	rte_soring_get_memsize;
> +	rte_soring_init;
> +	rte_soring_release;
> +};

^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [RFC 3/6] ring/soring: introduce Staged Ordered Ring
  2024-08-26 19:04   ` Mattias Rönnblom
@ 2024-09-03 13:55     ` Konstantin Ananyev
  0 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-03 13:55 UTC (permalink / raw)
  To: Mattias Rönnblom, Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb



> > Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> > with multiple processing 'stages'.
> > It is based on conventional DPDK rte_ring, re-uses many of its concepts,
> > and even substantial part of its code.
> > It can be viewed as an 'extension' of rte_ring functionality.
> > In particular, main SORING properties:
> > - circular ring buffer with fixed size objects
> > - producer, consumer plus multiple processing stages in the middle.
> > - allows to split objects processing into multiple stages.
> > - objects remain in the same ring while moving from one stage to the other,
> >    initial order is preserved, no extra copying needed.
> > - preserves the ingress order of objects within the queue across multiple
> >    stages, i.e.:
> >    at the same stage multiple threads can process objects from the ring in
> >    any order, but for the next stage objects will always appear in the
> >    original order.
> > - each stage (and producer/consumer) can be served by single and/or
> >    multiple threads.
> > - number of stages, size and number of objects in the ring are
> >    configurable at ring initialization time.
> >
> > Data-path API provides four main operations:
> > - enqueue/dequeue works in the same manner as for conventional rte_ring,
> >    all rte_ring synchronization types are supported.
> > - acquire/release - for each stage there is an acquire (start) and
> >    release (finish) operation.
> >    after some objects are 'acquired' - given thread can safely assume that
> >    it has exclusive possession of these objects till 'release' for them is
> >    invoked.
> >    Note that right now user has to release exactly the same number of
> >    objects that was acquired before.
> >    After 'release', objects can be 'acquired' by next stage and/or dequeued
> >    by the consumer (in case of last stage).
> >
> > Expected use-case: applications that uses pipeline model
> > (probably with multiple stages) for packet processing, when preserving
> > incoming packet order is important. I.E.: IPsec processing, etc.
> >
> 
> How does SORING related to Eventdev?

So far there is no direct relation. 
Though yes, DPDK eventdev framework also provides ‘ordered’ queue ability (along with other modes).
Again, as I mentioned in the cover-letter rte_soring uses similar concept as OPDL eventdev implementation. 

One of the main aims with rte_soring was to introduce  sort of extension to rte_ring,
while keeping its API and implementation as lightweight and generic as possible.
So it could be consumed by various apps that do use pipeline model,
but for whatever reason do not use (/plan to use) eventdev framework.

> Would it be feasible to reshape  this into a SW event device?

I guess an opposite approach might work better -  i.e. make some SW-based eventdev
Implementation to use rte_soring internally.
Though I didn't try to do that so far. 


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 0/6] Stage-Ordered API and other extensions for ring library
  2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                   ` (5 preceding siblings ...)
  2024-08-15  8:53 ` [RFC 6/6] ring: minimize reads of the counterpart cache-line Konstantin Ananyev
@ 2024-09-06 13:13 ` Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
                     ` (6 more replies)
  6 siblings, 7 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

v1 -> v2:
- dropped 
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring. 
- dropped patch from v1:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will 
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.
- patch #6 extends l3fwd sample app to work in pipeline (worker-pool) mode.
  Right now it is done for demonstration and performance comparison
  purposes, as it makes possible to run l3fwd in different modes:
  run-to-completion, eventdev, pipeline
  and perform sort-of 'apple-to-apple' performance comparisons.
  I am aware that in general community consensus on l3fwd is to keep its
  functionality simple and limited. From other side we already do have
  eventdev  mode for it, so why pipeline should be prohibited?
  Though if l3fwd is not an option, then we need to select some other
  existing sample app to integrate with. Probably ipsec-secgw would be the
  second best choice from my perspective, though it would require much more
  effort.
  Have to say that current l3fwd patch is way too big and unfinished,
  so if we'll decide to go forward with it, it has to be split and reworked.

Seeking community help/feedback (apart from usual patch review activity):
=========================================================================
- While we tested these changes quite extensively, our platform coverage
  is limitedi to x86.
  So would appreciate the feedback how it behaves on other architectures
  DPDK supports (ARM, PPC, etc.).
- Adding new (pipeline) mode for l3fwd sample app.
  Is it worth it? If not, what other sample app should be used to
  demonstrate new functionality we worked on? ipsec-secgw? Something else?

SORING overview
==============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK rte_ring,
re-uses many of its concepts, and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' - given thread
  can safely assume that it has exclusive ownership of these objects till
  it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode (see patch #5 for details) both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (5):
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  examples/l3fwd: make ACL work in pipeline and eventdev modes

 .mailmap                           |   1 +
 app/test/meson.build               |   3 +
 app/test/test_ring_stress_impl.h   |   1 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 827 +++++++++++++++++++++++++++++
 examples/l3fwd/l3fwd.h             |  55 ++
 examples/l3fwd/l3fwd_acl.c         | 125 ++++-
 examples/l3fwd/l3fwd_acl_event.h   | 258 +++++++++
 examples/l3fwd/l3fwd_event.c       |  14 +
 examples/l3fwd/l3fwd_event.h       |   1 +
 examples/l3fwd/l3fwd_sse.h         |  49 +-
 examples/l3fwd/l3fwd_wqp.c         | 274 ++++++++++
 examples/l3fwd/l3fwd_wqp.h         | 130 +++++
 examples/l3fwd/main.c              |  75 ++-
 examples/l3fwd/meson.build         |   1 +
 lib/ring/meson.build               |   4 +-
 lib/ring/rte_ring.c                |  87 ++-
 lib/ring/rte_ring.h                |  15 +
 lib/ring/rte_ring_c11_pvt.h        | 134 +----
 lib/ring/rte_ring_elem_pvt.h       | 183 +++++--
 lib/ring/rte_ring_generic_pvt.h    | 121 +----
 lib/ring/rte_ring_hts_elem_pvt.h   |  85 +--
 lib/ring/rte_ring_rts_elem_pvt.h   |  85 +--
 lib/ring/rte_soring.c              | 182 +++++++
 lib/ring/rte_soring.h              | 547 +++++++++++++++++++
 lib/ring/soring.c                  | 548 +++++++++++++++++++
 lib/ring/soring.h                  | 124 +++++
 lib/ring/version.map               |  26 +
 31 files changed, 4095 insertions(+), 425 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 examples/l3fwd/l3fwd_acl_event.h
 create mode 100644 examples/l3fwd/l3fwd_wqp.c
 create mode 100644 examples/l3fwd/l3fwd_wqp.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 1/6] ring: common functions for 'move head' ops
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-09-06 13:13   ` Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 2/6] ring: make copying functions generic Konstantin Ananyev
                     ` (5 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 2/6] ring: make copying functions generic
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-09-06 13:13   ` Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 3/6] ring: make dump function more verbose Konstantin Ananyev
                     ` (4 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 117 ++++++++++++++++++++---------------
 1 file changed, 68 insertions(+), 49 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..216cb6089f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
-	uint32_t *obj = (uint32_t *)obj_table;
+	uint32_t *obj = obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 3/6] ring: make dump function more verbose
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 2/6] ring: make copying functions generic Konstantin Ananyev
@ 2024-09-06 13:13   ` Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 4/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                     ` (3 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Eimear Morrissey

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 4a508bafad..3da86393c0 100644
--- a/.mailmap
+++ b/.mailmap
@@ -379,6 +379,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..8449cb4b15 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index c709f30497..d6f9e85c64 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 4/6] ring/soring: introduce Staged Ordered Ring
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                     ` (2 preceding siblings ...)
  2024-09-06 13:13   ` [RFCv2 3/6] ring: make dump function more verbose Konstantin Ananyev
@ 2024-09-06 13:13   ` Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 5/6] app/test: add unit tests for soring API Konstantin Ananyev
                     ` (2 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev, Eimear Morrissey

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/meson.build  |   4 +-
 lib/ring/rte_soring.c | 182 ++++++++++++++
 lib/ring/rte_soring.h | 547 +++++++++++++++++++++++++++++++++++++++++
 lib/ring/soring.c     | 548 ++++++++++++++++++++++++++++++++++++++++++
 lib/ring/soring.h     | 124 ++++++++++
 lib/ring/version.map  |  19 ++
 6 files changed, 1422 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..b6bc71b8c9
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,182 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..dd52a507e3
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,547 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/* max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the ring */
+	const char *name;
+	/** number of elemnts in the ring */
+	uint32_t elems;
+	/** size of elements in the ring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see @elem_size parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the ring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a ring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual ring elements and theri rec-codes. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use rte_ring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the @objs.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or @n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then @meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the @objs.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or @n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the @objs.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then @meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the @objs.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or @num.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then @meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or @num.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then @meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or @num.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then @meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or @num.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then @meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ * @return
+ *   - None.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as @elem_size parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as @meta_size
+ *   parameter used while creating the ring. If user created the soring with
+ *   @meta_size value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ * @return
+ *   - None.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..97fa38df66
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,548 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (RTE_SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'RTE_SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in RTE_SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		&ot.raw, nt.raw, rte_memory_order_acquire,
+		rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & RTE_SORING_ST_MASK) != RTE_SORING_ST_FINISH ||
+				st.ftoken != tail)
+			break;
+
+		k = st.stnum & ~RTE_SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_acquire);
+
+	do {
+		n = num;
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_relaxed);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	union soring_state st;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, r->state,
+				r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+
+		/* update status ring */
+		st.ftoken = head;
+		st.stnum = (RTE_SORING_ST_START | n);
+
+		rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+				rte_memory_order_relaxed);
+		*ftoken = head;
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+
+	idx = ftoken & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_acquire);
+
+	/* check state ring contents */
+	RTE_VERIFY(st.stnum == (RTE_SORING_ST_START | n) &&
+		st.ftoken == ftoken);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	st.stnum = RTE_SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_release);
+
+	if (tail == ftoken)
+		__rte_soring_stage_finalize(&stg->sht, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	return soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	return soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..c44d8912d9
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,124 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_START	RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define RTE_SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define RTE_SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 5/6] app/test: add unit tests for soring API
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                     ` (3 preceding siblings ...)
  2024-09-06 13:13   ` [RFCv2 4/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-09-06 13:13   ` Konstantin Ananyev
  2024-09-06 13:13   ` [RFCv2 6/6] examples/l3fwd: make ACL work in pipeline and eventdev modes Konstantin Ananyev
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev, Eimear Morrissey

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 827 +++++++++++++++++++++++++++++
 6 files changed, 1395 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index e29258e6ec..c290162e43 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -175,6 +175,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..1ba78a31f5
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,827 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill)
+{
+	uint32_t i;
+
+	static rte_spinlock_t dump_lock;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				__func__, rte_lcore_id(), num, i, elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname)
+{
+	if (bhv == RTE_RING_QUEUE_FIXED && exp != res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	if (bhv == RTE_RING_QUEUE_VARIABLE && exp < res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire));
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk));
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk));
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [RFCv2 6/6] examples/l3fwd: make ACL work in pipeline and eventdev modes
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                     ` (4 preceding siblings ...)
  2024-09-06 13:13   ` [RFCv2 5/6] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-09-06 13:13   ` Konstantin Ananyev
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-06 13:13 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront:
This is a huge commit that is combined from several ones.
For now, I submit it just for reference and demonstration purposes and
will probably remove it in future versions.
If will decide to go ahead with it, then it needs to be reworked and split
into several proper commits.

It adds for l3fwd:
 - eventdev mode for ACL lookup-mode
 - Introduce a worker-pool-mode
   (right now implemented for ACL lookup-mode only).
Worker-Pool mode is a simple pipeline model, with the following stages:
 1) I/O thread receives packets from NIC RX HW queues and enqueues them
    into the work queue
 2) Worker thread reads packets from the work queue(s),
    process them and then puts processed packets back into the
    work queue along with the processing status (routing info/error code).
 3) I/O thread dequeues packets and their status from the work queue,
    and based on it either TX packet or drops it.
Very similar to l3fwd-eventdev working model.

Note that it could be several I/O threads, each can serve one or multiple
HW RX queues. Also there could be several Worker threads, each of them can
process packets from multiple work queues in round-robin fashion.

Work queue can be one of the following types:
 - wqorder: allows Worker threads to process packets in any order,
   but guarantees that on dequeue stage the ingress order of packets
   will be preserved. I.E. at stage #3, I/O thread will get packets
   exactly in the same order as they were enqueued at stage #1.
 - wqunorder: doesn't provide any ordered guarantees.

'wqunroder' mode is implemented using 2 rte_ring structures per queue.
'wqorder' mode is implemtened using rte_soring structure per queue.

To facilitate this new functionality, command line parameters were
extended:
 --mode:
   Possible values one of: poll/eventdev/wqorder/wqorderS/wqunorder/wqunorderS
   Default value: poll
   - wqorder: Worker-Pool ordered mode with a separate work queue for each
     HW RX queue.
   - wqorderS: Worker-Pool ordered mode with one work queue per I/O thread.
   - wqunorder: Worker-Pool un-ordered mode with a separate work queue for each
     HW RX queue.
   - wqunorderS: Worker-Pool un-ordered mode with oen work queue per I/O thread.
 --wqsize: number of elements for each worker queue.
 --lookup-iter: forces to perform ACL lookup several times over the same
   packet. This is artificial parameter and is added temporally for
   benchmarking purposes. Will be removed in latest versions (if any).

Note that in Worker-Pool mode all free lcores that were not assigned as
I/O threads will be used as Worker threads.
As an example:
dpdk-l3fwd --lcores=53,55,57,59,61 ... -- \
-P -p f --config '(0,0,53)(1,0,53)(2,0,53)(3,0,53)' --lookup acl \
--parse-ptype --mode=wqorder ...
In that case lcore 53 will be used as I/O thread (stages #1,3)
to serve 4 HW RX queues,
while lcores 55,57,59,61 will serve as Worker threads (stage #2).

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 examples/l3fwd/l3fwd.h           |  55 +++++++
 examples/l3fwd/l3fwd_acl.c       | 125 +++++++++++---
 examples/l3fwd/l3fwd_acl_event.h | 258 +++++++++++++++++++++++++++++
 examples/l3fwd/l3fwd_event.c     |  14 ++
 examples/l3fwd/l3fwd_event.h     |   1 +
 examples/l3fwd/l3fwd_sse.h       |  49 +++++-
 examples/l3fwd/l3fwd_wqp.c       | 274 +++++++++++++++++++++++++++++++
 examples/l3fwd/l3fwd_wqp.h       | 130 +++++++++++++++
 examples/l3fwd/main.c            |  75 ++++++++-
 examples/l3fwd/meson.build       |   1 +
 10 files changed, 954 insertions(+), 28 deletions(-)
 create mode 100644 examples/l3fwd/l3fwd_acl_event.h
 create mode 100644 examples/l3fwd/l3fwd_wqp.c
 create mode 100644 examples/l3fwd/l3fwd_wqp.h

diff --git a/examples/l3fwd/l3fwd.h b/examples/l3fwd/l3fwd.h
index 93ce652d02..218f363764 100644
--- a/examples/l3fwd/l3fwd.h
+++ b/examples/l3fwd/l3fwd.h
@@ -77,6 +77,42 @@ struct __rte_cache_aligned lcore_rx_queue {
 	uint16_t queue_id;
 };
 
+enum L3FWD_WORKER_MODE {
+	L3FWD_WORKER_POLL,
+	L3FWD_WORKER_UNQUE,
+	L3FWD_WORKER_ORQUE,
+};
+
+struct l3fwd_wqp_param {
+	enum L3FWD_WORKER_MODE mode;
+	uint32_t qsize;    /**< Number of elems in worker queue */
+	int32_t single;    /**< use single queue per I/O (poll) thread */
+};
+
+extern struct l3fwd_wqp_param l3fwd_wqp_param;
+
+enum {
+	LCORE_WQ_IN,
+	LCORE_WQ_OUT,
+	LCORE_WQ_NUM,
+};
+
+union lcore_wq {
+	struct rte_ring *r[LCORE_WQ_NUM];
+	struct {
+		struct rte_soring *sor;
+		/* used by WQ, sort of thred-local var */
+		uint32_t ftoken;
+	};
+};
+
+struct lcore_wq_pool {
+	uint32_t nb_queue;
+	uint32_t qmask;
+	union lcore_wq queue[MAX_RX_QUEUE_PER_LCORE];
+	struct l3fwd_wqp_param prm;
+};
+
 struct __rte_cache_aligned lcore_conf {
 	uint16_t n_rx_queue;
 	struct lcore_rx_queue rx_queue_list[MAX_RX_QUEUE_PER_LCORE];
@@ -86,6 +122,7 @@ struct __rte_cache_aligned lcore_conf {
 	struct mbuf_table tx_mbufs[RTE_MAX_ETHPORTS];
 	void *ipv4_lookup_struct;
 	void *ipv6_lookup_struct;
+	struct lcore_wq_pool wqpool;
 };
 
 extern volatile bool force_quit;
@@ -115,6 +152,8 @@ extern struct acl_algorithms acl_alg[];
 
 extern uint32_t max_pkt_len;
 
+extern uint32_t l3fwd_lookup_iter_num;
+
 /* Send burst of packets on an output interface */
 static inline int
 send_burst(struct lcore_conf *qconf, uint16_t n, uint16_t port)
@@ -308,6 +347,22 @@ fib_event_main_loop_tx_q_vector(__rte_unused void *dummy);
 int
 fib_event_main_loop_tx_q_burst_vector(__rte_unused void *dummy);
 
+int
+acl_event_main_loop_tx_d(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_d_burst(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q_burst(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_d_vector(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_d_burst_vector(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q_vector(__rte_unused void *dummy);
+int
+acl_event_main_loop_tx_q_burst_vector(__rte_unused void *dummy);
 
 /* Return ipv4/ipv6 fwd lookup struct for ACL, LPM, EM or FIB. */
 void *
diff --git a/examples/l3fwd/l3fwd_acl.c b/examples/l3fwd/l3fwd_acl.c
index b635011ef7..8b550e495a 100644
--- a/examples/l3fwd/l3fwd_acl.c
+++ b/examples/l3fwd/l3fwd_acl.c
@@ -4,6 +4,7 @@
 
 #include "l3fwd.h"
 #include "l3fwd_route.h"
+#include "l3fwd_wqp.h"
 
 /*
  * Rule and trace formats definitions.
@@ -1003,19 +1004,21 @@ acl_process_pkts(struct rte_mbuf *pkts[MAX_PKT_BURST],
 	/* split packets burst depending on packet type (IPv4/IPv6) */
 	l3fwd_acl_prepare_acl_parameter(pkts, &acl_search, num);
 
-	if (acl_search.num_ipv4)
-		rte_acl_classify(acl_config.acx_ipv4[socketid],
+	for (i = l3fwd_lookup_iter_num; i != 0; i--) {
+		if (acl_search.num_ipv4)
+			rte_acl_classify(acl_config.acx_ipv4[socketid],
 				acl_search.data_ipv4,
 				acl_search.res_ipv4,
 				acl_search.num_ipv4,
 				DEFAULT_MAX_CATEGORIES);
 
-	if (acl_search.num_ipv6)
-		rte_acl_classify(acl_config.acx_ipv6[socketid],
+		if (acl_search.num_ipv6)
+			rte_acl_classify(acl_config.acx_ipv6[socketid],
 				acl_search.data_ipv6,
 				acl_search.res_ipv6,
 				acl_search.num_ipv6,
 				DEFAULT_MAX_CATEGORIES);
+	}
 
 	/* combine lookup results back, into one array of next hops */
 	n4 = 0;
@@ -1042,34 +1045,36 @@ acl_process_pkts(struct rte_mbuf *pkts[MAX_PKT_BURST],
 
 static inline void
 acl_send_packets(struct lcore_conf *qconf, struct rte_mbuf *pkts[],
-	uint16_t hops[], uint32_t num)
+	uint16_t hops[], uint32_t num, int step3)
 {
 #if defined ACL_SEND_MULTI
-	send_packets_multi(qconf, pkts, hops, num);
+	__send_packets_multi(qconf, pkts, hops, num, step3);
 #else
-	send_packets_single(qconf, pkts, hops, num);
+	if (step3 != 0)
+		send_packets_single(qconf, pkts, hops, num);
+	else {
+		uint32_t i;
+		for (i = 0; i != num; i++)
+			send_single_packet(qconf, pkts[i], hops[i]);
+	}
 #endif
 }
 
 /* main processing loop */
-int
-acl_main_loop(__rte_unused void *dummy)
+static int
+acl_poll_loop(struct lcore_conf *qconf, uint32_t lcore_id)
 {
 	struct rte_mbuf *pkts_burst[MAX_PKT_BURST];
 	uint16_t hops[MAX_PKT_BURST];
-	unsigned int lcore_id;
 	uint64_t prev_tsc, diff_tsc, cur_tsc;
-	int i, nb_rx;
+	uint32_t i, n, nb_rx;
 	uint16_t portid;
 	uint16_t queueid;
-	struct lcore_conf *qconf;
 	int socketid;
 	const uint64_t drain_tsc = (rte_get_tsc_hz() + US_PER_S - 1)
 			/ US_PER_S * BURST_TX_DRAIN_US;
 
 	prev_tsc = 0;
-	lcore_id = rte_lcore_id();
-	qconf = &lcore_conf[lcore_id];
 	socketid = rte_lcore_to_socket_id(lcore_id);
 
 	if (qconf->n_rx_queue == 0) {
@@ -1121,17 +1126,99 @@ acl_main_loop(__rte_unused void *dummy)
 			nb_rx = rte_eth_rx_burst(portid, queueid,
 				pkts_burst, MAX_PKT_BURST);
 
-			if (nb_rx > 0) {
-				acl_process_pkts(pkts_burst, hops, nb_rx,
-					socketid);
-				acl_send_packets(qconf, pkts_burst, hops,
-					nb_rx);
+			if (nb_rx != 0) {
+				if (l3fwd_wqp_param.mode == L3FWD_WORKER_POLL) {
+					acl_process_pkts(pkts_burst, hops,
+						nb_rx, socketid);
+					acl_send_packets(qconf, pkts_burst,
+						hops, nb_rx, 1);
+				} else {
+					n = lcore_wq_submit(&qconf->wqpool, i,
+						pkts_burst, nb_rx);
+					if (n != nb_rx) {
+						/* update stats counter */
+						rte_pktmbuf_free_bulk(
+							pkts_burst + n,
+							nb_rx - n);
+					}
+				}
+			}
+			if (l3fwd_wqp_param.mode != L3FWD_WORKER_POLL) {
+				nb_rx = lcore_wq_receive(&qconf->wqpool, i,
+					pkts_burst, hops, MAX_PKT_BURST);
+				if (nb_rx != 0)
+					acl_send_packets(qconf, pkts_burst,
+						hops, nb_rx, 0);
+			}
+		}
+	}
+	return 0;
+}
+
+/* WT processing loop */
+static int
+acl_wqp_loop(struct lcore_conf *qconf, uint32_t lcore_id)
+{
+	int32_t socketid;
+	uint32_t i, k, n;
+	struct rte_mbuf *pkts[MAX_PKT_BURST];
+	uint16_t hops[MAX_PKT_BURST];
+
+	socketid = rte_lcore_to_socket_id(lcore_id);
+
+	if (qconf->wqpool.nb_queue == 0) {
+		RTE_LOG(INFO, L3FWD, "%s: lcore %u has nothing to do\n",
+			__func__, lcore_id);
+		return 0;
+	}
+
+	RTE_LOG(INFO, L3FWD, "%s: entering loop on lcore %u\n",
+		__func__, lcore_id);
+
+	while (!force_quit) {
+
+		/*
+		 * Read packet from internal queues and process them
+		 */
+		for (i = 0; i < qconf->wqpool.nb_queue; ++i) {
+
+			n = lcore_wq_pull(&qconf->wqpool, i, pkts,
+				RTE_DIM(pkts));
+			if (n == 0)
+				continue;
+
+			acl_process_pkts(pkts, hops, n, socketid);
+			process_step3_burst(pkts, hops, n);
+			k = lcore_wq_push(&qconf->wqpool, i, pkts, hops, n);
+			if (n != k) {
+				/* stats update */
+				rte_pktmbuf_free_bulk(pkts + k, n - k);
 			}
 		}
 	}
 	return 0;
 }
 
+/* main processing loop */
+int
+acl_main_loop(__rte_unused void *dummy)
+{
+	uint32_t lcore_id;
+	struct lcore_conf *qconf;
+
+	lcore_id = rte_lcore_id();
+	qconf = &lcore_conf[lcore_id];
+
+	if (qconf->n_rx_queue != 0)
+		return acl_poll_loop(qconf, lcore_id);
+	else
+		return acl_wqp_loop(qconf, lcore_id);
+}
+
+#ifdef RTE_LIB_EVENTDEV
+#include "l3fwd_acl_event.h"
+#endif
+
 /* Not used by L3fwd ACL. */
 void *
 acl_get_ipv4_l3fwd_lookup_struct(__rte_unused const int socketid)
diff --git a/examples/l3fwd/l3fwd_acl_event.h b/examples/l3fwd/l3fwd_acl_event.h
new file mode 100644
index 0000000000..240dd3fb03
--- /dev/null
+++ b/examples/l3fwd/l3fwd_acl_event.h
@@ -0,0 +1,258 @@
+#include "l3fwd_event.h"
+
+/* One eventdev loop for single and burst using acl. */
+static __rte_always_inline void
+acl_event_loop(struct l3fwd_event_resources *evt_rsrc,
+		const uint8_t flags)
+{
+	uint32_t i, lcore_id, nb_deq, nb_enq;
+	int32_t socketid;
+	uint16_t hops[MAX_PKT_BURST];
+	struct rte_mbuf *pkts[MAX_PKT_BURST];
+	struct rte_event events[MAX_PKT_BURST];
+
+	const int event_p_id = l3fwd_get_free_event_port(evt_rsrc);
+	const uint8_t tx_q_id = evt_rsrc->evq.event_q_id[
+			evt_rsrc->evq.nb_queues - 1];
+	const uint8_t event_d_id = evt_rsrc->event_d_id;
+	const uint16_t deq_len = RTE_MIN(evt_rsrc->deq_depth, MAX_PKT_BURST);
+
+	if (event_p_id < 0)
+		return;
+
+	lcore_id = rte_lcore_id();
+	socketid = rte_lcore_to_socket_id(lcore_id);
+
+	RTE_LOG(INFO, L3FWD, "entering %s on lcore %u\n", __func__, lcore_id);
+
+	nb_deq = 0;
+	nb_enq = 0;
+
+	while (!force_quit) {
+		/* Read events from RX queues. */
+		nb_deq = rte_event_dequeue_burst(event_d_id, event_p_id,
+				events, deq_len, 0);
+		if (nb_deq == 0) {
+			rte_pause();
+			continue;
+		}
+
+		for (i = 0; i != nb_deq; i++) {
+			pkts[i] = events[i].mbuf;
+			if (flags & L3FWD_EVENT_TX_ENQ) {
+				events[i].queue_id = tx_q_id;
+				events[i].op = RTE_EVENT_OP_FORWARD;
+			}
+			rte_event_eth_tx_adapter_txq_set(pkts[i], 0);
+		}
+
+		acl_process_pkts(pkts, hops, nb_deq, socketid);
+
+		for (i = 0; i != nb_deq; i++) {
+			process_packet(pkts[i], &hops[i]);
+			pkts[i]->port = (hops[i] != BAD_PORT) ?
+				       hops[i] : pkts[i]->port;
+		}
+
+		if (flags & L3FWD_EVENT_TX_ENQ) {
+			nb_enq = rte_event_enqueue_burst(event_d_id, event_p_id,
+					events, nb_deq);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_enqueue_burst(event_d_id,
+						event_p_id, events + nb_enq,
+						nb_deq - nb_enq);
+		}
+
+		if (flags & L3FWD_EVENT_TX_DIRECT) {
+			nb_enq = rte_event_eth_tx_adapter_enqueue(event_d_id,
+					event_p_id, events, nb_deq, 0);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_eth_tx_adapter_enqueue(
+						event_d_id, event_p_id,
+						events + nb_enq,
+						nb_deq - nb_enq, 0);
+		}
+	}
+
+	l3fwd_event_worker_cleanup(event_d_id, event_p_id, events, nb_enq,
+				   nb_deq, 0);
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d_burst(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q_burst(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc =
+			l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
+
+static __rte_always_inline void
+acl_process_event_vector(struct rte_event_vector *vec, uint16_t *hops,
+	int32_t socketid)
+{
+	uint32_t i, k;
+
+	for (i = 0; i != vec->nb_elem; i += k) {
+		k = RTE_MIN(vec->nb_elem - i, (uint32_t)MAX_PKT_BURST);
+		acl_process_pkts(vec->mbufs + i, hops + i, k, socketid);
+	}
+
+#if defined ACL_SEND_MULTI
+	k = RTE_ALIGN_FLOOR(vec->nb_elem, FWDSTEP);
+
+	for (i = 0; i != k; i += FWDSTEP)
+		processx4_step3(&vec->mbufs[i], &hops[i]);
+	for (; i < vec->nb_elem; i++)
+		process_packet(vec->mbufs[i], &hops[i]);
+#else
+	for (i = 0; i < vec->nb_elem; i++)
+		process_packet(vec->mbufs[i], &hops[i]);
+#endif
+
+	process_event_vector(vec, hops);
+}
+
+static __rte_always_inline void
+acl_event_loop_vector(struct l3fwd_event_resources *evt_rsrc,
+		      const uint8_t flags)
+{
+	uint16_t *hops;
+	int32_t socketid;
+	uint32_t i, lcore_id, nb_deq, nb_enq;
+	struct rte_event events[MAX_PKT_BURST];
+
+	const int event_p_id = l3fwd_get_free_event_port(evt_rsrc);
+	const uint8_t tx_q_id =
+		evt_rsrc->evq.event_q_id[evt_rsrc->evq.nb_queues - 1];
+	const uint8_t event_d_id = evt_rsrc->event_d_id;
+	const uint16_t deq_len = evt_rsrc->deq_depth;
+
+	if (event_p_id < 0)
+		return;
+
+	lcore_id = rte_lcore_id();
+	socketid = rte_lcore_to_socket_id(lcore_id);
+
+	hops = rte_zmalloc_socket(NULL, sizeof(hops[0]) * evt_rsrc->vector_size,
+			RTE_CACHE_LINE_SIZE, socketid);
+	if (hops == NULL) {
+		RTE_LOG(ERR, L3FWD,
+			"%s: failed to alloc internal buffers on lcore %u\n",
+			__func__, lcore_id);
+		return;
+	}
+
+	RTE_LOG(INFO, L3FWD, "entering %s on lcore %u\n", __func__, lcore_id);
+
+	nb_deq = 0;
+	nb_enq = 0;
+
+	while (!force_quit) {
+		/* Read events from RX queues. */
+		nb_deq = rte_event_dequeue_burst(event_d_id, event_p_id, events,
+						 deq_len, 0);
+		if (nb_deq == 0) {
+			rte_pause();
+			continue;
+		}
+
+		for (i = 0; i < nb_deq; i++) {
+			if (flags & L3FWD_EVENT_TX_ENQ) {
+				events[i].queue_id = tx_q_id;
+				events[i].op = RTE_EVENT_OP_FORWARD;
+			}
+
+			acl_process_event_vector(events[i].vec, hops, socketid);
+		}
+
+		if (flags & L3FWD_EVENT_TX_ENQ) {
+			nb_enq = rte_event_enqueue_burst(event_d_id, event_p_id,
+							 events, nb_deq);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_enqueue_burst(
+					event_d_id, event_p_id, events + nb_enq,
+					nb_deq - nb_enq);
+		}
+
+		if (flags & L3FWD_EVENT_TX_DIRECT) {
+			nb_enq = rte_event_eth_tx_adapter_enqueue(
+				event_d_id, event_p_id, events, nb_deq, 0);
+			while (nb_enq < nb_deq && !force_quit)
+				nb_enq += rte_event_eth_tx_adapter_enqueue(
+					event_d_id, event_p_id, events + nb_enq,
+					nb_deq - nb_enq, 0);
+		}
+	}
+
+	l3fwd_event_worker_cleanup(event_d_id, event_p_id, events, nb_enq,
+				   nb_deq, 1);
+	rte_free(hops);
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_d_burst_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_DIRECT);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
+
+int __rte_noinline
+acl_event_main_loop_tx_q_burst_vector(__rte_unused void *dummy)
+{
+	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
+
+	acl_event_loop_vector(evt_rsrc, L3FWD_EVENT_TX_ENQ);
+	return 0;
+}
diff --git a/examples/l3fwd/l3fwd_event.c b/examples/l3fwd/l3fwd_event.c
index 20be22c6db..18ddd4b8b8 100644
--- a/examples/l3fwd/l3fwd_event.c
+++ b/examples/l3fwd/l3fwd_event.c
@@ -250,6 +250,16 @@ l3fwd_event_resource_setup(struct rte_eth_conf *port_conf)
 		[1][1][0] = fib_event_main_loop_tx_q_vector,
 		[1][1][1] = fib_event_main_loop_tx_q_burst_vector,
 	};
+	const event_loop_cb acl_event_loop[2][2][2] = {
+		[0][0][0] = acl_event_main_loop_tx_d,
+		[0][0][1] = acl_event_main_loop_tx_d_burst,
+		[0][1][0] = acl_event_main_loop_tx_q,
+		[0][1][1] = acl_event_main_loop_tx_q_burst,
+		[1][0][0] = acl_event_main_loop_tx_d_vector,
+		[1][0][1] = acl_event_main_loop_tx_d_burst_vector,
+		[1][1][0] = acl_event_main_loop_tx_q_vector,
+		[1][1][1] = acl_event_main_loop_tx_q_burst_vector,
+	};
 	uint32_t event_queue_cfg;
 	int ret;
 
@@ -293,6 +303,10 @@ l3fwd_event_resource_setup(struct rte_eth_conf *port_conf)
 	evt_rsrc->ops.fib_event_loop =
 		fib_event_loop[evt_rsrc->vector_enabled][evt_rsrc->tx_mode_q]
 			      [evt_rsrc->has_burst];
+
+	evt_rsrc->ops.acl_event_loop =
+		acl_event_loop[evt_rsrc->vector_enabled][evt_rsrc->tx_mode_q]
+			      [evt_rsrc->has_burst];
 }
 
 static void
diff --git a/examples/l3fwd/l3fwd_event.h b/examples/l3fwd/l3fwd_event.h
index c6a4a89127..21aa24c396 100644
--- a/examples/l3fwd/l3fwd_event.h
+++ b/examples/l3fwd/l3fwd_event.h
@@ -58,6 +58,7 @@ struct l3fwd_event_setup_ops {
 	event_loop_cb lpm_event_loop;
 	event_loop_cb em_event_loop;
 	event_loop_cb fib_event_loop;
+	event_loop_cb acl_event_loop;
 };
 
 struct l3fwd_event_resources {
diff --git a/examples/l3fwd/l3fwd_sse.h b/examples/l3fwd/l3fwd_sse.h
index 083729cdef..29c5c7c014 100644
--- a/examples/l3fwd/l3fwd_sse.h
+++ b/examples/l3fwd/l3fwd_sse.h
@@ -86,12 +86,35 @@ process_packet(struct rte_mbuf *pkt, uint16_t *dst_port)
 	_mm_storeu_si128((__m128i *)eth_hdr, te);
 }
 
+static inline void
+process_step3_burst(struct rte_mbuf *pkt[], uint16_t dst_port[], uint32_t num)
+{
+	uint32_t i, k;
+
+	k = RTE_ALIGN_FLOOR(num, FWDSTEP);
+
+	for (i = 0; i != k; i += FWDSTEP)
+		processx4_step3(pkt + i, dst_port + i);
+
+	/* Process up to last 3 packets one by one. */
+	switch (num % FWDSTEP) {
+	case 3:
+		process_packet(pkt[i + 2], dst_port + i + 2);
+		/* fall-through */
+	case 2:
+		process_packet(pkt[i + 1], dst_port + i + 1);
+		/* fall-through */
+	case 1:
+		process_packet(pkt[i], dst_port + i);
+	}
+}
+
 /**
  * Send packets burst from pkts_burst to the ports in dst_port array
  */
 static __rte_always_inline void
-send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
-		uint16_t dst_port[MAX_PKT_BURST], int nb_rx)
+__send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
+		uint16_t dst_port[MAX_PKT_BURST], int nb_rx, int step3)
 {
 	int32_t k;
 	int j = 0;
@@ -110,13 +133,15 @@ send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
 		lp = pnum;
 		lp[0] = 1;
 
-		processx4_step3(pkts_burst, dst_port);
+		if (step3 != 0)
+			processx4_step3(pkts_burst, dst_port);
 
 		/* dp1: <d[0], d[1], d[2], d[3], ... > */
 		dp1 = _mm_loadu_si128((__m128i *)dst_port);
 
 		for (j = FWDSTEP; j != k; j += FWDSTEP) {
-			processx4_step3(&pkts_burst[j], &dst_port[j]);
+			if (step3 != 0)
+				processx4_step3(&pkts_burst[j], &dst_port[j]);
 
 			/*
 			 * dp2:
@@ -155,17 +180,20 @@ send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
 	/* Process up to last 3 packets one by one. */
 	switch (nb_rx % FWDSTEP) {
 	case 3:
-		process_packet(pkts_burst[j], dst_port + j);
+		if (step3 != 0)
+			process_packet(pkts_burst[j], dst_port + j);
 		GROUP_PORT_STEP(dlp, dst_port, lp, pnum, j);
 		j++;
 		/* fall-through */
 	case 2:
-		process_packet(pkts_burst[j], dst_port + j);
+		if (step3 != 0)
+			process_packet(pkts_burst[j], dst_port + j);
 		GROUP_PORT_STEP(dlp, dst_port, lp, pnum, j);
 		j++;
 		/* fall-through */
 	case 1:
-		process_packet(pkts_burst[j], dst_port + j);
+		if (step3 != 0)
+			process_packet(pkts_burst[j], dst_port + j);
 		GROUP_PORT_STEP(dlp, dst_port, lp, pnum, j);
 		j++;
 	}
@@ -194,6 +222,13 @@ send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
 	}
 }
 
+static __rte_always_inline void
+send_packets_multi(struct lcore_conf *qconf, struct rte_mbuf **pkts_burst,
+		uint16_t dst_port[MAX_PKT_BURST], int nb_rx)
+{
+	__send_packets_multi(qconf, pkts_burst, dst_port, nb_rx, 1);
+}
+
 static __rte_always_inline uint16_t
 process_dst_port(uint16_t *dst_ports, uint16_t nb_elem)
 {
diff --git a/examples/l3fwd/l3fwd_wqp.c b/examples/l3fwd/l3fwd_wqp.c
new file mode 100644
index 0000000000..74f745d1af
--- /dev/null
+++ b/examples/l3fwd/l3fwd_wqp.c
@@ -0,0 +1,274 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "l3fwd.h"
+#include "l3fwd_wqp.h"
+
+static int
+wqp_ring_init(struct rte_ring **r, uint32_t num, size_t sz, int32_t sid,
+	uint32_t flags)
+{
+	char name[RTE_RING_NAMESIZE];
+
+	*r = rte_zmalloc_socket(NULL, sz, RTE_CACHE_LINE_SIZE, sid);
+	if (*r == NULL)
+		return -ENOMEM;
+
+	snprintf(name, sizeof(name), "%p", *r);
+	return rte_ring_init(*r, name, num, flags);
+}
+
+static int
+wqp_soring_init(struct rte_soring **r, struct rte_soring_param *prm, size_t sz,
+	int32_t sid)
+{
+	char name[RTE_RING_NAMESIZE];
+
+	*r = rte_zmalloc_socket(NULL, sz, RTE_CACHE_LINE_SIZE, sid);
+	if (*r == NULL)
+		return -ENOMEM;
+
+	snprintf(name, sizeof(name), "%p", *r);
+	prm->name = name;
+	return rte_soring_init(*r, prm);
+}
+
+static void
+wqp_fini(struct lcore_conf *lc)
+{
+	uint32_t i, j;
+
+	if (lc->n_rx_queue == 0)
+		return;
+
+	for (i = 0; i != lc->wqpool.nb_queue; i++) {
+		for (j = 0; j != RTE_DIM(lc->wqpool.queue[i].r); j++)
+			rte_free(lc->wqpool.queue[i].r[j]);
+	}
+
+	memset(&lc->wqpool, 0, sizeof(lc->wqpool));
+}
+
+static int
+l3fwd_wqp_unque_init(struct lcore_conf *lc, const struct l3fwd_wqp_param *prm,
+	uint32_t lcid)
+{
+	int32_t rc, sid;
+	uint32_t i, n, nq;
+	size_t szi, szo;
+
+	sid = rte_lcore_to_socket_id(lcid);
+
+	n = rte_align32pow2(prm->qsize);
+
+	szi = rte_ring_get_memsize(n);
+	szo = rte_ring_get_memsize_elem(sizeof(struct wqelm), n);
+
+	nq = (prm->single == 0) ? lc->n_rx_queue : 1;
+	lc->wqpool.nb_queue = nq;
+	lc->wqpool.qmask = (prm->single != 0) ? 0 : UINT32_MAX;
+
+	rc = 0;
+	for (i = 0; i != nq; i++) {
+
+		rc = wqp_ring_init(&lc->wqpool.queue[i].r[LCORE_WQ_IN], n, szi,
+			sid, RING_F_SP_ENQ);
+		if (rc != 0)
+			break;
+
+		rc = wqp_ring_init(&lc->wqpool.queue[i].r[LCORE_WQ_OUT], n, szo,
+			sid, RING_F_SC_DEQ);
+		if (rc != 0)
+			break;
+	}
+
+	if (i != nq) {
+		printf("error: %s failed at %u-th queue, error code: %d\n",
+			__func__, i, rc);
+		wqp_fini(lc);
+	}
+
+	lc->wqpool.prm = *prm;
+	return rc;
+}
+
+static int
+l3fwd_wqp_orque_init(struct lcore_conf *lc, const struct l3fwd_wqp_param *qprm,
+	uint32_t lcid)
+{
+	int32_t rc, sid;
+	uint32_t i, n, nq;
+	ssize_t sz;
+	struct rte_soring_param prm;
+
+	sid = rte_lcore_to_socket_id(lcid);
+
+	memset(&prm, 0, sizeof(prm));
+
+	n = 2 * qprm->qsize;
+	prm.elems = n;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.meta_size = sizeof(uint32_t);
+	prm.stages = 1;
+	prm.prod_synt = RTE_RING_SYNC_ST;
+	prm.cons_synt = RTE_RING_SYNC_ST;
+
+	sz = rte_soring_get_memsize(&prm);
+	if (sz < 0)
+		return sz;
+
+	nq = (qprm->single == 0) ? lc->n_rx_queue : 1;
+	lc->wqpool.nb_queue = nq;
+	lc->wqpool.qmask = (qprm->single != 0) ? 0 : UINT32_MAX;
+
+	rc = 0;
+	for (i = 0; i != nq; i++) {
+
+		rc = wqp_soring_init(&lc->wqpool.queue[i].sor, &prm, sz, sid);
+		if (rc != 0)
+			break;
+	}
+
+	if (i != nq) {
+		printf("error: %s failed at %u-th queue, error code: %d\n",
+			__func__, i, rc);
+		wqp_fini(lc);
+	}
+
+	lc->wqpool.prm = *qprm;
+	return rc;
+}
+
+static int
+wqp_init(struct lcore_conf *lc, const struct l3fwd_wqp_param *prm,
+	uint32_t lcid)
+{
+	/* this is I/O poll lcore */
+	if (lc->n_rx_queue != 0) {
+		if (prm->mode == L3FWD_WORKER_UNQUE)
+			return l3fwd_wqp_unque_init(lc, prm, lcid);
+		else if (prm->mode == L3FWD_WORKER_ORQUE)
+			return l3fwd_wqp_orque_init(lc, prm, lcid);
+		else
+			return -ENOTSUP;
+	}
+
+	return -ENOTSUP;
+}
+
+void
+l3fwd_wqp_fini(struct lcore_conf lc[RTE_MAX_LCORE])
+{
+	uint32_t lcid;
+
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++)
+		wqp_fini(lc + lcid);
+}
+
+static int
+check_set_wqp_param(struct l3fwd_wqp_param *prm)
+{
+	uint32_t n;
+
+	if (prm->qsize == 0) {
+		n = RTE_MAX(nb_rxd, nb_txd);
+		n = n + n / 2;
+		prm->qsize = n;
+	}
+
+	return 0;
+}
+
+static void print_wqp_param(const struct l3fwd_wqp_param *prm)
+{
+	printf("%s(%p): mode=%d, qsize=%u, single=%d\n",
+		__func__, prm, prm->mode, prm->qsize, prm->single);
+}
+
+int
+l3fwd_wqp_init(struct lcore_conf lc[RTE_MAX_LCORE],
+	const struct l3fwd_wqp_param *qprm)
+{
+	int32_t rc;
+	uint32_t i, j, k, lcid, m, n, nrxq, nwqt;
+	union lcore_wq *wqp;
+	struct l3fwd_wqp_param prm;
+
+	if (qprm->mode == L3FWD_WORKER_POLL)
+		return 0;
+
+	prm = *qprm;
+	rc = check_set_wqp_param(&prm);
+	print_wqp_param(&prm);
+	if (rc < 0) {
+		printf("error: %s invalid paramer values\n", __func__);
+		return rc;
+	}
+
+	nrxq = 0;
+	nwqt = 0;
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		if (rte_lcore_is_enabled(lcid) == 0)
+			continue;
+		if (lc[lcid].n_rx_queue != 0)
+			nrxq += (prm.single != 0) ? 1 : lc[lcid].n_rx_queue;
+		nwqt += (lc[lcid].n_rx_queue == 0);
+	}
+
+	printf("%s: total worker queues: %u, total WQ threads: %u\n",
+			__func__, nrxq, nwqt);
+	if (nrxq == 0)
+		return 0;
+
+	if (nrxq > nwqt * MAX_RX_QUEUE_PER_LCORE) {
+		printf("error: %s not enough WQ threads to handle all RXQs\n",
+			__func__);
+		return -EINVAL;
+	}
+
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		if (rte_lcore_is_enabled(lcid) == 0 || lc[lcid].n_rx_queue == 0)
+			continue;
+		rc = wqp_init(lc + lcid, &prm, lcid);
+		if (rc != 0)
+			break;
+	}
+	if (rc != 0)
+		return rc;
+
+	/* create a temp pool of all RX queues */
+	wqp = malloc(sizeof(wqp[0]) * nrxq);
+	if (wqp == NULL) {
+		l3fwd_wqp_fini(lc);
+		return -ENOMEM;
+	}
+
+	n = 0;
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		memcpy(wqp + n, lc[lcid].wqpool.queue,
+			lc[lcid].wqpool.nb_queue * sizeof(wqp[0]));
+		n += lc[lcid].wqpool.nb_queue;
+	}
+
+	/* distribute them across all worker threads */
+	k = 0;
+	m = RTE_MIN(RTE_DIM(lc[lcid].wqpool.queue), n);
+	for (lcid = 0; lcid != RTE_MAX_LCORE; lcid++) {
+		if (rte_lcore_is_enabled(lcid) == 0 ||
+				lc[lcid].wqpool.nb_queue != 0)
+			continue;
+		j = k;
+		for (i = 0; i != m; i++) {
+			lc[lcid].wqpool.queue[i] = wqp[j];
+			j = (j + 1) % n;
+		}
+		lc[lcid].wqpool.nb_queue = i;
+		lc[lcid].wqpool.qmask = UINT32_MAX;
+		lc[lcid].wqpool.prm = prm;
+		k = j;
+	}
+
+	free(wqp);
+	return rc;
+}
diff --git a/examples/l3fwd/l3fwd_wqp.h b/examples/l3fwd/l3fwd_wqp.h
new file mode 100644
index 0000000000..bd6ce88657
--- /dev/null
+++ b/examples/l3fwd/l3fwd_wqp.h
@@ -0,0 +1,130 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef L3FWD_WQP_H
+#define L3FWD_WQP_H
+
+#include <rte_soring.h>
+
+struct wqelm {
+	struct rte_mbuf *mb;
+	uint32_t rc;
+}  __rte_packed;
+
+
+void l3fwd_wqp_fini(struct lcore_conf lc[RTE_MAX_LCORE]);
+int l3fwd_wqp_init(struct lcore_conf lc[RTE_MAX_LCORE],
+	const struct l3fwd_wqp_param *prm);
+
+static inline uint32_t
+lcore_wq_submit(const struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf * const pkts[MAX_PKT_BURST], uint32_t num)
+{
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_IN];
+		return rte_ring_enqueue_burst(r, (void * const *)pkts, num,
+				NULL);
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+		return rte_soring_enqueue_burst(sor, pkts, num, NULL);
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+static inline uint32_t
+lcore_wq_receive(const struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf *pkts[MAX_PKT_BURST], uint16_t hops[MAX_PKT_BURST],
+	uint32_t num)
+{
+	uint32_t i, n;
+	uint32_t rcs[MAX_PKT_BURST];
+	struct wqelm elm[MAX_PKT_BURST];
+
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_OUT];
+
+		n = rte_ring_dequeue_burst_elem(r, elm, sizeof(elm[0]), num,
+				NULL);
+		for (i = 0; i != n; i++) {
+			pkts[i] = elm[i].mb;
+			hops[i] = elm[i].rc;
+		}
+
+		return n;
+
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+
+		n = rte_soring_dequeux_burst(sor, pkts, rcs, num, NULL);
+		for (i = 0; i != n; i++)
+			hops[i] = rcs[i];
+
+		return n;
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+static inline uint32_t
+lcore_wq_pull(struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf *pkts[MAX_PKT_BURST], uint32_t num)
+{
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_IN];
+		return rte_ring_dequeue_burst(r, (void **)pkts, num, NULL);
+
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+		return rte_soring_acquire_burst(sor, pkts, 0, num,
+			&wqp->queue[idx].ftoken, NULL);
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+static inline uint32_t
+lcore_wq_push(const struct lcore_wq_pool *wqp, uint32_t idx,
+	struct rte_mbuf * const pkts[MAX_PKT_BURST],
+	const uint16_t hops[MAX_PKT_BURST], uint32_t num)
+{
+	uint32_t i;
+	uint32_t rcs[MAX_PKT_BURST];
+	struct wqelm elm[MAX_PKT_BURST];
+
+	idx &= wqp->qmask;
+
+	if (wqp->prm.mode == L3FWD_WORKER_UNQUE) {
+		struct rte_ring *r = wqp->queue[idx].r[LCORE_WQ_OUT];
+
+		for (i = 0; i != num; i++) {
+			elm[i].mb = pkts[i];
+			elm[i].rc = hops[i];
+		}
+		return rte_ring_enqueue_burst_elem(r, elm, sizeof(elm[0]), num,
+				NULL);
+	} else if (wqp->prm.mode == L3FWD_WORKER_ORQUE) {
+		struct rte_soring *sor = wqp->queue[idx].sor;
+
+		for (i = 0; i != num; i++)
+			rcs[i] = hops[i];
+		rte_soring_releasx(sor, NULL, rcs, 0, num,
+			wqp->queue[idx].ftoken);
+		return num;
+	}
+
+	rte_errno = ENOTSUP;
+	return 0;
+}
+
+#endif /* L3FWD_WQP_H */
diff --git a/examples/l3fwd/main.c b/examples/l3fwd/main.c
index 01b763e5ba..5d2bc5c611 100644
--- a/examples/l3fwd/main.c
+++ b/examples/l3fwd/main.c
@@ -47,6 +47,7 @@
 #include "l3fwd.h"
 #include "l3fwd_event.h"
 #include "l3fwd_route.h"
+#include "l3fwd_wqp.h"
 
 #define MAX_TX_QUEUE_PER_PORT RTE_MAX_LCORE
 #define MAX_RX_QUEUE_PER_PORT 128
@@ -69,6 +70,10 @@ enum L3FWD_LOOKUP_MODE {
 };
 static enum L3FWD_LOOKUP_MODE lookup_mode;
 
+struct l3fwd_wqp_param l3fwd_wqp_param = {
+	.mode = L3FWD_WORKER_POLL,
+};
+
 /* Global variables. */
 static int numa_on = 1; /**< NUMA is enabled by default. */
 static int parse_ptype; /**< Parse packet type using rx callback, and */
@@ -246,6 +251,8 @@ const struct ipv6_l3fwd_route ipv6_l3fwd_route_array[] = {
 	{{32, 1, 2, 0, 0, 0, 0, 15, 0, 0, 0, 0, 0, 0, 0, 0}, 64, 15},
 };
 
+uint32_t l3fwd_lookup_iter_num = 1;
+
 /*
  * API's called during initialization to setup ACL/EM/LPM rules.
  */
@@ -453,6 +460,23 @@ print_usage(const char *prgname)
 		ACL_LEAD_CHAR, ROUTE_LEAD_CHAR, alg);
 }
 
+static int
+parse_uint_val(const char *str, uint32_t *val, uint32_t min, uint32_t max)
+{
+	char *end = NULL;
+	unsigned long v;
+
+	errno = 0;
+	v = strtoul(str, &end, 0);
+	if (errno != 0 || end == NULL || *end != '\0')
+		return -1;
+	if (v < min || v > max)
+		return -1;
+
+	*val = v;
+	return 0;
+}
+
 static int
 parse_max_pkt_len(const char *pktlen)
 {
@@ -572,16 +596,35 @@ parse_eth_dest(const char *optarg)
 }
 
 static void
-parse_mode(const char *optarg __rte_unused)
+parse_mode(const char *optarg)
 {
+	l3fwd_wqp_param.mode = L3FWD_WORKER_POLL;
+
 #ifdef RTE_LIB_EVENTDEV
 	struct l3fwd_event_resources *evt_rsrc = l3fwd_get_eventdev_rsrc();
 
+	evt_rsrc->enabled = false;
+
 	if (!strcmp(optarg, "poll"))
 		evt_rsrc->enabled = false;
 	else if (!strcmp(optarg, "eventdev"))
 		evt_rsrc->enabled = true;
+	else
 #endif
+	if (strcmp(optarg, "wqorder") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_ORQUE;
+		l3fwd_wqp_param.single = 0;
+	} else if (strcmp(optarg, "wqunorder") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_UNQUE;
+		l3fwd_wqp_param.single = 0;
+	} else if (strcmp(optarg, "wqorderS") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_ORQUE;
+		l3fwd_wqp_param.single = 1;
+	} else if (strcmp(optarg, "wqunorderS") == 0) {
+		l3fwd_wqp_param.mode = L3FWD_WORKER_UNQUE;
+		l3fwd_wqp_param.single = 1;
+	} else
+		rte_exit(EXIT_FAILURE, "unknown mode: %s\n", optarg);
 }
 
 static void
@@ -698,6 +741,8 @@ static const char short_options[] =
 #define CMD_LINE_OPT_RULE_IPV4 "rule_ipv4"
 #define CMD_LINE_OPT_RULE_IPV6 "rule_ipv6"
 #define CMD_LINE_OPT_ALG "alg"
+#define CMD_LINE_OPT_WQSIZE "wqsize"
+#define CMD_LINE_OPT_LOOKUP_ITER "lookup-iter"
 
 enum {
 	/* long options mapped to a short option */
@@ -726,7 +771,9 @@ enum {
 	CMD_LINE_OPT_LOOKUP_NUM,
 	CMD_LINE_OPT_ENABLE_VECTOR_NUM,
 	CMD_LINE_OPT_VECTOR_SIZE_NUM,
-	CMD_LINE_OPT_VECTOR_TMO_NS_NUM
+	CMD_LINE_OPT_VECTOR_TMO_NS_NUM,
+	CMD_LINE_OPT_WQSIZE_NUM,
+	CMD_LINE_OPT_LOOKUP_ITER_NUM,
 };
 
 static const struct option lgopts[] = {
@@ -753,6 +800,8 @@ static const struct option lgopts[] = {
 	{CMD_LINE_OPT_RULE_IPV4,   1, 0, CMD_LINE_OPT_RULE_IPV4_NUM},
 	{CMD_LINE_OPT_RULE_IPV6,   1, 0, CMD_LINE_OPT_RULE_IPV6_NUM},
 	{CMD_LINE_OPT_ALG,   1, 0, CMD_LINE_OPT_ALG_NUM},
+	{CMD_LINE_OPT_WQSIZE, 1, 0, CMD_LINE_OPT_WQSIZE_NUM},
+	{CMD_LINE_OPT_LOOKUP_ITER, 1, 0, CMD_LINE_OPT_LOOKUP_ITER_NUM},
 	{NULL, 0, 0, 0}
 };
 
@@ -934,6 +983,18 @@ parse_args(int argc, char **argv)
 		case CMD_LINE_OPT_ALG_NUM:
 			l3fwd_set_alg(optarg);
 			break;
+		case CMD_LINE_OPT_WQSIZE_NUM:
+			ret = parse_uint_val(optarg, &l3fwd_wqp_param.qsize,
+				RX_DESC_DEFAULT, UINT16_MAX);
+			if (ret < 0)
+				return ret;
+			break;
+		case CMD_LINE_OPT_LOOKUP_ITER_NUM:
+			ret = parse_uint_val(optarg, &l3fwd_lookup_iter_num,
+				1, UINT16_MAX);
+			if (ret < 0)
+				return ret;
+			break;
 		default:
 			print_usage(prgname);
 			return -1;
@@ -1588,6 +1649,8 @@ main(int argc, char **argv)
 			l3fwd_lkp.main_loop = evt_rsrc->ops.em_event_loop;
 		else if (lookup_mode == L3FWD_LOOKUP_FIB)
 			l3fwd_lkp.main_loop = evt_rsrc->ops.fib_event_loop;
+		else if (lookup_mode == L3FWD_LOOKUP_ACL)
+			l3fwd_lkp.main_loop = evt_rsrc->ops.acl_event_loop;
 		else
 			l3fwd_lkp.main_loop = evt_rsrc->ops.lpm_event_loop;
 	} else
@@ -1640,6 +1703,12 @@ main(int argc, char **argv)
 		}
 	}
 
+	/* init worker queues for lcores (if any) */
+	ret = l3fwd_wqp_init(lcore_conf, &l3fwd_wqp_param);
+	if (ret != 0)
+		rte_exit(EXIT_FAILURE, "l3fwd_wqp_init: err=%d, lcore=%u\n",
+			ret, lcore_id);
+
 	check_all_ports_link_status(enabled_port_mask);
 
 	ret = 0;
@@ -1695,6 +1764,8 @@ main(int argc, char **argv)
 	/* clean up config file routes */
 	l3fwd_lkp.free_routes();
 
+	l3fwd_wqp_fini(lcore_conf);
+
 	/* clean up the EAL */
 	rte_eal_cleanup();
 
diff --git a/examples/l3fwd/meson.build b/examples/l3fwd/meson.build
index c25de77bba..a024492fb1 100644
--- a/examples/l3fwd/meson.build
+++ b/examples/l3fwd/meson.build
@@ -16,6 +16,7 @@ sources = files(
         'l3fwd_event_generic.c',
         'l3fwd_fib.c',
         'l3fwd_lpm.c',
+        'l3fwd_wqp.c',
         'main.c',
 )
 if dpdk_conf.has('RTE_LIB_EVENTDEV')
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library
  2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                     ` (5 preceding siblings ...)
  2024-09-06 13:13   ` [RFCv2 6/6] examples/l3fwd: make ACL work in pipeline and eventdev modes Konstantin Ananyev
@ 2024-09-16 12:37   ` Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
                       ` (5 more replies)
  6 siblings, 6 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-16 12:37 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big, 
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
==============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (4):
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API

 .mailmap                           |   1 +
 app/test/meson.build               |   3 +
 app/test/test_ring_stress_impl.h   |   1 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 827 +++++++++++++++++++++++++++++
 lib/ring/meson.build               |   4 +-
 lib/ring/rte_ring.c                |  87 ++-
 lib/ring/rte_ring.h                |  15 +
 lib/ring/rte_ring_c11_pvt.h        | 134 +----
 lib/ring/rte_ring_elem_pvt.h       | 181 +++++--
 lib/ring/rte_ring_generic_pvt.h    | 121 +----
 lib/ring/rte_ring_hts_elem_pvt.h   |  85 +--
 lib/ring/rte_ring_rts_elem_pvt.h   |  85 +--
 lib/ring/rte_soring.c              | 182 +++++++
 lib/ring/rte_soring.h              | 547 +++++++++++++++++++
 lib/ring/soring.c                  | 548 +++++++++++++++++++
 lib/ring/soring.h                  | 124 +++++
 lib/ring/version.map               |  26 +
 21 files changed, 3140 insertions(+), 396 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v3 1/5] ring: common functions for 'move head' ops
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-09-16 12:37     ` Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 2/5] ring: make copying functions generic Konstantin Ananyev
                       ` (4 subsequent siblings)
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-16 12:37 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v3 2/5] ring: make copying functions generic
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-09-16 12:37     ` Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 3/5] ring: make dump function more verbose Konstantin Ananyev
                       ` (3 subsequent siblings)
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-16 12:37 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v3 3/5] ring: make dump function more verbose
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 2/5] ring: make copying functions generic Konstantin Ananyev
@ 2024-09-16 12:37     ` Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                       ` (2 subsequent siblings)
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-16 12:37 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Eimear Morrissey

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 4a508bafad..3da86393c0 100644
--- a/.mailmap
+++ b/.mailmap
@@ -379,6 +379,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..8449cb4b15 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index c709f30497..d6f9e85c64 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v3 4/5] ring/soring: introduce Staged Ordered Ring
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                       ` (2 preceding siblings ...)
  2024-09-16 12:37     ` [PATCH v3 3/5] ring: make dump function more verbose Konstantin Ananyev
@ 2024-09-16 12:37     ` Konstantin Ananyev
  2024-09-16 12:37     ` [PATCH v3 5/5] app/test: add unit tests for soring API Konstantin Ananyev
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-16 12:37 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev, Eimear Morrissey

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/meson.build  |   4 +-
 lib/ring/rte_soring.c | 182 ++++++++++++++
 lib/ring/rte_soring.h | 547 +++++++++++++++++++++++++++++++++++++++++
 lib/ring/soring.c     | 548 ++++++++++++++++++++++++++++++++++++++++++
 lib/ring/soring.h     | 124 ++++++++++
 lib/ring/version.map  |  19 ++
 6 files changed, 1422 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..b6bc71b8c9
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,182 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..81e25db86b
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,547 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/* max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the ring */
+	const char *name;
+	/** number of elemnts in the ring */
+	uint32_t elems;
+	/** size of elements in the ring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the ring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a ring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual ring elements and theri rec-codes. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use rte_ring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or @n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or @n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ * @return
+ *   - None.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ * @return
+ *   - None.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..97fa38df66
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,548 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (RTE_SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'RTE_SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in RTE_SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		&ot.raw, nt.raw, rte_memory_order_acquire,
+		rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & RTE_SORING_ST_MASK) != RTE_SORING_ST_FINISH ||
+				st.ftoken != tail)
+			break;
+
+		k = st.stnum & ~RTE_SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_acquire);
+
+	do {
+		n = num;
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_relaxed);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	union soring_state st;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, r->state,
+				r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+
+		/* update status ring */
+		st.ftoken = head;
+		st.stnum = (RTE_SORING_ST_START | n);
+
+		rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+				rte_memory_order_relaxed);
+		*ftoken = head;
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+
+	idx = ftoken & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_acquire);
+
+	/* check state ring contents */
+	RTE_VERIFY(st.stnum == (RTE_SORING_ST_START | n) &&
+		st.ftoken == ftoken);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	st.stnum = RTE_SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_release);
+
+	if (tail == ftoken)
+		__rte_soring_stage_finalize(&stg->sht, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	return soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	return soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..c44d8912d9
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,124 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_START	RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define RTE_SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define RTE_SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v3 5/5] app/test: add unit tests for soring API
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                       ` (3 preceding siblings ...)
  2024-09-16 12:37     ` [PATCH v3 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-09-16 12:37     ` Konstantin Ananyev
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-16 12:37 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Konstantin Ananyev, Eimear Morrissey

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 827 +++++++++++++++++++++++++++++
 6 files changed, 1395 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index e29258e6ec..c290162e43 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -175,6 +175,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..1ba78a31f5
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,827 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill)
+{
+	uint32_t i;
+
+	static rte_spinlock_t dump_lock;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				__func__, rte_lcore_id(), num, i, elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname)
+{
+	if (bhv == RTE_RING_QUEUE_FIXED && exp != res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	if (bhv == RTE_RING_QUEUE_VARIABLE && exp < res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire));
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk));
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk));
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library
  2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                       ` (4 preceding siblings ...)
  2024-09-16 12:37     ` [PATCH v3 5/5] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-09-17 12:09     ` Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
                         ` (6 more replies)
  5 siblings, 7 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-17 12:09 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

v2 -> v3:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (4):
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API

 .mailmap                               |   1 +
 app/test/meson.build                   |   3 +
 app/test/test_ring_stress_impl.h       |   1 +
 app/test/test_soring.c                 | 442 +++++++++++++
 app/test/test_soring_mt_stress.c       |  40 ++
 app/test/test_soring_stress.c          |  48 ++
 app/test/test_soring_stress.h          |  35 ++
 app/test/test_soring_stress_impl.h     | 827 +++++++++++++++++++++++++
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_ring.c                    |  87 ++-
 lib/ring/rte_ring.h                    |  15 +
 lib/ring/rte_ring_c11_pvt.h            | 134 +---
 lib/ring/rte_ring_elem_pvt.h           | 181 ++++--
 lib/ring/rte_ring_generic_pvt.h        | 121 +---
 lib/ring/rte_ring_hts_elem_pvt.h       |  85 +--
 lib/ring/rte_ring_rts_elem_pvt.h       |  85 +--
 lib/ring/rte_soring.c                  | 182 ++++++
 lib/ring/rte_soring.h                  | 543 ++++++++++++++++
 lib/ring/soring.c                      | 548 ++++++++++++++++
 lib/ring/soring.h                      | 124 ++++
 lib/ring/version.map                   |  26 +
 22 files changed, 3144 insertions(+), 396 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v4 1/5] ring: common functions for 'move head' ops
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-09-17 12:09       ` Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 2/5] ring: make copying functions generic Konstantin Ananyev
                         ` (5 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-17 12:09 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v4 2/5] ring: make copying functions generic
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-09-17 12:09       ` Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 3/5] ring: make dump function more verbose Konstantin Ananyev
                         ` (4 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-17 12:09 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v4 3/5] ring: make dump function more verbose
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 2/5] ring: make copying functions generic Konstantin Ananyev
@ 2024-09-17 12:09       ` Konstantin Ananyev
  2024-09-17 12:09       ` [PATCH v4 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                         ` (3 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-17 12:09 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 4a508bafad..3da86393c0 100644
--- a/.mailmap
+++ b/.mailmap
@@ -379,6 +379,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..8449cb4b15 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index c709f30497..d6f9e85c64 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v4 4/5] ring/soring: introduce Staged Ordered Ring
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                         ` (2 preceding siblings ...)
  2024-09-17 12:09       ` [PATCH v4 3/5] ring: make dump function more verbose Konstantin Ananyev
@ 2024-09-17 12:09       ` Konstantin Ananyev
  2024-09-19 17:03         ` Jerin Jacob
  2024-09-17 12:09       ` [PATCH v4 5/5] app/test: add unit tests for soring API Konstantin Ananyev
                         ` (2 subsequent siblings)
  6 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-17 12:09 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_soring.c                  | 182 ++++++++
 lib/ring/rte_soring.h                  | 543 ++++++++++++++++++++++++
 lib/ring/soring.c                      | 548 +++++++++++++++++++++++++
 lib/ring/soring.h                      | 124 ++++++
 lib/ring/version.map                   |  19 +
 7 files changed, 1426 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index 0ff70d9057..3ceeef7eb6 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -55,6 +55,14 @@ New Features
      Also, make sure to start the actual text at the margin.
      =======================================================
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..b6bc71b8c9
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,182 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..9c35b4a18c
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,543 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/* max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the ring */
+	const char *name;
+	/** number of elemnts in the ring */
+	uint32_t elems;
+	/** size of elements in the ring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the ring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a ring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual ring elements and theri rec-codes. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use rte_ring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the ring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or @n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or @n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the ring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the ring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the ring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the ring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the ring for given stage.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining ring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the ring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite ring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the ring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the ring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..a4439f00fa
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,548 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (RTE_SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'RTE_SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in RTE_SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_acquire, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & RTE_SORING_ST_MASK) != RTE_SORING_ST_FINISH ||
+				st.ftoken != tail)
+			break;
+
+		k = st.stnum & ~RTE_SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_acquire);
+
+	do {
+		n = num;
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_relaxed);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	union soring_state st;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, r->state,
+				r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+
+		/* update status ring */
+		st.ftoken = head;
+		st.stnum = (RTE_SORING_ST_START | n);
+
+		rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+				rte_memory_order_relaxed);
+		*ftoken = head;
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+
+	idx = ftoken & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_acquire);
+
+	/* check state ring contents */
+	RTE_VERIFY(st.stnum == (RTE_SORING_ST_START | n) &&
+		st.ftoken == ftoken);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	st.stnum = RTE_SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_release);
+
+	if (tail == ftoken)
+		__rte_soring_stage_finalize(&stg->sht, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	return soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	return soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..c44d8912d9
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,124 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_START	RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define RTE_SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define RTE_SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v4 5/5] app/test: add unit tests for soring API
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                         ` (3 preceding siblings ...)
  2024-09-17 12:09       ` [PATCH v4 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-09-17 12:09       ` Konstantin Ananyev
  2024-10-12 18:09       ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Stephen Hemminger
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-09-17 12:09 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 827 +++++++++++++++++++++++++++++
 6 files changed, 1395 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index e29258e6ec..c290162e43 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -175,6 +175,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..1ba78a31f5
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,827 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill)
+{
+	uint32_t i;
+
+	static rte_spinlock_t dump_lock;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				__func__, rte_lcore_id(), num, i, elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname)
+{
+	if (bhv == RTE_RING_QUEUE_FIXED && exp != res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	if (bhv == RTE_RING_QUEUE_VARIABLE && exp < res) {
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire));
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk));
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk));
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v4 4/5] ring/soring: introduce Staged Ordered Ring
  2024-09-17 12:09       ` [PATCH v4 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-09-19 17:03         ` Jerin Jacob
  0 siblings, 0 replies; 101+ messages in thread
From: Jerin Jacob @ 2024-09-19 17:03 UTC (permalink / raw)
  To: Konstantin Ananyev
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, eimear.morrissey,
	Konstantin Ananyev

On Tue, Sep 17, 2024 at 5:49 PM Konstantin Ananyev
<konstantin.v.ananyev@yandex.ru> wrote:
>
> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
>
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'.
> It is based on conventional DPDK rte_ring, re-uses many of its concepts,
> and even substantial part of its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in the middle.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages, i.e.:
>   at the same stage multiple threads can process objects from the ring in
>   any order, but for the next stage objects will always appear in the
>   original order.
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> - number of stages, size and number of objects in the ring are
>   configurable at ring initialization time.
>
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation.
>   after some objects are 'acquired' - given thread can safely assume that
>   it has exclusive possession of these objects till 'release' for them is
>   invoked.
>   Note that right now user has to release exactly the same number of
>   objects that was acquired before.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
>
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important. I.E.: IPsec processing, etc.
>
> Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
> Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> ---
>  doc/guides/rel_notes/release_24_11.rst |   8 +
>  lib/ring/meson.build                   |   4 +-
>  lib/ring/rte_soring.c                  | 182 ++++++++
>  lib/ring/rte_soring.h                  | 543 ++++++++++++++++++++++++
>  lib/ring/soring.c                      | 548 +++++++++++++++++++++++++
>  lib/ring/soring.h                      | 124 ++++++
>  lib/ring/version.map                   |  19 +
>  7 files changed, 1426 insertions(+), 2 deletions(-)
>  create mode 100644 lib/ring/rte_soring.c
>  create mode 100644 lib/ring/rte_soring.h
>  create mode 100644 lib/ring/soring.c
>  create mode 100644 lib/ring/soring.h

Good feature and makes sense to not make as eventdev driver.

# I think, it is worth updating doc/guides/prog_guide/ring_lib.rst for
this new type and means to use it.
# Missing rte_soring.h update in doc/api/doxy-api-index.md



> diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
> new file mode 100644
> index 0000000000..9c35b4a18c
> --- /dev/null
> +++ b/lib/ring/rte_soring.h
> @@ -0,0 +1,543 @@
> +/* SPDX-License-Identifier: BSD-3-Clause
> + * Copyright(c) 2024 Huawei Technologies Co., Ltd
> + */
> +
> +#ifndef _RTE_SORING_H_
> +#define _RTE_SORING_H_
> +
> +/**
> + * @file
> + * This file contains definition of RTE soring (Staged Ordered Ring) public API.
> + * Brief description:
> + * enqueue/dequeue works the same as for conventional rte_ring:
> + * any rte_ring sync types can be used, etc.
> + * Plus there could be multiple 'stages'.
> + * For each stage there is an acquire (start) and release (finish) operation.
> + * after some elems are 'acquired' - user  can safely assume that he has
> + * exclusive possession of these elems till 'release' for them is done.
> + * Note that right now user has to release exactly the same number of elems
> + * he acquired before.
> + * After 'release', elems can be 'acquired' by next stage and/or dequeued
> + * (in case of last stage).
> + */
> +
> +#ifdef __cplusplus
> +extern "C" {
> +#endif
> +
> +#include <rte_ring.h>
> +
> +/* upper 2 bits are used for status */

Add Doxygen comment for public symbol

> +#define RTE_SORING_ST_BIT       30
> +
> +/* max possible number of elements in the soring */

Add Doxygen comment for public symbol

> +#define RTE_SORING_ELEM_MAX    (RTE_BIT32(RTE_SORING_ST_BIT) - 1)
> +
> +struct rte_soring_param {
> +       /** expected name of the ring */
> +       const char *name;
> +       /** number of elemnts in the ring */
> +       uint32_t elems;
> +       /** size of elements in the ring, must be a multiple of 4 */
> +       uint32_t elem_size;
> +       /**
> +        * size of metadata for each elem, must be a multiple of 4.
> +        * This parameter defines a size of supplementary and optional
> +        * array of metadata associated with each object in the soring.
> +        * While element size is configurable (see 'elem_size' parameter above),
> +        * so user can specify it big enough to hold both object and its
> +        * metadata together, for performance reasons it might be plausible
> +        * to access them as separate arrays.
> +        * Common usage scenario when such separation helps:
> +        * enqueue() - writes to objects array
> +        * acquire() - reads from objects array
> +        * release() - writes to metadata array (as an example: return code)
> +        * dequeue() - reads both objects and metadata array
> +        */
> +       uint32_t meta_size;
> +       /** number of stages in the ring */
> +       uint32_t stages;
> +       /** sync type for producer */
> +       enum rte_ring_sync_type prod_synt;
> +       /** sync type for consumer */
> +       enum rte_ring_sync_type cons_synt;
> +};
> +
> +struct rte_soring;
> +
> +/**
> + * Calculate the memory size needed for a soring
> + *
> + * This function returns the number of bytes needed for a ring, given
> + * the expected parameters for it. This value is the sum of the size of
> + * the internal metadata and the size of the memory needed by the
> + * actual ring elements and theri rec-codes. The value is aligned to a cache

typo - theri

> + * line size.
> + *
> + * @param prm
> + *   Pointer to the structure that contains soring creation paramers.
> + * @return
> + *   - The memory size needed for the soring on success.
> + *   - -EINVAL if provided paramer values are invalid.
> + */
> +__rte_experimental
> +ssize_t
> +rte_soring_get_memsize(const struct rte_soring_param *prm);
> +
> +/**
> + * Initialize a soring structure.
> + *
> + * Initialize a soring structure in memory pointed by "r".
> + * The size of the memory area must be large enough to store the soring
> + * internal structures plus the objects and ret-code tables.

typo- ret-code

> + * It is strongly advised to use rte_ring_get_memsize() to get the
> + * appropriate size.
> + *
> + * @param r
> + *   Pointer to the soring structure.

I think, you can also mention, memory allocated with size provided by
rte_soring_get_memsize().
Also add @see to cross-reference.

> + * @param prm
> + *   Pointer to the structure that contains soring creation paramers.
> + * @return
> + *   - 0 on success, or a negative error code.
> + */
> +__rte_experimental
> +int
> +rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
> +
> +/**
> + * Return the total number of filled entries in a ring.

Across the patch, a ring can be replaced by a soring ring

> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @return
> + *   The number of entries in the ring.
> + */
> +__rte_experimental
> +unsigned int
> +rte_soring_count(const struct rte_soring *r);
> +
> +/**
> + * Return the total number of unfilled entries in a ring.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @return
> + *   The number of free entries in the ring.
> + */
> +__rte_experimental
> +unsigned int
> +rte_soring_free_count(const struct rte_soring *r);
> +
> +/**
> + * Dump the status of the soring
> + *
> + * @param f
> + *   A pointer to a file for output
> + * @param r
> + *   Pointer to the soring structure.
> + */
> +__rte_experimental
> +void
> +rte_soring_dump(FILE *f, const struct rte_soring *r);
> +
> +/**
> + * Enqueue several objects on the ring.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param objs
> + *   A pointer to an array of objects to enqueue.
> + *   Size of objects to enqueue must be the same value as 'elem_size' parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param n
> + *   The number of objects to add in the ring from the 'objs'.
> + * @param free_space
> + *   if non-NULL, returns the amount of space in the ring after the
> + *   enqueue operation has finished.
> + * @return
> + *   - Actual number of objects enqueued, either 0 or @n.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
> +       uint32_t n, uint32_t *free_space);
> +
> +/**
> + * Enqueue several objects plus metadata on the ring.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param objs
> + *   A pointer to an array of objects to enqueue.
> + *   Size of objects to enqueue must be the same value as 'elem_size' parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param meta
> + *   A pointer to an array of metadata values for each object to enqueue.
> + *   Note that if user not using object metadata values, then this parameter
> + *   can be NULL.
> + *   Size of elements in this array must be the same value as 'meta_size'
> + *   parameter used while creating the ring. If user created the soring with
> + *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
> + *   Otherwise the results are undefined.
> + * @param n
> + *   The number of objects to add in the ring from the 'objs'.
> + * @param free_space
> + *   if non-NULL, returns the amount of space in the ring after the
> + *   enqueue operation has finished.
> + * @return
> + *   - Actual number of objects enqueued, either 0 or @n.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
> +       const void *meta, uint32_t n, uint32_t *free_space);
> +
> +/**
> + * Enqueue several objects on the ring.

Across the patch, _burst and _bulk has the same API comment

> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param objs
> + *   A pointer to an array of objects to enqueue.
> + *   Size of objects to enqueue must be the same value as 'elem_size' parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param n
> + *   The number of objects to add in the ring from the 'objs'.
> + * @param free_space
> + *   if non-NULL, returns the amount of space in the ring after the
> + *   enqueue operation has finished.
> + * @return
> + *   - Actual number of objects enqueued.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
> +       uint32_t n, uint32_t *free_space);

> +
> +/**
> + * Acquire several objects from the ring for given stage.
> + *
> + * @param r
> + *   A pointer to the soring structure.
> + * @param objs
> + *   A pointer to an array of objects to acquire.
> + *   Size of objects must be the same value as 'elem_size' parameter
> + *   used while creating the ring. Otherwise the results are undefined.
> + * @param stage
> + *   Stage to acquire objects for.
> + * @param num
> + *   The number of objects to acquire.
> + * @param ftoken
> + *   Pointer to the opaque 'token' value used by release() op.
> + *   User has to store this value somewhere, and later provide to the
> + *   release().
> + * @param available
> + *   If non-NULL, returns the number of remaining ring entries for given stage
> + *   after the acquire has finished.
> + * @return
> + *   - Actual number of objects acquired, either 0 or 'num'.
> + */
> +__rte_experimental
> +uint32_t
> +rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
> +       uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);

Does stage needs to be uint32_t?

^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                         ` (4 preceding siblings ...)
  2024-09-17 12:09       ` [PATCH v4 5/5] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-10-12 18:09       ` Stephen Hemminger
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Stephen Hemminger @ 2024-10-12 18:09 UTC (permalink / raw)
  To: Konstantin Ananyev
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, eimear.morrissey,
	Konstantin Ananyev

On Tue, 17 Sep 2024 13:09:41 +0100
Konstantin Ananyev <konstantin.v.ananyev@yandex.ru> wrote:

> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> 
> v2 -> v3:
> - fix compilation/doxygen complains (attempt #2)
> - updated release notes
> 
> v2 -> v3:
> - fix compilation/doxygen complains
> - dropped patch:
>   "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
>   As was mentioned in the patch desctiption it was way too big,
>   controversial and incomplete. If the community is ok to introduce
>   pipeline model into the l3fwd, then it is propbably worth to be
>   a separate patch series.
> 
> v1 -> v2:
> - rename 'elmst/objst' to 'meta' (Morten)
> - introduce new data-path APIs set: one with both meta{} and objs[],
>   second with just objs[] (Morten)
> - split data-path APIs into burst/bulk flavours (same as rte_ring)
> - added dump function for te_soring and improved dump() for rte_ring.
> - dropped patch:
>   " ring: minimize reads of the counterpart cache-line"
>   - no performance gain observed
>   - actually it does change behavior of conventional rte_ring
>     enqueue/dequeue APIs -
>     it could return available/free less then actually exist in the ring.
>     As in some other libs we reliy on that information - it will
>     introduce problems.
> 
> The main aim of these series is to extend ring library with
> new API that allows user to create/use Staged-Ordered-Ring (SORING)
> abstraction. In addition to that there are few other patches that serve
> different purposes:
> - first two patches are just code reordering to de-duplicate
>   and generalize existing rte_ring code.
> - patch #3 extends rte_ring_dump() to correctly print head/tail metadata
>   for different sync modes.
> - next two patches introduce SORING API into the ring library and
>   provide UT for it.
> 
> SORING overview
> ===============
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'. It is based on conventional DPDK
> rte_ring, re-uses many of its concepts, and even substantial part of
> its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in between.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> 
> - number of stages, size and number of objects in the ring are
>  configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation. After some objects are 'acquired' -
>   given thread can safely assume that it has exclusive ownership of
>   these objects till it will invoke 'release' for them.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important.
> 
> The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
> but the internals are different.
> In particular, SORING maintains internal array of 'states' for each element
> in the ring that is  shared by all threads/processes that access the ring.
> That allows 'release' to avoid excessive waits on the tail value and helps
> to improve performancei and scalability.
> In terms of performance, with our measurements rte_soring and
> conventional rte_ring provide nearly identical numbers.
> As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
> l3fwd (--lookup=acl) in pipeline mode [2] both
> rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
> number of worker lcores.
> 
> [1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
> [2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/
> 
> Eimear Morrissey (1):
>   ring: make dump function more verbose
> 
> Konstantin Ananyev (4):
>   ring: common functions for 'move head' ops
>   ring: make copying functions generic
>   ring/soring: introduce Staged Ordered Ring
>   app/test: add unit tests for soring API
> 
>  .mailmap                               |   1 +
>  app/test/meson.build                   |   3 +
>  app/test/test_ring_stress_impl.h       |   1 +
>  app/test/test_soring.c                 | 442 +++++++++++++
>  app/test/test_soring_mt_stress.c       |  40 ++
>  app/test/test_soring_stress.c          |  48 ++
>  app/test/test_soring_stress.h          |  35 ++
>  app/test/test_soring_stress_impl.h     | 827 +++++++++++++++++++++++++
>  doc/guides/rel_notes/release_24_11.rst |   8 +
>  lib/ring/meson.build                   |   4 +-
>  lib/ring/rte_ring.c                    |  87 ++-
>  lib/ring/rte_ring.h                    |  15 +
>  lib/ring/rte_ring_c11_pvt.h            | 134 +---
>  lib/ring/rte_ring_elem_pvt.h           | 181 ++++--
>  lib/ring/rte_ring_generic_pvt.h        | 121 +---
>  lib/ring/rte_ring_hts_elem_pvt.h       |  85 +--
>  lib/ring/rte_ring_rts_elem_pvt.h       |  85 +--
>  lib/ring/rte_soring.c                  | 182 ++++++
>  lib/ring/rte_soring.h                  | 543 ++++++++++++++++
>  lib/ring/soring.c                      | 548 ++++++++++++++++
>  lib/ring/soring.h                      | 124 ++++
>  lib/ring/version.map                   |  26 +
>  22 files changed, 3144 insertions(+), 396 deletions(-)
>  create mode 100644 app/test/test_soring.c
>  create mode 100644 app/test/test_soring_mt_stress.c
>  create mode 100644 app/test/test_soring_stress.c
>  create mode 100644 app/test/test_soring_stress.h
>  create mode 100644 app/test/test_soring_stress_impl.h
>  create mode 100644 lib/ring/rte_soring.c
>  create mode 100644 lib/ring/rte_soring.h
>  create mode 100644 lib/ring/soring.c
>  create mode 100644 lib/ring/soring.h
> 

Makes sense, fix the review comments please.
Also, to keep checkpatch spell checker from generating lots of false
positives; recommend updating build-dict.sh to elide soring from the generated dictionary.

^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 0/6] Stage-Ordered API and other extensions for ring library
  2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                         ` (5 preceding siblings ...)
  2024-10-12 18:09       ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Stephen Hemminger
@ 2024-10-15 13:01       ` Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
                           ` (9 more replies)
  6 siblings, 10 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen,
	Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

NOTE UPFRONT: this version is still not ready for merging.
Missing items:
- ARM/PPC tests passing
- PG update

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (5):
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                               |   1 +
 app/test/meson.build                   |   3 +
 app/test/suites/meson.build            |  10 +
 app/test/test.h                        |   1 +
 app/test/test_ring_stress.c            |   2 +-
 app/test/test_ring_stress_impl.h       |   1 +
 app/test/test_soring.c                 | 442 +++++++++++++
 app/test/test_soring_mt_stress.c       |  40 ++
 app/test/test_soring_stress.c          |  48 ++
 app/test/test_soring_stress.h          |  35 ++
 app/test/test_soring_stress_impl.h     | 834 +++++++++++++++++++++++++
 devtools/build-dict.sh                 |   1 +
 doc/api/doxy-api-index.md              |   1 +
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_ring.c                    |  87 ++-
 lib/ring/rte_ring.h                    |  15 +
 lib/ring/rte_ring_c11_pvt.h            | 134 +---
 lib/ring/rte_ring_elem_pvt.h           | 181 ++++--
 lib/ring/rte_ring_generic_pvt.h        | 121 +---
 lib/ring/rte_ring_hts_elem_pvt.h       |  85 +--
 lib/ring/rte_ring_rts_elem_pvt.h       |  85 +--
 lib/ring/rte_soring.c                  | 182 ++++++
 lib/ring/rte_soring.h                  | 555 ++++++++++++++++
 lib/ring/soring.c                      | 561 +++++++++++++++++
 lib/ring/soring.h                      | 124 ++++
 lib/ring/version.map                   |  26 +
 27 files changed, 3190 insertions(+), 397 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 1/6] ring: common functions for 'move head' ops
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
@ 2024-10-15 13:01         ` Konstantin Ananyev
  2024-10-15 15:04           ` Morten Brørup
  2024-10-15 13:01         ` [PATCH v5 2/6] ring: make copying functions generic Konstantin Ananyev
                           ` (8 subsequent siblings)
  9 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen,
	Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 2/6] ring: make copying functions generic
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-10-15 13:01         ` Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 3/6] ring: make dump function more verbose Konstantin Ananyev
                           ` (7 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen,
	Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 3/6] ring: make dump function more verbose
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 2/6] ring: make copying functions generic Konstantin Ananyev
@ 2024-10-15 13:01         ` Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 4/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                           ` (6 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index f51b1dda5d..2b8d0e4b4f 100644
--- a/.mailmap
+++ b/.mailmap
@@ -383,6 +383,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..8449cb4b15 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 4/6] ring/soring: introduce Staged Ordered Ring
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (2 preceding siblings ...)
  2024-10-15 13:01         ` [PATCH v5 3/6] ring: make dump function more verbose Konstantin Ananyev
@ 2024-10-15 13:01         ` Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 5/6] app/test: add unit tests for soring API Konstantin Ananyev
                           ` (5 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen,
	Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 devtools/build-dict.sh                 |   1 +
 doc/api/doxy-api-index.md              |   1 +
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_soring.c                  | 182 ++++++++
 lib/ring/rte_soring.h                  | 555 ++++++++++++++++++++++++
 lib/ring/soring.c                      | 561 +++++++++++++++++++++++++
 lib/ring/soring.h                      | 124 ++++++
 lib/ring/version.map                   |  19 +
 9 files changed, 1453 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index abd44b1861..63fd0b2715 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -172,6 +172,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index 26590f1b28..e921f38165 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -183,6 +183,14 @@ New Features
   A new ``rte_telemetry_register_cmd_arg`` function is available to pass an opaque value to
   telemetry endpoint callback.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..b6bc71b8c9
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,182 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..21e5dc35ba
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,555 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elemnts in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..f03b9a3d6a
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,561 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (RTE_SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'RTE_SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in RTE_SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & RTE_SORING_ST_MASK) != RTE_SORING_ST_FINISH ||
+				st.ftoken != tail)
+			break;
+
+		k = st.stnum & ~RTE_SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_release,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	union soring_state st;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, r->state,
+				r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+
+		/* check and update status ring */
+		st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+				rte_memory_order_relaxed);
+		RTE_VERIFY(st.raw == 0);
+
+		st.ftoken = head;
+		st.stnum = (RTE_SORING_ST_START | n);
+
+		rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+				rte_memory_order_relaxed);
+		*ftoken = head;
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	idx = ftoken & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	RTE_VERIFY(st.stnum == (RTE_SORING_ST_START | n) &&
+		st.ftoken == ftoken);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = RTE_SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == ftoken)
+		__rte_soring_stage_finalize(&stg->sht, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..c44d8912d9
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,124 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define RTE_SORING_ST_START	RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define RTE_SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define RTE_SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 5/6] app/test: add unit tests for soring API
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (3 preceding siblings ...)
  2024-10-15 13:01         ` [PATCH v5 4/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-10-15 13:01         ` Konstantin Ananyev
  2024-10-15 13:01         ` [PATCH v5 6/6] test: add stress test suite Konstantin Ananyev
                           ` (4 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen,
	Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 834 +++++++++++++++++++++++++++++
 6 files changed, 1402 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index fe248b786c..66860e44e3 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -176,6 +176,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..5ea9bde360
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,834 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v5 6/6] test: add stress test suite
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (4 preceding siblings ...)
  2024-10-15 13:01         ` [PATCH v5 5/6] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-10-15 13:01         ` Konstantin Ananyev
  2024-10-15 15:59         ` [PATCH v5 0/6] Stage-Ordered API and other extensions for ring library Stephen Hemminger
                           ` (3 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-15 13:01 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen,
	Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add a new test suite which purpose is to run 'stress' tests:
main purpose is put a pressure to dpdk sync algorithms
to flag their misbehaving/slowdown/etc.
Right now it consists from just 2 test-cases:
meson test --suite stress-tests --list
DPDK:stress-tests / ring_stress_autotest
DPDK:stress-tests / soring_stress_autotest

These tests are quite time consuming (~15 mins each),
that's another reason to put them into a separate test-suite.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/suites/meson.build   | 10 ++++++++++
 app/test/test.h               |  1 +
 app/test/test_ring_stress.c   |  2 +-
 app/test/test_soring_stress.c |  2 +-
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/app/test/suites/meson.build b/app/test/suites/meson.build
index 191702cf76..e482373330 100644
--- a/app/test/suites/meson.build
+++ b/app/test/suites/meson.build
@@ -5,6 +5,7 @@
 # to complete, so timeout to 10 minutes
 timeout_seconds = 600
 timeout_seconds_fast = 10
+timeout_seconds_stress = 900
 
 test_no_huge_args = ['--no-huge', '-m', '2048']
 has_hugepage = run_command(has_hugepages_cmd, check: true).stdout().strip() != '0'
@@ -21,6 +22,7 @@ endif
 # - fast_tests
 # - perf_tests
 # - driver_tests
+# - stress_tests
 test_suites = run_command(get_test_suites_cmd, autotest_sources,
          check: true).stdout().strip().split()
 foreach suite:test_suites
@@ -39,6 +41,14 @@ foreach suite:test_suites
                     timeout: timeout_seconds,
                     is_parallel: false)
         endforeach
+    elif suite_name == 'stress-tests'
+        foreach t: suite_tests
+            test(t, dpdk_test,
+                    env: ['DPDK_TEST=' + t],
+                    timeout: timeout_seconds_stress,
+                    is_parallel: false,
+                    suite: suite_name)
+        endforeach
     elif suite_name != 'fast-tests'
         # simple cases - tests without parameters or special handling
         foreach t: suite_tests
diff --git a/app/test/test.h b/app/test/test.h
index 15e23d297f..ebc4864bf8 100644
--- a/app/test/test.h
+++ b/app/test/test.h
@@ -208,5 +208,6 @@ void add_test_command(struct test_command *t);
 #define REGISTER_FAST_TEST(cmd, no_huge, ASan, func)  REGISTER_TEST_COMMAND(cmd, func)
 #define REGISTER_PERF_TEST REGISTER_TEST_COMMAND
 #define REGISTER_DRIVER_TEST REGISTER_TEST_COMMAND
+#define REGISTER_STRESS_TEST REGISTER_TEST_COMMAND
 
 #endif
diff --git a/app/test/test_ring_stress.c b/app/test/test_ring_stress.c
index 1af45e0fc8..82e19b02c3 100644
--- a/app/test/test_ring_stress.c
+++ b/app/test/test_ring_stress.c
@@ -63,4 +63,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(ring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(ring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
index 334af6a29c..e5655d49cb 100644
--- a/app/test/test_soring_stress.c
+++ b/app/test/test_soring_stress.c
@@ -45,4 +45,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(soring_stress_autotest, test_ring_stress);
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v5 1/6] ring: common functions for 'move head' ops
  2024-10-15 13:01         ` [PATCH v5 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-10-15 15:04           ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-10-15 15:04 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, eimear.morrissey, stephen, Konstantin Ananyev

> From: Konstantin Ananyev [mailto:konstantin.v.ananyev@yandex.ru]
> Sent: Tuesday, 15 October 2024 15.01

Cleaning up is good. :-)

Only a few initial comments; haven't reviewed in depth.


> +__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
> +		const struct rte_ring_headtail *s, uint32_t capacity,
> +		unsigned int is_st, unsigned int n,
> +		enum rte_ring_queue_behavior behavior,
> +		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)

No description for the new function?
Also in the other files.

>  {
> -	const uint32_t capacity = r->capacity;
> -	uint32_t cons_tail;
> -	unsigned int max = n;
> +	uint32_t stail;
>  	int success;
> +	unsigned int max = n;
> 
> -	*old_head = rte_atomic_load_explicit(&r->prod.head,
> rte_memory_order_relaxed);
> +	*old_head = rte_atomic_load_explicit(&d->head,
> +			rte_memory_order_relaxed);
>  	do {
>  		/* Reset n to the initial burst count */
>  		n = max;
> @@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r,
> unsigned int is_sp,
>  		/* load-acquire synchronize with store-release of ht->tail
>  		 * in update_tail.
>  		 */
> -		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
> +		stail = rte_atomic_load_explicit(&s->tail,
>  					rte_memory_order_acquire);
> 
>  		/* The subtraction is done between two unsigned 32bits
> value
>  		 * (the result is always modulo 32 bits even if we have
> -		 * *old_head > cons_tail). So 'free_entries' is always
> between 0
> +		 * *old_head > s->tail). So 'free_entries' is always
> between 0

The comment refers to 'free_entries', which was renamed to 'entries' in the code.
Also in the other files.


^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v5 0/6] Stage-Ordered API and other extensions for ring library
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (5 preceding siblings ...)
  2024-10-15 13:01         ` [PATCH v5 6/6] test: add stress test suite Konstantin Ananyev
@ 2024-10-15 15:59         ` Stephen Hemminger
  2024-10-15 16:02         ` Stephen Hemminger
                           ` (2 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Stephen Hemminger @ 2024-10-15 15:59 UTC (permalink / raw)
  To: Konstantin Ananyev
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, eimear.morrissey,
	Konstantin Ananyev

On Tue, 15 Oct 2024 14:01:05 +0100
Konstantin Ananyev <konstantin.v.ananyev@yandex.ru> wrote:

> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> 
> NOTE UPFRONT: this version is still not ready for merging.
> Missing items:
> - ARM/PPC tests passing
> - PG update
> 
> v4 -> v5
> - fix public API/doc comments from Jerin
> - update devtools/build-dict.sh (Stephen)
> - fix MSVC warnings
> - introduce new test-suite for meson (stress) with
>   ring_stress_autotest and soring_stress_autotest in it
> - enhance error report in tests
> - reorder some sync code in soring and add extra checks
>   (for better debuggability)
> 
> v3 -> v4:
> - fix compilation/doxygen complains (attempt #2)
> - updated release notes
> 
> v2 -> v3:
> - fix compilation/doxygen complains
> - dropped patch:
>   "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
>   As was mentioned in the patch desctiption it was way too big,
>   controversial and incomplete. If the community is ok to introduce
>   pipeline model into the l3fwd, then it is propbably worth to be
>   a separate patch series.
> 
> v1 -> v2:
> - rename 'elmst/objst' to 'meta' (Morten)
> - introduce new data-path APIs set: one with both meta{} and objs[],
>   second with just objs[] (Morten)
> - split data-path APIs into burst/bulk flavours (same as rte_ring)
> - added dump function for te_soring and improved dump() for rte_ring.
> - dropped patch:
>   " ring: minimize reads of the counterpart cache-line"
>   - no performance gain observed
>   - actually it does change behavior of conventional rte_ring
>     enqueue/dequeue APIs -
>     it could return available/free less then actually exist in the ring.
>     As in some other libs we reliy on that information - it will
>     introduce problems.
> 
> The main aim of these series is to extend ring library with
> new API that allows user to create/use Staged-Ordered-Ring (SORING)
> abstraction. In addition to that there are few other patches that serve
> different purposes:
> - first two patches are just code reordering to de-duplicate
>   and generalize existing rte_ring code.
> - patch #3 extends rte_ring_dump() to correctly print head/tail metadata
>   for different sync modes.
> - next two patches introduce SORING API into the ring library and
>   provide UT for it.
> 
> SORING overview
> ===============
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'. It is based on conventional DPDK
> rte_ring, re-uses many of its concepts, and even substantial part of
> its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in between.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> 
> - number of stages, size and number of objects in the ring are
>  configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation. After some objects are 'acquired' -
>   given thread can safely assume that it has exclusive ownership of
>   these objects till it will invoke 'release' for them.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important.
> 
> The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
> but the internals are different.
> In particular, SORING maintains internal array of 'states' for each element
> in the ring that is  shared by all threads/processes that access the ring.
> That allows 'release' to avoid excessive waits on the tail value and helps
> to improve performancei and scalability.
> In terms of performance, with our measurements rte_soring and
> conventional rte_ring provide nearly identical numbers.
> As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
> l3fwd (--lookup=acl) in pipeline mode [2] both
> rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
> number of worker lcores.
> 
> [1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
> [2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/
> 
> Eimear Morrissey (1):
>   ring: make dump function more verbose
> 
> Konstantin Ananyev (5):
>   ring: common functions for 'move head' ops
>   ring: make copying functions generic
>   ring/soring: introduce Staged Ordered Ring
>   app/test: add unit tests for soring API
>   test: add stress test suite
> 
>  .mailmap                               |   1 +
>  app/test/meson.build                   |   3 +
>  app/test/suites/meson.build            |  10 +
>  app/test/test.h                        |   1 +
>  app/test/test_ring_stress.c            |   2 +-
>  app/test/test_ring_stress_impl.h       |   1 +
>  app/test/test_soring.c                 | 442 +++++++++++++
>  app/test/test_soring_mt_stress.c       |  40 ++
>  app/test/test_soring_stress.c          |  48 ++
>  app/test/test_soring_stress.h          |  35 ++
>  app/test/test_soring_stress_impl.h     | 834 +++++++++++++++++++++++++
>  devtools/build-dict.sh                 |   1 +
>  doc/api/doxy-api-index.md              |   1 +
>  doc/guides/rel_notes/release_24_11.rst |   8 +
>  lib/ring/meson.build                   |   4 +-
>  lib/ring/rte_ring.c                    |  87 ++-
>  lib/ring/rte_ring.h                    |  15 +
>  lib/ring/rte_ring_c11_pvt.h            | 134 +---
>  lib/ring/rte_ring_elem_pvt.h           | 181 ++++--
>  lib/ring/rte_ring_generic_pvt.h        | 121 +---
>  lib/ring/rte_ring_hts_elem_pvt.h       |  85 +--
>  lib/ring/rte_ring_rts_elem_pvt.h       |  85 +--
>  lib/ring/rte_soring.c                  | 182 ++++++
>  lib/ring/rte_soring.h                  | 555 ++++++++++++++++
>  lib/ring/soring.c                      | 561 +++++++++++++++++
>  lib/ring/soring.h                      | 124 ++++
>  lib/ring/version.map                   |  26 +
>  27 files changed, 3190 insertions(+), 397 deletions(-)
>  create mode 100644 app/test/test_soring.c
>  create mode 100644 app/test/test_soring_mt_stress.c
>  create mode 100644 app/test/test_soring_stress.c
>  create mode 100644 app/test/test_soring_stress.h
>  create mode 100644 app/test/test_soring_stress_impl.h
>  create mode 100644 lib/ring/rte_soring.c
>  create mode 100644 lib/ring/rte_soring.h
>  create mode 100644 lib/ring/soring.c
>  create mode 100644 lib/ring/soring.h
> 

And some build failures
####################################################################################
#### [Begin job log] "ubuntu-22.04-gcc-mini" at step Build and test
####################################################################################
../lib/eal/include/rte_bitops.h:1481:9: note: in expansion of macro ‘__RTE_BIT_OVERLOAD_SZ_4R’
 1481 |         __RTE_BIT_OVERLOAD_SZ_4R(family, fun, qualifier, 64, ret_type, arg1_type, arg1_name, \
      |         ^~~~~~~~~~~~~~~~~~~~~~~~
../lib/eal/include/rte_bitops.h:1497:1: note: in expansion of macro ‘__RTE_BIT_OVERLOAD_4R’
 1497 | __RTE_BIT_OVERLOAD_4R(atomic_, test_and_assign,, bool, unsigned int, nr, bool, value,
      | ^~~~~~~~~~~~~~~~~~~~~
../lib/eal/include/rte_bitops.h:1463:1: note: previous declaration ‘bool rte_bit_atomic_test_and_assign(uint32_t*, unsigned int, bool, int)’
 1463 | rte_bit_ ## family ## fun(qualifier uint ## size ## _t *addr, arg1_type arg1_name, \
      | ^~~~~~~~
../lib/eal/include/rte_bitops.h:1472:9: note: in expansion of macro ‘__RTE_BIT_OVERLOAD_V_4R’
 1472 |         __RTE_BIT_OVERLOAD_V_4R(family,, fun, qualifier, size, ret_type, arg1_type, arg1_name, \
      |         ^~~~~~~~~~~~~~~~~~~~~~~
../lib/eal/include/rte_bitops.h:1479:9: note: in expansion of macro ‘__RTE_BIT_OVERLOAD_SZ_4R’
 1479 |         __RTE_BIT_OVERLOAD_SZ_4R(family, fun, qualifier, 32, ret_type, arg1_type, arg1_name, \
      |         ^~~~~~~~~~~~~~~~~~~~~~~~
../lib/eal/include/rte_bitops.h:1497:1: note: in expansion of macro ‘__RTE_BIT_OVERLOAD_4R’
 1497 | __RTE_BIT_OVERLOAD_4R(atomic_, test_and_assign,, bool, unsigned int, nr, bool, value,
      | ^~~~~~~~~~~~~~~~~~~~~
[847/912] Compiling C++ object 'buildtools/chkincs/fe389a9@@chkincs-cpp at exe/meson-generated_rte_mbuf_dyn.cpp.o'.
[848/912] Compiling C++ object 'buildtools/chkincs/fe389a9@@chkincs-cpp at exe/meson-generated_rte_mempool.cpp.o'.
[849/912] Compiling C++ object 'buildtools/chkincs/fe389a9@@chkincs-cpp at exe/meson-generated_rte_mempool_trace_fp.cpp.o'.
[850/912] Compiling C++ object 'buildtools/chkincs/fe389a9@@chkincs-cpp at exe/meson-generated_rte_mbuf.cpp.o'.
[851/912] Compiling C object 'app/a172ced@@dpdk-test at exe/test_test_memcpy_perf.c.o'.
ninja: build stopped: subcommand failed.
##[error]Process completed with exit code 1.
####################################################################################
#### [End job log] "ubuntu-22.04-gcc-mini" at step Build and test
####################################################################################




^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v5 0/6] Stage-Ordered API and other extensions for ring library
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (6 preceding siblings ...)
  2024-10-15 15:59         ` [PATCH v5 0/6] Stage-Ordered API and other extensions for ring library Stephen Hemminger
@ 2024-10-15 16:02         ` Stephen Hemminger
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  9 siblings, 0 replies; 101+ messages in thread
From: Stephen Hemminger @ 2024-10-15 16:02 UTC (permalink / raw)
  To: Konstantin Ananyev
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, eimear.morrissey,
	Konstantin Ananyev

On Tue, 15 Oct 2024 14:01:05 +0100
Konstantin Ananyev <konstantin.v.ananyev@yandex.ru> wrote:

> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> 
> NOTE UPFRONT: this version is still not ready for merging.
> Missing items:
> - ARM/PPC tests passing
> - PG update
> 
> v4 -> v5
> - fix public API/doc comments from Jerin
> - update devtools/build-dict.sh (Stephen)
> - fix MSVC warnings
> - introduce new test-suite for meson (stress) with
>   ring_stress_autotest and soring_stress_autotest in it
> - enhance error report in tests
> - reorder some sync code in soring and add extra checks
>   (for better debuggability)
> 
> v3 -> v4:
> - fix compilation/doxygen complains (attempt #2)
> - updated release notes
> 
> v2 -> v3:
> - fix compilation/doxygen complains
> - dropped patch:
>   "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
>   As was mentioned in the patch desctiption it was way too big,
>   controversial and incomplete. If the community is ok to introduce
>   pipeline model into the l3fwd, then it is propbably worth to be
>   a separate patch series.
> 
> v1 -> v2:
> - rename 'elmst/objst' to 'meta' (Morten)
> - introduce new data-path APIs set: one with both meta{} and objs[],
>   second with just objs[] (Morten)
> - split data-path APIs into burst/bulk flavours (same as rte_ring)
> - added dump function for te_soring and improved dump() for rte_ring.
> - dropped patch:
>   " ring: minimize reads of the counterpart cache-line"
>   - no performance gain observed
>   - actually it does change behavior of conventional rte_ring
>     enqueue/dequeue APIs -
>     it could return available/free less then actually exist in the ring.
>     As in some other libs we reliy on that information - it will
>     introduce problems.
> 
> The main aim of these series is to extend ring library with
> new API that allows user to create/use Staged-Ordered-Ring (SORING)
> abstraction. In addition to that there are few other patches that serve
> different purposes:
> - first two patches are just code reordering to de-duplicate
>   and generalize existing rte_ring code.
> - patch #3 extends rte_ring_dump() to correctly print head/tail metadata
>   for different sync modes.
> - next two patches introduce SORING API into the ring library and
>   provide UT for it.
> 
> SORING overview
> ===============
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'. It is based on conventional DPDK
> rte_ring, re-uses many of its concepts, and even substantial part of
> its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in between.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> 
> - number of stages, size and number of objects in the ring are
>  configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation. After some objects are 'acquired' -
>   given thread can safely assume that it has exclusive ownership of
>   these objects till it will invoke 'release' for them.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important.
> 
> The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
> but the internals are different.
> In particular, SORING maintains internal array of 'states' for each element
> in the ring that is  shared by all threads/processes that access the ring.
> That allows 'release' to avoid excessive waits on the tail value and helps
> to improve performancei and scalability.
> In terms of performance, with our measurements rte_soring and
> conventional rte_ring provide nearly identical numbers.
> As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
> l3fwd (--lookup=acl) in pipeline mode [2] both
> rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
> number of worker lcores.
> 
> [1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
> [2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Also needs its own page in the programmers guide (like ring and eventdev) you could probably use
a variant of the text in this commit message.

^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (7 preceding siblings ...)
  2024-10-15 16:02         ` Stephen Hemminger
@ 2024-10-21 16:08         ` Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
                             ` (5 more replies)
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  9 siblings, 6 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

NOTE UPFRONT: this version is still not ready for merging.
Missing items:
- ARM/PPC tests passing
- PG update

v5 -> v6
- fix problem with ring_stress_autotest (Phanendra)
- added more checks and debug output

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (6):
  test/ring: fix failure with custom number of lcores
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                               |   1 +
 app/test/meson.build                   |   3 +
 app/test/suites/meson.build            |  10 +
 app/test/test.h                        |   1 +
 app/test/test_ring_stress.c            |   2 +-
 app/test/test_ring_stress_impl.h       |   3 +-
 app/test/test_soring.c                 | 442 +++++++++++++
 app/test/test_soring_mt_stress.c       |  40 ++
 app/test/test_soring_stress.c          |  48 ++
 app/test/test_soring_stress.h          |  35 ++
 app/test/test_soring_stress_impl.h     | 834 +++++++++++++++++++++++++
 devtools/build-dict.sh                 |   1 +
 doc/api/doxy-api-index.md              |   1 +
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_ring.c                    |  87 ++-
 lib/ring/rte_ring.h                    |  15 +
 lib/ring/rte_ring_c11_pvt.h            | 134 +---
 lib/ring/rte_ring_elem_pvt.h           | 181 ++++--
 lib/ring/rte_ring_generic_pvt.h        | 121 +---
 lib/ring/rte_ring_hts_elem_pvt.h       |  85 +--
 lib/ring/rte_ring_rts_elem_pvt.h       |  85 +--
 lib/ring/rte_soring.c                  | 201 ++++++
 lib/ring/rte_soring.h                  | 555 ++++++++++++++++
 lib/ring/soring.c                      | 593 ++++++++++++++++++
 lib/ring/soring.h                      | 127 ++++
 lib/ring/version.map                   |  26 +
 27 files changed, 3245 insertions(+), 398 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 1/7] test/ring: fix failure with custom number of lcores
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
@ 2024-10-21 16:08           ` Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
                             ` (4 subsequent siblings)
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, Konstantin Ananyev, stable,
	Phanendra Vukkisala

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

ring_stress_autotest fails to initialize the ring when RTE_MAX_LCORE value
is not a number of 2.
There is a flaw in calculation required number of elements in the ring.
Fix it by aligning number of elements to next power of 2.

Fixes: bf28df24e915 ("test/ring: add contention stress test")
Cc: stable@dpdk.org

Reported-by: Phanendra Vukkisala <pvukkisala@marvell.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/test_ring_stress_impl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..ee5274aeef 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -297,7 +297,7 @@ mt1_init(struct rte_ring **rng, void **data, uint32_t num)
 	*data = elm;
 
 	/* alloc ring */
-	nr = 2 * num;
+	nr = rte_align32pow2(2 * num);
 	sz = rte_ring_get_memsize(nr);
 	r = rte_zmalloc(NULL, sz, alignof(typeof(*r)));
 	if (r == NULL) {
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 2/7] ring: common functions for 'move head' ops
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-10-21 16:08           ` Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 3/7] ring: make copying functions generic Konstantin Ananyev
                             ` (3 subsequent siblings)
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 3/7] ring: make copying functions generic
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-10-21 16:08           ` Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 4/7] ring: make dump function more verbose Konstantin Ananyev
                             ` (2 subsequent siblings)
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 4/7] ring: make dump function more verbose
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
                             ` (2 preceding siblings ...)
  2024-10-21 16:08           ` [PATCH v6 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-10-21 16:08           ` Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 6/7] app/test: add unit tests for soring API Konstantin Ananyev
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 5290420258..7d6fb51ca8 100644
--- a/.mailmap
+++ b/.mailmap
@@ -388,6 +388,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index ee5274aeef..f99a7ff675 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
                             ` (3 preceding siblings ...)
  2024-10-21 16:08           ` [PATCH v6 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-10-21 16:08           ` Konstantin Ananyev
  2024-10-21 16:08           ` [PATCH v6 6/7] app/test: add unit tests for soring API Konstantin Ananyev
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 devtools/build-dict.sh                 |   1 +
 doc/api/doxy-api-index.md              |   1 +
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_soring.c                  | 201 +++++++++
 lib/ring/rte_soring.h                  | 555 +++++++++++++++++++++++
 lib/ring/soring.c                      | 593 +++++++++++++++++++++++++
 lib/ring/soring.h                      | 127 ++++++
 lib/ring/version.map                   |  19 +
 9 files changed, 1507 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index 266c8b90dc..ac7f3cd010 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -173,6 +173,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index fa4822d928..b8684d9f78 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -247,6 +247,14 @@ New Features
   Added ability for node to advertise and update multiple xstat counters,
   that can be retrieved using ``rte_graph_cluster_stats_get``.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..46378ff845
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,201 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+/* compilation-time checks */
+static void
+soring_compilation_checks(void)
+{
+	RTE_BUILD_BUG_ON((sizeof(struct rte_soring) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, cons) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, prod) &
+			RTE_CACHE_LINE_MASK) != 0);
+
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, tail) !=
+		offsetof(struct soring_stage_headtail, tail.pos));
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, sync_type) !=
+		offsetof(struct soring_stage_headtail, unused));
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	soring_compilation_checks();
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..21e5dc35ba
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,555 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elemnts in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..092ea93636
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,593 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht, uint32_t stage,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t ftkn, head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		ftkn = SORING_FTKN_MAKE(tail, stage);
+
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & SORING_ST_MASK) != SORING_ST_FINISH ||
+				st.ftoken != ftkn)
+			break;
+
+		k = st.stnum & ~SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acq_rel,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht, ns,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_verify_state(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	const char *msg, union soring_state val,  union soring_state exp)
+{
+	if (val.raw != exp.raw) {
+		rte_soring_dump(stderr, r);
+		rte_panic("line:%d from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};\n",
+			__LINE__, msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+	}
+}
+
+/* check and update state ring at acquire op*/
+static __rte_always_inline void
+acquire_state_update(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	uint32_t ftoken, uint32_t num)
+{
+	union soring_state st;
+	const union soring_state est = {.raw = 0};
+
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	st.ftoken = ftoken;
+	st.stnum = (SORING_ST_START | num);
+
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, stage - 1,
+				r->state, r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+		*ftoken = SORING_FTKN_MAKE(head, stage);
+
+		/* check and update state value */
+		acquire_state_update(r, stage, idx, *ftoken, n);
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, pos, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	const union soring_state est = {
+		.stnum = (SORING_ST_START | n),
+		.ftoken = ftoken,
+	};
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	pos = SORING_FTKN_POS(ftoken, stage);
+	idx = pos & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == pos)
+		__rte_soring_stage_finalize(&stg->sht, stage, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..1e6f2efc17
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,127 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define SORING_ST_START		RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+#define SORING_FTKN_MAKE(pos, stg)	((pos) + (stg))
+#define SORING_FTKN_POS(ftk, stg)	((ftk) - (stg))
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 6/7] app/test: add unit tests for soring API
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
                             ` (4 preceding siblings ...)
  2024-10-21 16:08           ` [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-10-21 16:08           ` Konstantin Ananyev
  5 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 16:08 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, Konstantin Ananyev

From: Konstantin Ananyev <konstantin.ananyev@huawei.com>

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 834 +++++++++++++++++++++++++++++
 6 files changed, 1402 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index 0f7e11969a..e59f27fd51 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -177,6 +177,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..df4ca9564c
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,834 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
                           ` (8 preceding siblings ...)
  2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
@ 2024-10-21 17:47         ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
                             ` (8 more replies)
  9 siblings, 9 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

NOTE UPFRONT: this version is still not ready for merging.
Missing items:
- ARM/PPC tests passing
- PG update

v5 -> v6
- fix problem with ring_stress_autotest (Phanendra)
- added more checks and debug output

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (6):
  test/ring: fix failure with custom number of lcores
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                               |   1 +
 app/test/meson.build                   |   3 +
 app/test/suites/meson.build            |  10 +
 app/test/test.h                        |   1 +
 app/test/test_ring_stress.c            |   2 +-
 app/test/test_ring_stress_impl.h       |   3 +-
 app/test/test_soring.c                 | 442 +++++++++++++
 app/test/test_soring_mt_stress.c       |  40 ++
 app/test/test_soring_stress.c          |  48 ++
 app/test/test_soring_stress.h          |  35 ++
 app/test/test_soring_stress_impl.h     | 834 +++++++++++++++++++++++++
 devtools/build-dict.sh                 |   1 +
 doc/api/doxy-api-index.md              |   1 +
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_ring.c                    |  87 ++-
 lib/ring/rte_ring.h                    |  15 +
 lib/ring/rte_ring_c11_pvt.h            | 134 +---
 lib/ring/rte_ring_elem_pvt.h           | 181 ++++--
 lib/ring/rte_ring_generic_pvt.h        | 121 +---
 lib/ring/rte_ring_hts_elem_pvt.h       |  85 +--
 lib/ring/rte_ring_rts_elem_pvt.h       |  85 +--
 lib/ring/rte_soring.c                  | 201 ++++++
 lib/ring/rte_soring.h                  | 555 ++++++++++++++++
 lib/ring/soring.c                      | 593 ++++++++++++++++++
 lib/ring/soring.h                      | 127 ++++
 lib/ring/version.map                   |  26 +
 27 files changed, 3245 insertions(+), 398 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 1/7] test/ring: fix failure with custom number of lcores
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
                             ` (7 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen, stable, Phanendra Vukkisala

ring_stress_autotest fails to initialize the ring when RTE_MAX_LCORE value
is not a number of 2.
There is a flaw in calculation required number of elements in the ring.
Fix it by aligning number of elements to next power of 2.

Fixes: bf28df24e915 ("test/ring: add contention stress test")
Cc: stable@dpdk.org

Reported-by: Phanendra Vukkisala <pvukkisala@marvell.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/test_ring_stress_impl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..ee5274aeef 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -297,7 +297,7 @@ mt1_init(struct rte_ring **rng, void **data, uint32_t num)
 	*data = elm;
 
 	/* alloc ring */
-	nr = 2 * num;
+	nr = rte_align32pow2(2 * num);
 	sz = rte_ring_get_memsize(nr);
 	r = rte_zmalloc(NULL, sz, alignof(typeof(*r)));
 	if (r == NULL) {
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 2/7] ring: common functions for 'move head' ops
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 3/7] ring: make copying functions generic Konstantin Ananyev
                             ` (6 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 134 +++++--------------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 121 ++++------------------------
 lib/ring/rte_ring_hts_elem_pvt.h |  85 ++++++--------------
 lib/ring/rte_ring_rts_elem_pvt.h |  85 ++++++--------------
 5 files changed, 149 insertions(+), 342 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..048933ddc6 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -28,41 +28,19 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	rte_atomic_store_explicit(&ht->tail, new_val, rte_memory_order_release);
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +51,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..12f3595926 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -29,36 +29,13 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 	ht->tail = new_val;
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
- * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
- * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
- * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +43,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +53,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..ed5f16879f 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -50,20 +50,16 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue
- */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,7 +70,7 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -82,12 +78,12 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +96,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +124,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..027409a3fa 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -64,20 +64,17 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 	}
 }
 
-/**
- * @internal This function updates the producer head for enqueue.
- */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,7 +85,7 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
@@ -96,12 +93,12 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * *old_head > cons_tail). So 'free_entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +111,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +140,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 3/7] ring: make copying functions generic
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 4/7] ring: make dump function more verbose Konstantin Ananyev
                             ` (5 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 4/7] ring: make dump function more verbose
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                             ` (2 preceding siblings ...)
  2024-10-21 17:47           ` [PATCH v6 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                             ` (4 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 5290420258..7d6fb51ca8 100644
--- a/.mailmap
+++ b/.mailmap
@@ -388,6 +388,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index ee5274aeef..f99a7ff675 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                             ` (3 preceding siblings ...)
  2024-10-21 17:47           ` [PATCH v6 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 6/7] app/test: add unit tests for soring API Konstantin Ananyev
                             ` (3 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 devtools/build-dict.sh                 |   1 +
 doc/api/doxy-api-index.md              |   1 +
 doc/guides/rel_notes/release_24_11.rst |   8 +
 lib/ring/meson.build                   |   4 +-
 lib/ring/rte_soring.c                  | 201 +++++++++
 lib/ring/rte_soring.h                  | 555 +++++++++++++++++++++++
 lib/ring/soring.c                      | 593 +++++++++++++++++++++++++
 lib/ring/soring.h                      | 127 ++++++
 lib/ring/version.map                   |  19 +
 9 files changed, 1507 insertions(+), 2 deletions(-)
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index 266c8b90dc..ac7f3cd010 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -173,6 +173,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index fa4822d928..b8684d9f78 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -247,6 +247,14 @@ New Features
   Added ability for node to advertise and update multiple xstat counters,
   that can be retrieved using ``rte_graph_cluster_stats_get``.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..46378ff845
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,201 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+/* compilation-time checks */
+static void
+soring_compilation_checks(void)
+{
+	RTE_BUILD_BUG_ON((sizeof(struct rte_soring) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, cons) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, prod) &
+			RTE_CACHE_LINE_MASK) != 0);
+
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, tail) !=
+		offsetof(struct soring_stage_headtail, tail.pos));
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, sync_type) !=
+		offsetof(struct soring_stage_headtail, unused));
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	soring_compilation_checks();
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..21e5dc35ba
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,555 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <rte_ring.h>
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elemnts in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..092ea93636
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,593 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht, uint32_t stage,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t ftkn, head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		ftkn = SORING_FTKN_MAKE(tail, stage);
+
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & SORING_ST_MASK) != SORING_ST_FINISH ||
+				st.ftoken != ftkn)
+			break;
+
+		k = st.stnum & ~SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acq_rel,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht, ns,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_verify_state(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	const char *msg, union soring_state val,  union soring_state exp)
+{
+	if (val.raw != exp.raw) {
+		rte_soring_dump(stderr, r);
+		rte_panic("line:%d from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};\n",
+			__LINE__, msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+	}
+}
+
+/* check and update state ring at acquire op*/
+static __rte_always_inline void
+acquire_state_update(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	uint32_t ftoken, uint32_t num)
+{
+	union soring_state st;
+	const union soring_state est = {.raw = 0};
+
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	st.ftoken = ftoken;
+	st.stnum = (SORING_ST_START | num);
+
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, stage - 1,
+				r->state, r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+		*ftoken = SORING_FTKN_MAKE(head, stage);
+
+		/* check and update state value */
+		acquire_state_update(r, stage, idx, *ftoken, n);
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, pos, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	const union soring_state est = {
+		.stnum = (SORING_ST_START | n),
+		.ftoken = ftoken,
+	};
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	pos = SORING_FTKN_POS(ftoken, stage);
+	idx = pos & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == pos)
+		__rte_soring_stage_finalize(&stg->sht, stage, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..1e6f2efc17
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,127 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define SORING_ST_START		RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+#define SORING_FTKN_MAKE(pos, stg)	((pos) + (stg))
+#define SORING_FTKN_POS(ftk, stg)	((ftk) - (stg))
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 6/7] app/test: add unit tests for soring API
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                             ` (4 preceding siblings ...)
  2024-10-21 17:47           ` [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-21 17:47           ` [PATCH v6 7/7] test: add stress test suite Konstantin Ananyev
                             ` (2 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 834 +++++++++++++++++++++++++++++
 6 files changed, 1402 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index 0f7e11969a..e59f27fd51 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -177,6 +177,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..df4ca9564c
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,834 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v6 7/7] test: add stress test suite
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                             ` (5 preceding siblings ...)
  2024-10-21 17:47           ` [PATCH v6 6/7] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-10-21 17:47           ` Konstantin Ananyev
  2024-10-28 17:18           ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library David Christensen
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-21 17:47 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

Add a new test suite which purpose is to run 'stress' tests:
main purpose is put a pressure to dpdk sync algorithms
to flag their misbehaving/slowdown/etc.
Right now it consists from just 2 test-cases:
meson test --suite stress-tests --list
DPDK:stress-tests / ring_stress_autotest
DPDK:stress-tests / soring_stress_autotest

These tests are quite time consuming (~15 mins each),
that's another reason to put them into a separate test-suite.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/suites/meson.build   | 10 ++++++++++
 app/test/test.h               |  1 +
 app/test/test_ring_stress.c   |  2 +-
 app/test/test_soring_stress.c |  2 +-
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/app/test/suites/meson.build b/app/test/suites/meson.build
index 191702cf76..e482373330 100644
--- a/app/test/suites/meson.build
+++ b/app/test/suites/meson.build
@@ -5,6 +5,7 @@
 # to complete, so timeout to 10 minutes
 timeout_seconds = 600
 timeout_seconds_fast = 10
+timeout_seconds_stress = 900
 
 test_no_huge_args = ['--no-huge', '-m', '2048']
 has_hugepage = run_command(has_hugepages_cmd, check: true).stdout().strip() != '0'
@@ -21,6 +22,7 @@ endif
 # - fast_tests
 # - perf_tests
 # - driver_tests
+# - stress_tests
 test_suites = run_command(get_test_suites_cmd, autotest_sources,
          check: true).stdout().strip().split()
 foreach suite:test_suites
@@ -39,6 +41,14 @@ foreach suite:test_suites
                     timeout: timeout_seconds,
                     is_parallel: false)
         endforeach
+    elif suite_name == 'stress-tests'
+        foreach t: suite_tests
+            test(t, dpdk_test,
+                    env: ['DPDK_TEST=' + t],
+                    timeout: timeout_seconds_stress,
+                    is_parallel: false,
+                    suite: suite_name)
+        endforeach
     elif suite_name != 'fast-tests'
         # simple cases - tests without parameters or special handling
         foreach t: suite_tests
diff --git a/app/test/test.h b/app/test/test.h
index 15e23d297f..ebc4864bf8 100644
--- a/app/test/test.h
+++ b/app/test/test.h
@@ -208,5 +208,6 @@ void add_test_command(struct test_command *t);
 #define REGISTER_FAST_TEST(cmd, no_huge, ASan, func)  REGISTER_TEST_COMMAND(cmd, func)
 #define REGISTER_PERF_TEST REGISTER_TEST_COMMAND
 #define REGISTER_DRIVER_TEST REGISTER_TEST_COMMAND
+#define REGISTER_STRESS_TEST REGISTER_TEST_COMMAND
 
 #endif
diff --git a/app/test/test_ring_stress.c b/app/test/test_ring_stress.c
index 1af45e0fc8..82e19b02c3 100644
--- a/app/test/test_ring_stress.c
+++ b/app/test/test_ring_stress.c
@@ -63,4 +63,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(ring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(ring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
index 334af6a29c..e5655d49cb 100644
--- a/app/test/test_soring_stress.c
+++ b/app/test/test_soring_stress.c
@@ -45,4 +45,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(soring_stress_autotest, test_ring_stress);
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                             ` (6 preceding siblings ...)
  2024-10-21 17:47           ` [PATCH v6 7/7] test: add stress test suite Konstantin Ananyev
@ 2024-10-28 17:18           ` David Christensen
  2024-10-29 14:32             ` Konstantin Ananyev
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
  8 siblings, 1 reply; 101+ messages in thread
From: David Christensen @ 2024-10-28 17:18 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, eimear.morrissey, stephen

[-- Attachment #1: Type: text/plain, Size: 316 bytes --]



On 10/21/24 10:47 AM, Konstantin Ananyev wrote:
> NOTE UPFRONT: this version is still not ready for merging.
> Missing items:
> - ARM/PPC tests passing
> - PG update

SORING tests pass on a P9 PowerNV system. (Results attached for review.)
Original ring_stress_autotest still fails as documented in Bug 1527.

Dave

[-- Attachment #2: soring-ppc-pass.txt --]
[-- Type: text/plain, Size: 24203 bytes --]

~/src/dpdk/build/app/test/dpdk-test -l 2-127 --no-pci --no-huge soring_stress_autotest
EAL: Detected CPU lcores: 128
EAL: Detected NUMA nodes: 2
EAL: Static memory layout is selected, amount of reserved memory can be adjusted with -m or --socket-mem
EAL: Detected static linkage of DPDK
EAL: Multi-process socket /run/user/1000/dpdk/rte/mp_socket
EAL: Selected IOVA mode 'VA'
APP: HPET is not enabled, using TSC as default timer
RTE>>soring_stress_autotest
TEST-CASE MT MT_DEQENQ-MT_STG1-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720098753(60000192.88 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=27847142,
		nb_obj=787705656,
		nb_cycle=3088261505751,
		obj/call(avg): 28.29
		cycles/obj(avg): 3920.58
		cycles/call(avg): 110900.48
		max cycles/call=69561043(135861.41 usec),
		min cycles/call=63(0.12 usec),
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=27847142,
		nb_obj=787709713,
		nb_cycle=51194411622,
		obj/call(avg): 28.29
		cycles/obj(avg): 64.99
		cycles/call(avg): 1838.41
		max cycles/call=810480(1582.97 usec),
		min cycles/call=22(0.04 usec),
	},
};
soring <mt1_init>@0x103d87900
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=787705656
  cons.tail=787705656
  prod.sync_type=multi thread
  prod.head=787713848
  prod.tail=787713848
  nb_stage=1
  stage[0].tail.pos=787705709
  stage[0].tail.sync=0
  stage[0].head=787709713
TEST-CASE MT MT_DEQENQ-MT_STG1-PRCS OK
TEST-CASE MT MT_DEQENQ-MT_STG1-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720091030(60000177.79 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=28317736,
		nb_obj=774446728,
		nb_cycle=3840007072287,
		obj/call(avg): 27.35
		cycles/obj(avg): 4958.39
		cycles/call(avg): 135604.31
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=28317736,
		nb_obj=774450875,
		nb_cycle=3840007072287,
		obj/call(avg): 27.35
		cycles/obj(avg): 4958.36
		cycles/call(avg): 135604.31
	},
};
soring <mt1_init>@0x103d87900
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=774446728
  cons.tail=774446728
  prod.sync_type=multi thread
  prod.head=774454920
  prod.tail=774454920
  nb_stage=1
  stage[0].tail.pos=774446751
  stage[0].tail.sync=0
  stage[0].head=774450875
TEST-CASE MT MT_DEQENQ-MT_STG1-AVG OK
TEST-CASE MT MT_DEQENQ-MT_STG4-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720110117(60000215.07 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=29697958,
		nb_obj=846300330,
		nb_cycle=2230878102976,
		obj/call(avg): 28.50
		cycles/obj(avg): 2636.04
		cycles/call(avg): 75118.91
		max cycles/call=46136094(90109.56 usec),
		min cycles/call=63(0.12 usec),
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=29697958,
		nb_obj=846304420,
		nb_cycle=47534073852,
		obj/call(avg): 28.50
		cycles/obj(avg): 56.17
		cycles/call(avg): 1600.58
		max cycles/call=269319(526.01 usec),
		min cycles/call=21(0.04 usec),
	},
	STAGE#1={
		nb_lcore=125,
		nb_call=29697958,
		nb_obj=846303741,
		nb_cycle=42701590291,
		obj/call(avg): 28.50
		cycles/obj(avg): 50.46
		cycles/call(avg): 1437.86
		max cycles/call=289448(565.33 usec),
		min cycles/call=27(0.05 usec),
	},
	STAGE#2={
		nb_lcore=125,
		nb_call=29697958,
		nb_obj=846302445,
		nb_cycle=41652357581,
		obj/call(avg): 28.50
		cycles/obj(avg): 49.22
		cycles/call(avg): 1402.53
		max cycles/call=2025124(3955.32 usec),
		min cycles/call=27(0.05 usec),
	},
	STAGE#3={
		nb_lcore=125,
		nb_call=29697958,
		nb_obj=846301760,
		nb_cycle=40106867652,
		obj/call(avg): 28.50
		cycles/obj(avg): 47.39
		cycles/call(avg): 1350.49
		max cycles/call=237244(463.37 usec),
		min cycles/call=27(0.05 usec),
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=846300330
  cons.tail=846300330
  prod.sync_type=multi thread
  prod.head=846308522
  prod.tail=846308522
  nb_stage=4
  stage[0].tail.pos=846303758
  stage[0].tail.sync=0
  stage[0].head=846304420
  stage[1].tail.pos=846302512
  stage[1].tail.sync=0
  stage[1].head=846303741
  stage[2].tail.pos=846302445
  stage[2].tail.sync=0
  stage[2].head=846302445
  stage[3].tail.pos=846300393
  stage[3].tail.sync=0
  stage[3].head=846301760
TEST-CASE MT MT_DEQENQ-MT_STG4-PRCS OK
TEST-CASE MT MT_DEQENQ-MT_STG4-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720110142(60000215.12 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=28451159,
		nb_obj=846733560,
		nb_cycle=3840008212464,
		obj/call(avg): 29.76
		cycles/obj(avg): 4535.08
		cycles/call(avg): 134968.43
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=28451159,
		nb_obj=846737450,
		nb_cycle=3840008212464,
		obj/call(avg): 29.76
		cycles/obj(avg): 4535.06
		cycles/call(avg): 134968.43
	},
	STAGE#1={
		nb_lcore=125,
		nb_call=28451159,
		nb_obj=846736826,
		nb_cycle=3840008212464,
		obj/call(avg): 29.76
		cycles/obj(avg): 4535.07
		cycles/call(avg): 134968.43
	},
	STAGE#2={
		nb_lcore=125,
		nb_call=28451159,
		nb_obj=846736166,
		nb_cycle=3840008212464,
		obj/call(avg): 29.76
		cycles/obj(avg): 4535.07
		cycles/call(avg): 134968.43
	},
	STAGE#3={
		nb_lcore=125,
		nb_call=28451159,
		nb_obj=846735134,
		nb_cycle=3840008212464,
		obj/call(avg): 29.76
		cycles/obj(avg): 4535.08
		cycles/call(avg): 134968.43
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=846733560
  cons.tail=846733560
  prod.sync_type=multi thread
  prod.head=846741752
  prod.tail=846741752
  nb_stage=4
  stage[0].tail.pos=846736856
  stage[0].tail.sync=0
  stage[0].head=846737450
  stage[1].tail.pos=846736212
  stage[1].tail.sync=0
  stage[1].head=846736826
  stage[2].tail.pos=846735186
  stage[2].tail.sync=0
  stage[2].head=846736166
  stage[3].tail.pos=846733570
  stage[3].tail.sync=0
  stage[3].head=846735134
TEST-CASE MT MT_DEQENQ-MT_STG4-AVG OK
TEST-CASE MT MTRTS_DEQENQ-MT_STG4-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720037900(60000074.02 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=55786006,
		nb_obj=889191739,
		nb_cycle=2558329725635,
		obj/call(avg): 15.94
		cycles/obj(avg): 2877.14
		cycles/call(avg): 45859.70
		max cycles/call=2790909(5450.99 usec),
		min cycles/call=49(0.10 usec),
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=55786006,
		nb_obj=889198697,
		nb_cycle=73560844419,
		obj/call(avg): 15.94
		cycles/obj(avg): 82.73
		cycles/call(avg): 1318.63
		max cycles/call=262968(513.61 usec),
		min cycles/call=23(0.04 usec),
	},
	STAGE#1={
		nb_lcore=125,
		nb_call=55786006,
		nb_obj=889198386,
		nb_cycle=111168637303,
		obj/call(avg): 15.94
		cycles/obj(avg): 125.02
		cycles/call(avg): 1992.77
		max cycles/call=994515(1942.41 usec),
		min cycles/call=27(0.05 usec),
	},
	STAGE#2={
		nb_lcore=125,
		nb_call=55786006,
		nb_obj=889198113,
		nb_cycle=109356595441,
		obj/call(avg): 15.94
		cycles/obj(avg): 122.98
		cycles/call(avg): 1960.29
		max cycles/call=678340(1324.88 usec),
		min cycles/call=27(0.05 usec),
	},
	STAGE#3={
		nb_lcore=125,
		nb_call=55786006,
		nb_obj=889197965,
		nb_cycle=104683790166,
		obj/call(avg): 15.94
		cycles/obj(avg): 117.73
		cycles/call(avg): 1876.52
		max cycles/call=1251419(2444.18 usec),
		min cycles/call=27(0.05 usec),
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread - RTS
  cons.head.pos=889191739
  cons.head.cnt=27709173
  cons.tail.pos=889191739
  cons.tail.cnt=27709173
  cons.htd_max=0
  prod.sync_type=multi thread - RTS
  prod.head.pos=889199931
  prod.head.cnt=27717365
  prod.tail.pos=889199931
  prod.tail.cnt=27717365
  prod.htd_max=0
  nb_stage=4
  stage[0].tail.pos=889198697
  stage[0].tail.sync=0
  stage[0].head=889198697
  stage[1].tail.pos=889198386
  stage[1].tail.sync=0
  stage[1].head=889198386
  stage[2].tail.pos=889198113
  stage[2].tail.sync=0
  stage[2].head=889198113
  stage[3].tail.pos=889191739
  stage[3].tail.sync=0
  stage[3].head=889197965
TEST-CASE MT MTRTS_DEQENQ-MT_STG4-PRCS OK
TEST-CASE MT MTRTS_DEQENQ-MT_STG4-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720037694(60000073.62 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=63933184,
		nb_obj=886396598,
		nb_cycle=3840002670696,
		obj/call(avg): 13.86
		cycles/obj(avg): 4332.15
		cycles/call(avg): 60062.75
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=63933184,
		nb_obj=886403255,
		nb_cycle=3840002670696,
		obj/call(avg): 13.86
		cycles/obj(avg): 4332.12
		cycles/call(avg): 60062.75
	},
	STAGE#1={
		nb_lcore=125,
		nb_call=63933184,
		nb_obj=886403014,
		nb_cycle=3840002670696,
		obj/call(avg): 13.86
		cycles/obj(avg): 4332.12
		cycles/call(avg): 60062.75
	},
	STAGE#2={
		nb_lcore=125,
		nb_call=63933184,
		nb_obj=886402603,
		nb_cycle=3840002670696,
		obj/call(avg): 13.86
		cycles/obj(avg): 4332.12
		cycles/call(avg): 60062.75
	},
	STAGE#3={
		nb_lcore=125,
		nb_call=63933184,
		nb_obj=886402335,
		nb_cycle=3840002670696,
		obj/call(avg): 13.86
		cycles/obj(avg): 4332.12
		cycles/call(avg): 60062.75
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread - RTS
  cons.head.pos=886396598
  cons.head.cnt=27771444
  cons.tail.pos=886396598
  cons.tail.cnt=27771444
  cons.htd_max=0
  prod.sync_type=multi thread - RTS
  prod.head.pos=886404790
  prod.head.cnt=27779636
  prod.tail.pos=886404790
  prod.tail.cnt=27779636
  prod.htd_max=0
  nb_stage=4
  stage[0].tail.pos=886403086
  stage[0].tail.sync=0
  stage[0].head=886403255
  stage[1].tail.pos=886403014
  stage[1].tail.sync=0
  stage[1].head=886403014
  stage[2].tail.pos=886402377
  stage[2].tail.sync=0
  stage[2].head=886402603
  stage[3].tail.pos=886396624
  stage[3].tail.sync=0
  stage[3].head=886402335
TEST-CASE MT MTRTS_DEQENQ-MT_STG4-AVG OK
TEST-CASE MT MTHTS_DEQENQ-MT_STG4-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720074024(60000144.58 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=46551171,
		nb_obj=1068227537,
		nb_cycle=2499810847933,
		obj/call(avg): 22.95
		cycles/obj(avg): 2340.15
		cycles/call(avg): 53700.28
		max cycles/call=15438015(30152.37 usec),
		min cycles/call=49(0.10 usec),
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=46551171,
		nb_obj=1068232309,
		nb_cycle=99067129064,
		obj/call(avg): 22.95
		cycles/obj(avg): 92.74
		cycles/call(avg): 2128.13
		max cycles/call=1958062(3824.34 usec),
		min cycles/call=23(0.04 usec),
	},
	STAGE#1={
		nb_lcore=125,
		nb_call=46551171,
		nb_obj=1068231598,
		nb_cycle=46852180540,
		obj/call(avg): 22.95
		cycles/obj(avg): 43.86
		cycles/call(avg): 1006.47
		max cycles/call=248770(485.88 usec),
		min cycles/call=27(0.05 usec),
	},
	STAGE#2={
		nb_lcore=125,
		nb_call=46551171,
		nb_obj=1068230644,
		nb_cycle=44096278409,
		obj/call(avg): 22.95
		cycles/obj(avg): 41.28
		cycles/call(avg): 947.26
		max cycles/call=270901(529.10 usec),
		min cycles/call=27(0.05 usec),
	},
	STAGE#3={
		nb_lcore=125,
		nb_call=46551171,
		nb_obj=1068229762,
		nb_cycle=44541158664,
		obj/call(avg): 22.95
		cycles/obj(avg): 41.70
		cycles/call(avg): 956.82
		max cycles/call=227162(443.68 usec),
		min cycles/call=27(0.05 usec),
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread - HTS
  cons.head=1068227537
  cons.tail=1068227537
  prod.sync_type=multi thread - HTS
  prod.head=1068235729
  prod.tail=1068235729
  nb_stage=4
  stage[0].tail.pos=1068232309
  stage[0].tail.sync=0
  stage[0].head=1068232309
  stage[1].tail.pos=1068231598
  stage[1].tail.sync=0
  stage[1].head=1068231598
  stage[2].tail.pos=1068230644
  stage[2].tail.sync=0
  stage[2].head=1068230644
  stage[3].tail.pos=1068227579
  stage[3].tail.sync=0
  stage[3].head=1068229762
TEST-CASE MT MTHTS_DEQENQ-MT_STG4-PRCS OK
TEST-CASE MT MTHTS_DEQENQ-MT_STG4-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720069256(60000135.27 usec),
	DEQ+ENQ={
		nb_lcore=125,
		nb_call=47383088,
		nb_obj=1047939017,
		nb_cycle=3840004139686,
		obj/call(avg): 22.12
		cycles/obj(avg): 3664.34
		cycles/call(avg): 81041.66
	},
	STAGE#0={
		nb_lcore=125,
		nb_call=47383088,
		nb_obj=1047943910,
		nb_cycle=3840004139686,
		obj/call(avg): 22.12
		cycles/obj(avg): 3664.32
		cycles/call(avg): 81041.66
	},
	STAGE#1={
		nb_lcore=125,
		nb_call=47383088,
		nb_obj=1047943575,
		nb_cycle=3840004139686,
		obj/call(avg): 22.12
		cycles/obj(avg): 3664.32
		cycles/call(avg): 81041.66
	},
	STAGE#2={
		nb_lcore=125,
		nb_call=47383088,
		nb_obj=1047942583,
		nb_cycle=3840004139686,
		obj/call(avg): 22.12
		cycles/obj(avg): 3664.33
		cycles/call(avg): 81041.66
	},
	STAGE#3={
		nb_lcore=125,
		nb_call=47383088,
		nb_obj=1047941571,
		nb_cycle=3840004139686,
		obj/call(avg): 22.12
		cycles/obj(avg): 3664.33
		cycles/call(avg): 81041.66
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread - HTS
  cons.head=1047939017
  cons.tail=1047939017
  prod.sync_type=multi thread - HTS
  prod.head=1047947209
  prod.tail=1047947209
  nb_stage=4
  stage[0].tail.pos=1047943910
  stage[0].tail.sync=0
  stage[0].head=1047943910
  stage[1].tail.pos=1047943575
  stage[1].tail.sync=0
  stage[1].head=1047943575
  stage[2].tail.pos=1047942583
  stage[2].tail.sync=0
  stage[2].head=1047942583
  stage[3].tail.pos=1047939033
  stage[3].tail.sync=0
  stage[3].head=1047941571
TEST-CASE MT MTHTS_DEQENQ-MT_STG4-AVG OK
TEST-CASE MT MT_DEQENQ-MT_STG5-1:1-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720046097(60000090.03 usec),
	DEQ+ENQ={
		nb_lcore=63,
		nb_call=193297299,
		nb_obj=830587316,
		nb_cycle=1753304759124,
		obj/call(avg): 4.30
		cycles/obj(avg): 2110.92
		cycles/call(avg): 9070.51
		max cycles/call=205886284(402121.65 usec),
		min cycles/call=60(0.12 usec),
	},
	STAGE#0={
		nb_lcore=62,
		nb_call=41824714,
		nb_obj=830593145,
		nb_cycle=45140522868,
		obj/call(avg): 19.86
		cycles/obj(avg): 54.35
		cycles/call(avg): 1079.28
		max cycles/call=224483(438.44 usec),
		min cycles/call=22(0.04 usec),
	},
	STAGE#1={
		nb_lcore=62,
		nb_call=41824714,
		nb_obj=830592406,
		nb_cycle=62951345574,
		obj/call(avg): 19.86
		cycles/obj(avg): 75.79
		cycles/call(avg): 1505.12
		max cycles/call=188947(369.04 usec),
		min cycles/call=26(0.05 usec),
	},
	STAGE#2={
		nb_lcore=62,
		nb_call=41824714,
		nb_obj=830591483,
		nb_cycle=66296234652,
		obj/call(avg): 19.86
		cycles/obj(avg): 79.82
		cycles/call(avg): 1585.10
		max cycles/call=207984(406.22 usec),
		min cycles/call=26(0.05 usec),
	},
	STAGE#3={
		nb_lcore=62,
		nb_call=41824714,
		nb_obj=830591193,
		nb_cycle=67630143639,
		obj/call(avg): 19.86
		cycles/obj(avg): 81.42
		cycles/call(avg): 1616.99
		max cycles/call=583718660(1140075.51 usec),
		min cycles/call=26(0.05 usec),
	},
	STAGE#4={
		nb_lcore=62,
		nb_call=41824714,
		nb_obj=830589415,
		nb_cycle=98176191780,
		obj/call(avg): 19.86
		cycles/obj(avg): 118.20
		cycles/call(avg): 2347.32
		max cycles/call=1186780(2317.93 usec),
		min cycles/call=26(0.05 usec),
	},
};
soring <mt1_init>@0x103d87500
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=830587316
  cons.tail=830587316
  prod.sync_type=multi thread
  prod.head=830595508
  prod.tail=830595508
  nb_stage=5
  stage[0].tail.pos=830593145
  stage[0].tail.sync=0
  stage[0].head=830593145
  stage[1].tail.pos=830591493
  stage[1].tail.sync=0
  stage[1].head=830592406
  stage[2].tail.pos=830591483
  stage[2].tail.sync=0
  stage[2].head=830591483
  stage[3].tail.pos=830589511
  stage[3].tail.sync=0
  stage[3].head=830591193
  stage[4].tail.pos=830589415
  stage[4].tail.sync=0
  stage[4].head=830589415
TEST-CASE MT MT_DEQENQ-MT_STG5-1:1-PRCS OK
TEST-CASE MT MT_DEQENQ-MT_STG5-1:1-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720040696(60000079.48 usec),
	DEQ+ENQ={
		nb_lcore=63,
		nb_call=67558484,
		nb_obj=878226749,
		nb_cycle=3840002743760,
		obj/call(avg): 13.00
		cycles/obj(avg): 4372.45
		cycles/call(avg): 56839.68
	},
	STAGE#0={
		nb_lcore=62,
		nb_call=29755675,
		nb_obj=878233006,
		nb_cycle=3840002743760,
		obj/call(avg): 29.51
		cycles/obj(avg): 4372.42
		cycles/call(avg): 129051.11
	},
	STAGE#1={
		nb_lcore=62,
		nb_call=29755675,
		nb_obj=878232563,
		nb_cycle=3840002743760,
		obj/call(avg): 29.51
		cycles/obj(avg): 4372.42
		cycles/call(avg): 129051.11
	},
	STAGE#2={
		nb_lcore=62,
		nb_call=29755675,
		nb_obj=878229963,
		nb_cycle=3840002743760,
		obj/call(avg): 29.51
		cycles/obj(avg): 4372.43
		cycles/call(avg): 129051.11
	},
	STAGE#3={
		nb_lcore=62,
		nb_call=29755675,
		nb_obj=878229573,
		nb_cycle=3840002743760,
		obj/call(avg): 29.51
		cycles/obj(avg): 4372.44
		cycles/call(avg): 129051.11
	},
	STAGE#4={
		nb_lcore=62,
		nb_call=29755675,
		nb_obj=878229175,
		nb_cycle=3840002743760,
		obj/call(avg): 29.51
		cycles/obj(avg): 4372.44
		cycles/call(avg): 129051.11
	},
};
soring <mt1_init>@0x103d87500
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=878226749
  cons.tail=878226749
  prod.sync_type=multi thread
  prod.head=878234941
  prod.tail=878234941
  nb_stage=5
  stage[0].tail.pos=878233006
  stage[0].tail.sync=0
  stage[0].head=878233006
  stage[1].tail.pos=878229980
  stage[1].tail.sync=0
  stage[1].head=878232563
  stage[2].tail.pos=878229963
  stage[2].tail.sync=0
  stage[2].head=878229963
  stage[3].tail.pos=878229573
  stage[3].tail.sync=0
  stage[3].head=878229573
  stage[4].tail.pos=878226791
  stage[4].tail.sync=0
  stage[4].head=878229175
TEST-CASE MT MT_DEQENQ-MT_STG5-1:1-AVG OK
TEST-CASE MT MT_DEQENQ-MT_STG3-1:3-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720017757(60000034.68 usec),
	DEQ+ENQ={
		nb_lcore=32,
		nb_call=31528155,
		nb_obj=998446438,
		nb_cycle=855915150579,
		obj/call(avg): 31.67
		cycles/obj(avg): 857.25
		cycles/call(avg): 27147.64
		max cycles/call=72687573(141967.92 usec),
		min cycles/call=60(0.12 usec),
	},
	STAGE#0={
		nb_lcore=31,
		nb_call=320722118,
		nb_obj=998453896,
		nb_cycle=682556209823,
		obj/call(avg): 3.11
		cycles/obj(avg): 683.61
		cycles/call(avg): 2128.19
		max cycles/call=1237928(2417.83 usec),
		min cycles/call=20(0.04 usec),
	},
	STAGE#1={
		nb_lcore=31,
		nb_call=194349733,
		nb_obj=998452839,
		nb_cycle=676488489398,
		obj/call(avg): 5.14
		cycles/obj(avg): 677.54
		cycles/call(avg): 3480.78
		max cycles/call=968055(1890.73 usec),
		min cycles/call=25(0.05 usec),
	},
	STAGE#2={
		nb_lcore=31,
		nb_call=411662927,
		nb_obj=998452583,
		nb_cycle=677820349719,
		obj/call(avg): 2.43
		cycles/obj(avg): 678.87
		cycles/call(avg): 1646.54
		max cycles/call=1315169(2568.69 usec),
		min cycles/call=25(0.05 usec),
	},
};
soring <mt1_init>@0x103d87700
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=998446438
  cons.tail=998446438
  prod.sync_type=multi thread
  prod.head=998454630
  prod.tail=998454630
  nb_stage=3
  stage[0].tail.pos=998453896
  stage[0].tail.sync=0
  stage[0].head=998453896
  stage[1].tail.pos=998452637
  stage[1].tail.sync=0
  stage[1].head=998452839
  stage[2].tail.pos=998446499
  stage[2].tail.sync=0
  stage[2].head=998452583
TEST-CASE MT MT_DEQENQ-MT_STG3-1:3-PRCS OK
TEST-CASE MT MT_DEQENQ_MT_STG3-1:3-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720024728(60000048.30 usec),
	DEQ+ENQ={
		nb_lcore=32,
		nb_call=107802554,
		nb_obj=947693347,
		nb_cycle=3840000721156,
		obj/call(avg): 8.79
		cycles/obj(avg): 4051.94
		cycles/call(avg): 35620.68
	},
	STAGE#0={
		nb_lcore=31,
		nb_call=567711845,
		nb_obj=947700589,
		nb_cycle=3840000721156,
		obj/call(avg): 1.67
		cycles/obj(avg): 4051.91
		cycles/call(avg): 6764.00
	},
	STAGE#1={
		nb_lcore=31,
		nb_call=256412852,
		nb_obj=947700067,
		nb_cycle=3840000721156,
		obj/call(avg): 3.70
		cycles/obj(avg): 4051.92
		cycles/call(avg): 14975.85
	},
	STAGE#2={
		nb_lcore=31,
		nb_call=582085636,
		nb_obj=947699643,
		nb_cycle=3840000721156,
		obj/call(avg): 1.63
		cycles/obj(avg): 4051.92
		cycles/call(avg): 6596.97
	},
};
soring <mt1_init>@0x103d87700
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=multi thread
  cons.head=947693347
  cons.tail=947693347
  prod.sync_type=multi thread
  prod.head=947701539
  prod.tail=947701539
  nb_stage=3
  stage[0].tail.pos=947700124
  stage[0].tail.sync=0
  stage[0].head=947700589
  stage[1].tail.pos=947699721
  stage[1].tail.sync=0
  stage[1].head=947700067
  stage[2].tail.pos=947693392
  stage[2].tail.sync=0
  stage[2].head=947699643
TEST-CASE MT MT_DEQENQ_MT_STG3-1:3-AVG OK
TEST-CASE MT ST_DEQENQ-MT_STG4-PRCS START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720013384(60000026.14 usec),
	DEQ+ENQ={
		nb_lcore=1,
		nb_call=12619106,
		nb_obj=447944710,
		nb_cycle=8624496083,
		obj/call(avg): 35.50
		cycles/obj(avg): 19.25
		cycles/call(avg): 683.45
		max cycles/call=14091(27.52 usec),
		min cycles/call=87(0.17 usec),
	},
	STAGE#0={
		nb_lcore=124,
		nb_call=441581732,
		nb_obj=447952874,
		nb_cycle=390660949632,
		obj/call(avg): 1.01
		cycles/obj(avg): 872.10
		cycles/call(avg): 884.69
		max cycles/call=380056(742.30 usec),
		min cycles/call=22(0.04 usec),
	},
	STAGE#1={
		nb_lcore=124,
		nb_call=441581732,
		nb_obj=447952874,
		nb_cycle=1262220406919,
		obj/call(avg): 1.01
		cycles/obj(avg): 2817.75
		cycles/call(avg): 2858.41
		max cycles/call=2355156(4599.91 usec),
		min cycles/call=26(0.05 usec),
	},
	STAGE#2={
		nb_lcore=124,
		nb_call=441581732,
		nb_obj=447952874,
		nb_cycle=1377046741852,
		obj/call(avg): 1.01
		cycles/obj(avg): 3074.09
		cycles/call(avg): 3118.44
		max cycles/call=276530217(540098.08 usec),
		min cycles/call=26(0.05 usec),
	},
	STAGE#3={
		nb_lcore=124,
		nb_call=441581732,
		nb_obj=447952870,
		nb_cycle=490812305789,
		obj/call(avg): 1.01
		cycles/obj(avg): 1095.68
		cycles/call(avg): 1111.49
		max cycles/call=500174(976.90 usec),
		min cycles/call=26(0.05 usec),
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=single thread
  cons.head=447944710
  cons.tail=447944710
  prod.sync_type=single thread
  prod.head=447952902
  prod.tail=447952902
  nb_stage=4
  stage[0].tail.pos=447952874
  stage[0].tail.sync=0
  stage[0].head=447952874
  stage[1].tail.pos=447952874
  stage[1].tail.sync=0
  stage[1].head=447952874
  stage[2].tail.pos=447952874
  stage[2].tail.sync=0
  stage[2].head=447952874
  stage[3].tail.pos=447944776
  stage[3].tail.sync=0
  stage[3].head=447952870
TEST-CASE MT ST_DEQENQ-MT_STG4-PRCS OK
TEST-CASE MT ST_DEQENQ-MT_STG4-AVG START
lcore_stat_dump(AGGREGATE)={
	nb_cycle=30720014501(60000028.32 usec),
	DEQ+ENQ={
		nb_lcore=1,
		nb_call=12786318,
		nb_obj=453861172,
		nb_cycle=3840000700136,
		obj/call(avg): 35.50
		cycles/obj(avg): 8460.74
		cycles/call(avg): 300321.07
	},
	STAGE#0={
		nb_lcore=124,
		nb_call=473456135,
		nb_obj=453869322,
		nb_cycle=3840000700136,
		obj/call(avg): 0.96
		cycles/obj(avg): 8460.59
		cycles/call(avg): 8110.57
	},
	STAGE#1={
		nb_lcore=124,
		nb_call=473456135,
		nb_obj=453869322,
		nb_cycle=3840000700136,
		obj/call(avg): 0.96
		cycles/obj(avg): 8460.59
		cycles/call(avg): 8110.57
	},
	STAGE#2={
		nb_lcore=124,
		nb_call=473456135,
		nb_obj=453869322,
		nb_cycle=3840000700136,
		obj/call(avg): 0.96
		cycles/obj(avg): 8460.59
		cycles/call(avg): 8110.57
	},
	STAGE#3={
		nb_lcore=124,
		nb_call=473456135,
		nb_obj=453869274,
		nb_cycle=3840000700136,
		obj/call(avg): 0.96
		cycles/obj(avg): 8460.59
		cycles/call(avg): 8110.57
	},
};
soring <mt1_init>@0x103d87600
  size=16384
  capacity=8192
  esize=8
  msize=0
  used=8192
  avail=0
  cons.sync_type=single thread
  cons.head=453861172
  cons.tail=453861172
  prod.sync_type=single thread
  prod.head=453869364
  prod.tail=453869364
  nb_stage=4
  stage[0].tail.pos=453869322
  stage[0].tail.sync=0
  stage[0].head=453869322
  stage[1].tail.pos=453869322
  stage[1].tail.sync=0
  stage[1].head=453869322
  stage[2].tail.pos=453869322
  stage[2].tail.sync=0
  stage[2].head=453869322
  stage[3].tail.pos=453861268
  stage[3].tail.sync=0
  stage[3].head=453869274
TEST-CASE MT ST_DEQENQ-MT_STG4-AVG OK
Number of tests:	14
Success:	14
Failed:	0
Test OK

^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-28 17:18           ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library David Christensen
@ 2024-10-29 14:32             ` Konstantin Ananyev
  0 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-29 14:32 UTC (permalink / raw)
  To: David Christensen, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, drc, ruifeng.wang,
	mb, Eimear Morrissey, stephen



> 
> On 10/21/24 10:47 AM, Konstantin Ananyev wrote:
> > NOTE UPFRONT: this version is still not ready for merging.
> > Missing items:
> > - ARM/PPC tests passing
> > - PG update
> 
> SORING tests pass on a P9 PowerNV system. (Results attached for review.)
> Original ring_stress_autotest still fails as documented in Bug 1527.

Thanks a lot Dave, I suppose it is a positive thing for soring.
About ring_stress_autotest - I'll try to have another look at RTS mode
next week or so, but it is really hard without access to the machine
where the problem is reproducible.
 

^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                             ` (7 preceding siblings ...)
  2024-10-28 17:18           ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library David Christensen
@ 2024-10-30 21:22           ` Konstantin Ananyev
  2024-10-30 21:22             ` [PATCH v7 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
                               ` (9 more replies)
  8 siblings, 10 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:22 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Testing coverage (passed):
x86_64, i686, PPC, ARM   

Would like to express my gratitude to all community members who helped me
with testing it on different platforms, in particular:
David Christensen <drc@linux.ibm.com>
Cody Cheng <ccheng@iol.unh.edu>
Patrick Robb <probb@iol.unh.edu>
Phanendra Vukkisala <pvukkisala@marvell.com>
Chengwen Feng <fengchengwen@huawei.com>

v6 -> v7
- updated Programmer Guide (Jerin, Morten, Stephen)
- fix some functions in public headers without comments (Morten)
- update debug checks, added new macro for that: RTE_SORING_DEBUG
  (disabled by default).

v5 -> v6
- fix problem with ring_stress_autotest (Phanendra)
- added more checks and debug output

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (6):
  test/ring: fix failure with custom number of lcores
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                                  |   1 +
 app/test/meson.build                      |   3 +
 app/test/suites/meson.build               |  10 +
 app/test/test.h                           |   1 +
 app/test/test_ring_stress.c               |   2 +-
 app/test/test_ring_stress_impl.h          |   3 +-
 app/test/test_soring.c                    | 442 ++++++++++++
 app/test/test_soring_mt_stress.c          |  40 ++
 app/test/test_soring_stress.c             |  48 ++
 app/test/test_soring_stress.h             |  35 +
 app/test/test_soring_stress_impl.h        | 834 ++++++++++++++++++++++
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 ++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_ring.c                       |  87 ++-
 lib/ring/rte_ring.h                       |  15 +
 lib/ring/rte_ring_c11_pvt.h               | 156 ++--
 lib/ring/rte_ring_elem_pvt.h              | 181 +++--
 lib/ring/rte_ring_generic_pvt.h           | 143 ++--
 lib/ring/rte_ring_hts_elem_pvt.h          | 107 ++-
 lib/ring/rte_ring_rts_elem_pvt.h          | 107 ++-
 lib/ring/rte_soring.c                     | 198 +++++
 lib/ring/rte_soring.h                     | 556 +++++++++++++++
 lib/ring/soring.c                         | 613 ++++++++++++++++
 lib/ring/soring.h                         | 138 ++++
 lib/ring/version.map                      |  26 +
 29 files changed, 4217 insertions(+), 380 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 1/7] test/ring: fix failure with custom number of lcores
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
@ 2024-10-30 21:22             ` Konstantin Ananyev
  2024-11-07 11:50               ` Morten Brørup
  2024-10-30 21:22             ` [PATCH v7 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
                               ` (8 subsequent siblings)
  9 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:22 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen, stable

ring_stress_autotest fails to initialize the ring when RTE_MAX_LCORE value
is not a number of 2.
There is a flaw in calculation required number of elements in the ring.
Fix it by aligning number of elements to next power of 2.

Fixes: bf28df24e915 ("test/ring: add contention stress test")
Cc: stable@dpdk.org

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/test_ring_stress_impl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..ee5274aeef 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -297,7 +297,7 @@ mt1_init(struct rte_ring **rng, void **data, uint32_t num)
 	*data = elm;
 
 	/* alloc ring */
-	nr = 2 * num;
+	nr = rte_align32pow2(2 * num);
 	sz = rte_ring_get_memsize(nr);
 	r = rte_zmalloc(NULL, sz, alignof(typeof(*r)));
 	if (r == NULL) {
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 2/7] ring: common functions for 'move head' ops
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
  2024-10-30 21:22             ` [PATCH v7 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-10-30 21:22             ` Konstantin Ananyev
  2024-11-07 11:31               ` Morten Brørup
  2024-10-30 21:23             ` [PATCH v7 3/7] ring: make copying functions generic Konstantin Ananyev
                               ` (7 subsequent siblings)
  9 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:22 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_c11_pvt.h      | 156 ++++++++++---------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 143 +++++++++-------------------
 lib/ring/rte_ring_hts_elem_pvt.h | 107 ++++++++++-----------
 lib/ring/rte_ring_rts_elem_pvt.h | 107 ++++++++++-----------
 5 files changed, 255 insertions(+), 324 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..b9388af0da 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -11,6 +11,17 @@
 #ifndef _RTE_RING_C11_PVT_H_
 #define _RTE_RING_C11_PVT_H_
 
+/**
+ * @file rte_ring_c11_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -29,40 +40,45 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +89,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..affd2d5ba7 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -10,6 +10,17 @@
 #ifndef _RTE_RING_GENERIC_PVT_H_
 #define _RTE_RING_GENERIC_PVT_H_
 
+/**
+ * @file rte_ring_generic_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -30,35 +41,39 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +81,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +91,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..e2b82dd1e6 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -51,19 +51,39 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,20 +94,20 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +120,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +148,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..96825931f8 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -65,19 +65,40 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue.
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,20 +109,20 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +135,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +164,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 3/7] ring: make copying functions generic
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
  2024-10-30 21:22             ` [PATCH v7 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
  2024-10-30 21:22             ` [PATCH v7 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-10-30 21:23             ` Konstantin Ananyev
  2024-11-07 11:46               ` Morten Brørup
  2024-10-30 21:23             ` [PATCH v7 4/7] ring: make dump function more verbose Konstantin Ananyev
                               ` (6 subsequent siblings)
  9 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:23 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 4/7] ring: make dump function more verbose
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (2 preceding siblings ...)
  2024-10-30 21:23             ` [PATCH v7 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-10-30 21:23             ` Konstantin Ananyev
  2024-11-07 11:49               ` Morten Brørup
  2024-10-30 21:23             ` [PATCH v7 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                               ` (5 subsequent siblings)
  9 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:23 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 5290420258..7d6fb51ca8 100644
--- a/.mailmap
+++ b/.mailmap
@@ -388,6 +388,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index ee5274aeef..f99a7ff675 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..261f2a06db 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "single thread";
+	case RTE_RING_SYNC_MT:
+		return "multi thread";
+	case RTE_RING_SYNC_MT_RTS:
+		return "multi thread - RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "multi thread - HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 5/7] ring/soring: introduce Staged Ordered Ring
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (3 preceding siblings ...)
  2024-10-30 21:23             ` [PATCH v7 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-10-30 21:23             ` Konstantin Ananyev
  2024-11-07 12:07               ` Morten Brørup
  2024-10-30 21:23             ` [PATCH v7 6/7] app/test: add unit tests for soring API Konstantin Ananyev
                               ` (4 subsequent siblings)
  9 siblings, 1 reply; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:23 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 +++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_soring.c                     | 198 +++++++
 lib/ring/rte_soring.h                     | 556 +++++++++++++++++++
 lib/ring/soring.c                         | 613 +++++++++++++++++++++
 lib/ring/soring.h                         | 138 +++++
 lib/ring/version.map                      |  19 +
 11 files changed, 2373 insertions(+), 2 deletions(-)
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index 266c8b90dc..ac7f3cd010 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -173,6 +173,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/prog_guide/img/soring-pic1.svg b/doc/guides/prog_guide/img/soring-pic1.svg
new file mode 100644
index 0000000000..c97e66ca43
--- /dev/null
+++ b/doc/guides/prog_guide/img/soring-pic1.svg
@@ -0,0 +1,635 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<!-- SPDX-License-Identifier: BSD-3-Clause -->
+
+<!-- Copyright(c) 2010 Intel Corporation -->
+
+<svg
+   width="484.05716"
+   height="385.63785"
+   id="svg3388"
+   version="1.1"
+   inkscape:version="1.2.1 (9c6d41e410, 2022-07-14)"
+   sodipodi:docname="soring-pic1.svg"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:dc="http://purl.org/dc/elements/1.1/">
+  <defs
+     id="defs3390">
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend"
+       style="overflow:visible">
+      <path
+         id="path4317"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       sodipodi:type="inkscape:persp3d"
+       inkscape:vp_x="0 : 526.18109 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_z="744.09448 : 526.18109 : 1"
+       inkscape:persp3d-origin="372.04724 : 350.78739 : 1"
+       id="perspective3396" />
+    <inkscape:perspective
+       id="perspective4180"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-6"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-0"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-3"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-06"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-5"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-69"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281-2"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4767"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-7"
+       style="overflow:visible">
+      <path
+         id="path4317-4"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective4799"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4824"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4915"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4937"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4962"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4993"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-0"
+       style="overflow:visible">
+      <path
+         id="path4317-6"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker4999"
+       style="overflow:visible">
+      <path
+         id="path5001"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5091"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-9"
+       style="overflow:visible">
+      <path
+         id="path4317-0"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5121"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-1"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-9"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5710"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-6"
+       style="overflow:visible">
+      <path
+         id="path4317-7"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5738"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="1"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="232.14286"
+     inkscape:cy="168.21429"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     inkscape:window-width="1483"
+     inkscape:window-height="1002"
+     inkscape:window-x="83"
+     inkscape:window-y="16"
+     inkscape:window-maximized="0"
+     inkscape:snap-grids="false"
+     inkscape:snap-to-guides="true"
+     showguides="false"
+     fit-margin-top="0.1"
+     fit-margin-left="0.1"
+     fit-margin-right="0.1"
+     fit-margin-bottom="0.1"
+     inkscape:showpageshadow="2"
+     inkscape:pagecheckerboard="0"
+     inkscape:deskcolor="#d1d1d1">
+    <inkscape:grid
+       type="xygrid"
+       id="grid5162"
+       empspacing="5"
+       visible="true"
+       enabled="true"
+       snapvisiblegridlinesonly="true"
+       originx="-162.97143"
+       originy="-368.03525"
+       spacingx="1"
+       spacingy="1" />
+  </sodipodi:namedview>
+  <metadata
+     id="metadata3393">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-162.97143,-298.68909)">
+    <rect
+       style="fill:#ffd080;fill-opacity:1;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+       id="rect4257"
+       width="439.41635"
+       height="58.588848"
+       x="186.87822"
+       y="463.44324"
+       rx="11.631636"
+       ry="11.631636" />
+    <g
+       id="g4259"
+       transform="translate(108.51492,3.9469318)">
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="83.143028"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="137.00014"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-3"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="190.85725"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-1"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="244.71437"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-6"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="298.57147"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-2"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="352.42859"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-15"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="406.28571"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-4"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="460.14282"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-65"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+    </g>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="528.41443"
+       y="496.36957"
+       id="text4269-5-6-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5-0"
+         x="528.41443"
+         y="496.36957"
+         style="font-size:14px;line-height:1.25">obj5</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="313.90488"
+       y="495.49646"
+       id="text4269"><tspan
+         sodipodi:role="line"
+         id="tspan4271"
+         x="313.90488"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj1</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="368.95203"
+       y="495.49646"
+       id="text4269-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-5"
+         x="368.95203"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj2</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="422.99518"
+       y="495.49646"
+       id="text4269-5"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4"
+         x="422.99518"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj3</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 323.57143,578.07647 0,-42.14286"
+       id="path4309"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="289.85715"
+       y="589.505"
+       id="text4787"><tspan
+         sodipodi:role="line"
+         id="tspan4789"
+         x="289.85715"
+         y="589.505"
+         style="font-size:14px;line-height:1.25">cons_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="293.45334"
+       y="603.41034"
+       id="text4787-3"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0"
+         x="293.45334"
+         y="603.41034"
+         style="font-size:14px;line-height:1.25">cons_tail</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="561.2981"
+       y="602.81482"
+       id="text4787-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8"
+         x="561.2981"
+         y="602.81482"
+         style="font-size:14px;line-height:1.25">prod_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="566.75146"
+       y="587.72028"
+       id="text4787-3-6"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0-8"
+         x="566.75146"
+         y="587.72028"
+         style="font-size:14px;line-height:1.25">prod_tail</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="315.21933"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="172.28571"
+       y="308.93362"
+       id="text4891"><tspan
+         sodipodi:role="line"
+         id="tspan4893"
+         x="172.28571"
+         y="308.93362"
+         style="font-size:14px;line-height:1.25">lstages states</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889-8"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="529.93365"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="170.89287"
+       y="684.09021"
+       id="text4891-4"><tspan
+         sodipodi:role="line"
+         id="tspan4893-3"
+         x="170.89287"
+         y="684.09021"
+         style="font-size:14px;line-height:1.25">producer and consumer states</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 379.25296,407.43361 v 42.14286"
+       id="path4309-8"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 486.68152,407.43361 0,42.14286"
+       id="path4309-4-9"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;white-space:pre;inline-size:99.1521;display:inline;fill:#000000;fill-opacity:1;stroke:none"
+       x="296.992"
+       y="399.48123"
+       id="text4787-3-64"
+       transform="translate(54)"><tspan
+         x="296.992"
+         y="399.48123"
+         id="tspan1631"><tspan
+           style="font-size:14px;line-height:1.25"
+           id="tspan1629">stage[1].tail</tspan></tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="440.26532"
+       y="399.48123"
+       id="text4787-7-5"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0"
+         x="440.26532"
+         y="399.48123"
+         style="font-size:14px;line-height:1.25">stage[1].head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="441.55243"
+       y="377.24719"
+       id="text4787-7-5-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-4"
+         x="441.55243"
+         y="377.24719"
+         style="font-size:14px;line-height:1.25">stage[0].tail</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 593.14285,407.43361 v 42.14286"
+       id="path4309-4-9-9"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="M 594,575.57647 V 533.43361"
+       id="path4309-4-3"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="476.46902"
+       y="495.12097"
+       id="text4269-5-6"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5"
+         x="476.46902"
+         y="495.12097"
+         style="font-size:14px;line-height:1.25">obj4</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="552.98102"
+       y="397.24719"
+       id="text4787-7-5-74"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-8"
+         x="552.98102"
+         y="397.24719"
+         style="font-size:14px;line-height:1.25">stage[0].head</tspan></text>
+  </g>
+</svg>
diff --git a/doc/guides/prog_guide/ring_lib.rst b/doc/guides/prog_guide/ring_lib.rst
index f7dbba0e4e..a023c2f697 100644
--- a/doc/guides/prog_guide/ring_lib.rst
+++ b/doc/guides/prog_guide/ring_lib.rst
@@ -491,6 +491,208 @@ Following is an example of usage:
 Note that between ``_start_`` and ``_finish_`` no other thread can proceed
 with enqueue(/dequeue) operation till ``_finish_`` completes.
 
+Staged Ordered Ring API
+-----------------------
+
+Staged-Ordered-Ring (SORING) API provides a SW abstraction for *ordered* queues
+with multiple processing *stages*. It is based on conventional DPDK
+``rte_ring`` API, re-uses many of its concepts, and even substantial part of
+its code. It can be viewed as an 'extension' of ``rte_ring`` functionality.
+In particular, main SORING properties:
+
+*   circular ring buffer with fixed size objects and related metadata
+
+*   producer, consumer plus multiple processing stages in between.
+
+*   allows to split objects processing into multiple stages.
+
+*   objects remain in the same ring while moving from one stage to the other,
+    initial order is preserved, no extra copying needed.
+
+*   preserves the ingress order of objects within the queue across multiple
+    stages
+
+*   each stage (and producer/consumer) can be served by single and/or
+    multiple threads.
+
+*   number of stages, size and number of objects and their metadata in the
+    ring are configurable at ring initialization time.
+
+Data-Path API
+~~~~~~~~~~~~~
+
+SORING data-path API provided four main operations:
+
+*   ``enqueue``/``dequeue`` works in the same manner as for conventional
+    ``rte_ring``, all rte_ring synchronization types are supported.
+
+*   ``acquire``/``release`` - for each stage there is an ``acquire`` (start)
+    and ``release`` (finish) operation.
+    After some objects are ``acquired`` - given thread can safely assume that
+    it has exclusive possession of these objects till ``release`` for them is
+    invoked.
+    Note that right now user has to release exactly the same number of
+    objects that was acquired before.
+    After objects are ``released``, given thread loses its possession on them,
+    and they can be either acquired by next stage or dequeued
+    by the consumer (in case of last stage).
+
+A simplified representation of a SORING with two stages is shown below.
+On that picture ``obj5`` and ``obj4`` elements are acquired by stage 0,
+``obj2`` and ``obj3`` are acquired by stage 1, while ``obj11`` was already
+released by stage 1 and is ready to be consumed.
+
+.. _figure_soring1:
+
+.. figure:: img/soring-pic1.*
+
+Along with traditional flavor there are enhanced versions for all these
+data-path operations: ``enqueux``/``dequeux``/``acquirx``/``releasx``.
+All enhanced versions take as extra parameter a pointer to an array of
+metadata values.
+At initialization user can request within the ``soring`` supplementary and
+optional array of metadata associated with each object in the ``soring``.
+While ``soring`` element size is configurable and user can specify it big
+enough to hold both object and its metadata together,
+for performance reasons it might be plausible to access them as separate arrays.
+Note that users are free to mix and match both versions of data-path API in
+a way they like.
+As an example, possible usage scenario when such separation helps:
+
+.. code-block:: c
+
+    /*
+     * use pointer to mbuf as soring element, while tx_state
+     * as a metadata.
+     * In this example we use a soring with just one stage.
+     */
+     union tx_state {
+         /* negative values for error */
+         int32_t rc;
+         /* otherwise contain valid TX port and queue IDs*/
+         struct {
+             uint16_t port_id;
+             uint16_t queue_id;
+          } tx;
+     };
+     struct rte_soring *soring;
+
+
+producer/consumer part:
+
+.. code-block:: c
+
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* enqueue - writes to soring objects array no need to update metadata */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_enqueue_burst(soring, pkts, num, NULL);
+     ....
+     /* dequeux - reads both packets and related tx_state */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_dequeux_burst(soring, pkts, txst, num, NULL);
+
+     /*
+      * TX packets out, or drop in case of error.
+      * Note that we don't need to dereference the soring objects itself
+      * to make a decision.
+      */
+     uint32_t i, j, k, n;
+     struct rte_mbuf *dr[MAX_PKT_BURST];
+
+     k = 0;
+     for (i = 0; i != num; i++) {
+         /* packet processing reports an error */
+         if (txst[i].rc < 0)
+             dr[k++] = pkts[i];
+         /* valid packet, send it out */
+         else {
+             /* group consequitive packets with the same port and queue IDs */
+             for (j = i + 1; j < num; j++)
+                 if (txst[j].rc != txst[i].rc)
+                     break;
+
+             n = rte_eth_tx_burst(txst[i].tx.port_id, txst[i].tx.queue_id,
+                             pkts + i, j - i);
+             if (i + n != j) {
+                 /* decide with unsent packets if any */
+              }
+         }
+      }
+      /* drop errorneous packets */
+      if (k != 0)
+          rte_pktmbuf_free_bulk(dr, k);
+
+acquire/release part:
+
+.. code-block:: c
+
+     uint32_t ftoken;
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* acquire - grab some packets to process */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_acquire_burst(soring, pkts, 0, num, &ftoken, NULL);
+
+     /* process packets, fill txst[] for each */
+     do_process_packets(pkts, txst, num);
+
+     /*
+      * release - assuming that do_process_packets() didn't change
+      * contents of pkts[], we need to update soring metadata array only.
+      */
+     rte_soring_releasx(soring, NULL, txst, 0, num, ftoken);
+
+Use Cases
+~~~~~~~~~~
+
+Expected use-cases include applications that use pipeline model
+(probably with multiple stages) for packet processing, when preserving
+incoming packet order is important. I.E.: IPsec processing, etc.
+
+SORING internals
+~~~~~~~~~~~~~~~~
+
+*   In addition to accessible by the user array of objects (and metadata),
+    ``soirng`` also contains an internal array of states. Each ``state[]``
+    corresponds to exactly one object within the soring. That ``state[]``
+    array is used by ``acquire``/``release``/``dequeue`` operations to
+    store internal information and should not be accessed by the user directly.
+
+*   At ``acquire``, soring  moves stage's head (in a same way as ``rte_ring``
+    ``move_head`` does), plus it saves in ``state[stage.old_head]``
+    information about how many elements were acquired, acquired head position,
+    and special flag value to indicate that given elements are acquired
+    (``SORING_ST_START``).
+    Note that ``acquire`` returns an opaque ``ftoken`` value that user has
+    to provide for ``release`` function.
+
+*   ``release`` extracts old head value from provided by user ``ftoken`` and
+    checks that corresponding ``state[]`` entry contains expected values
+    (mostly for sanity purposes). Then it marks this ``state[]`` entry with
+    ``SORING_ST_FINISH`` flag to indicate that given subset of objects was
+    released. After that, it checks does stage's old ``head`` value equals to
+    its current ``tail`` value. If so, then it performs ``finalize``
+    operation, otherwise ``release`` just returns.
+
+*   As ``state[]`` is shared by all threads, some other thread can perform
+    ``finalize`` operation for given stage. That allows ``release`` to avoid
+    excessive waits on the ``tail`` value.
+    Main purpose of ``finalize`` operation is to walk through ``state[]``
+    array  from current stage's ``tail`` position up to its ``head``,
+    check ``state[]`` and move stage ``tail`` through elements that already
+    are released (in ``SORING_ST_FINISH`` state).
+    Along with that, corresponding ``state[]`` entries are reset back to zero.
+    Note that ``finalize`` for given stage can be called from multiple places:
+    from ``release`` for that stage or from ``acquire`` for next stage, or
+    even from consumer's ``dequeue`` - in case given stage is the last one.
+    So ``finalize`` has to be MT-safe and inside it we have to guarantee that
+    at any given moment only one thread can update stage's ``tail`` and reset
+    corresponding ``state[]`` entries.
+
+
 References
 ----------
 
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index fa4822d928..b8684d9f78 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -247,6 +247,14 @@ New Features
   Added ability for node to advertise and update multiple xstat counters,
   that can be retrieved using ``rte_graph_cluster_stats_get``.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..e297a2181d
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,198 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if ret-code size is a multiple of 4B */
+	if (stsize % 4 != 0) {
+		SORING_LOG(ERR, "invalid retcode size: %u", stsize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t stsize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * stsize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+/* compilation-time checks */
+static void
+soring_compilation_checks(void)
+{
+	RTE_BUILD_BUG_ON((sizeof(struct rte_soring) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, cons) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, prod) &
+			RTE_CACHE_LINE_MASK) != 0);
+
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, tail) !=
+		offsetof(struct soring_stage_headtail, tail.pos));
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, sync_type) !=
+		offsetof(struct soring_stage_headtail, unused));
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	soring_compilation_checks();
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..6d0976b1ac
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,556 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Extra debugging might be enabled with RTE_SORING_DEBUG macro.
+ */
+
+#include <rte_ring.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elemnts in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and ret-code tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..e8fe890597
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,613 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head
+ * plus current stage index).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht, uint32_t stage,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t ftkn, head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/* Ensure the head is read before rstate[] */
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		ftkn = SORING_FTKN_MAKE(tail, stage);
+
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & SORING_ST_MASK) != SORING_ST_FINISH ||
+				st.ftoken != ftkn)
+			break;
+
+		k = st.stnum & ~SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acq_rel,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht, ns,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+/*
+ * Verify internal SORING state.
+ * WARNING: if expected value is not equal to actual one, it means that for
+ * whatever reason SORING data constancy is broken. That is a very serious
+ * problem that most likely will cause race-conditions, memory corruption,
+ * program crash.
+ * To ease debugging it user might rebuild ring library with
+ * RTE_SORING_DEBUG enabled.
+ */
+static __rte_always_inline void
+soring_verify_state(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	const char *msg, union soring_state val,  union soring_state exp)
+{
+	if (val.raw != exp.raw) {
+#ifdef RTE_SORING_DEBUG
+		rte_soring_dump(stderr, r);
+		rte_panic("line:%d from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};\n",
+			__LINE__, msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#else
+		SORING_LOG(EMERG, "from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};",
+			msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#endif
+	}
+}
+
+/* check and update state ring at acquire op*/
+static __rte_always_inline void
+acquire_state_update(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	uint32_t ftoken, uint32_t num)
+{
+	union soring_state st;
+	const union soring_state est = {.raw = 0};
+
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	st.ftoken = ftoken;
+	st.stnum = (SORING_ST_START | num);
+
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, stage - 1,
+				r->state, r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+		*ftoken = SORING_FTKN_MAKE(head, stage);
+
+		/* check and update state value */
+		acquire_state_update(r, stage, idx, *ftoken, n);
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, pos, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	const union soring_state est = {
+		.stnum = (SORING_ST_START | n),
+		.ftoken = ftoken,
+	};
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	pos = SORING_FTKN_POS(ftoken, stage);
+	idx = pos & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == pos)
+		__rte_soring_stage_finalize(&stg->sht, stage, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..b252ae4a61
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,138 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+/* logging stuff, register our own tag for SORING */
+#include <rte_log.h>
+
+extern int soring_logtype;
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+/**
+ * SORING internal state for each element
+ */
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define SORING_ST_START		RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+#define SORING_FTKN_MAKE(pos, stg)	((pos) + (stg))
+#define SORING_FTKN_POS(ftk, stg)	((ftk) - (stg))
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 6/7] app/test: add unit tests for soring API
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (4 preceding siblings ...)
  2024-10-30 21:23             ` [PATCH v7 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-10-30 21:23             ` Konstantin Ananyev
  2024-10-30 21:23             ` [PATCH v7 7/7] test: add stress test suite Konstantin Ananyev
                               ` (3 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:23 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 834 +++++++++++++++++++++++++++++
 6 files changed, 1402 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index 0f7e11969a..e59f27fd51 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -177,6 +177,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..df4ca9564c
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,834 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v7 7/7] test: add stress test suite
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (5 preceding siblings ...)
  2024-10-30 21:23             ` [PATCH v7 6/7] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-10-30 21:23             ` Konstantin Ananyev
  2024-11-07 10:41             ` [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                               ` (2 subsequent siblings)
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-10-30 21:23 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add a new test suite which purpose is to run 'stress' tests:
main purpose is put a pressure to dpdk sync algorithms
to flag their misbehaving/slowdown/etc.
Right now it consists from just 2 test-cases:
meson test --suite stress-tests --list
DPDK:stress-tests / ring_stress_autotest
DPDK:stress-tests / soring_stress_autotest

These tests are quite time consuming (~15 mins each),
that's another reason to put them into a separate test-suite.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
---
 app/test/suites/meson.build   | 10 ++++++++++
 app/test/test.h               |  1 +
 app/test/test_ring_stress.c   |  2 +-
 app/test/test_soring_stress.c |  2 +-
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/app/test/suites/meson.build b/app/test/suites/meson.build
index 191702cf76..e482373330 100644
--- a/app/test/suites/meson.build
+++ b/app/test/suites/meson.build
@@ -5,6 +5,7 @@
 # to complete, so timeout to 10 minutes
 timeout_seconds = 600
 timeout_seconds_fast = 10
+timeout_seconds_stress = 900
 
 test_no_huge_args = ['--no-huge', '-m', '2048']
 has_hugepage = run_command(has_hugepages_cmd, check: true).stdout().strip() != '0'
@@ -21,6 +22,7 @@ endif
 # - fast_tests
 # - perf_tests
 # - driver_tests
+# - stress_tests
 test_suites = run_command(get_test_suites_cmd, autotest_sources,
          check: true).stdout().strip().split()
 foreach suite:test_suites
@@ -39,6 +41,14 @@ foreach suite:test_suites
                     timeout: timeout_seconds,
                     is_parallel: false)
         endforeach
+    elif suite_name == 'stress-tests'
+        foreach t: suite_tests
+            test(t, dpdk_test,
+                    env: ['DPDK_TEST=' + t],
+                    timeout: timeout_seconds_stress,
+                    is_parallel: false,
+                    suite: suite_name)
+        endforeach
     elif suite_name != 'fast-tests'
         # simple cases - tests without parameters or special handling
         foreach t: suite_tests
diff --git a/app/test/test.h b/app/test/test.h
index 15e23d297f..ebc4864bf8 100644
--- a/app/test/test.h
+++ b/app/test/test.h
@@ -208,5 +208,6 @@ void add_test_command(struct test_command *t);
 #define REGISTER_FAST_TEST(cmd, no_huge, ASan, func)  REGISTER_TEST_COMMAND(cmd, func)
 #define REGISTER_PERF_TEST REGISTER_TEST_COMMAND
 #define REGISTER_DRIVER_TEST REGISTER_TEST_COMMAND
+#define REGISTER_STRESS_TEST REGISTER_TEST_COMMAND
 
 #endif
diff --git a/app/test/test_ring_stress.c b/app/test/test_ring_stress.c
index 1af45e0fc8..82e19b02c3 100644
--- a/app/test/test_ring_stress.c
+++ b/app/test/test_ring_stress.c
@@ -63,4 +63,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(ring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(ring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
index 334af6a29c..e5655d49cb 100644
--- a/app/test/test_soring_stress.c
+++ b/app/test/test_soring_stress.c
@@ -45,4 +45,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(soring_stress_autotest, test_ring_stress);
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (6 preceding siblings ...)
  2024-10-30 21:23             ` [PATCH v7 7/7] test: add stress test suite Konstantin Ananyev
@ 2024-11-07 10:41             ` Konstantin Ananyev
  2024-11-07 16:16             ` Stephen Hemminger
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
  9 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 10:41 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, Eimear Morrissey, stephen, thomas,
	david.marchand

Hi everyone,

Does anyone have any extra comments for that series?
If not, would it be possible to consider it for 24.11 rc2?
Thomas and David what are your thoughts on that subject?
Thanks
Konstantin

> -----Original Message-----
> From: Konstantin Ananyev <konstantin.ananyev@huawei.com>
> Sent: Wednesday, October 30, 2024 9:23 PM
> To: dev@dpdk.org
> Cc: honnappa.nagarahalli@arm.com; jerinj@marvell.com; hemant.agrawal@nxp.com; bruce.richardson@intel.com;
> drc@linux.vnet.ibm.com; ruifeng.wang@arm.com; mb@smartsharesystems.com; Eimear Morrissey
> <eimear.morrissey@huawei.com>; stephen@networkplumber.org
> Subject: [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library
> 
> Testing coverage (passed):
> x86_64, i686, PPC, ARM
> 
> Would like to express my gratitude to all community members who helped me
> with testing it on different platforms, in particular:
> David Christensen <drc@linux.ibm.com>
> Cody Cheng <ccheng@iol.unh.edu>
> Patrick Robb <probb@iol.unh.edu>
> Phanendra Vukkisala <pvukkisala@marvell.com>
> Chengwen Feng <fengchengwen@huawei.com>
> 
> v6 -> v7
> - updated Programmer Guide (Jerin, Morten, Stephen)
> - fix some functions in public headers without comments (Morten)
> - update debug checks, added new macro for that: RTE_SORING_DEBUG
>   (disabled by default).
> 
> v5 -> v6
> - fix problem with ring_stress_autotest (Phanendra)
> - added more checks and debug output
> 
> v4 -> v5
> - fix public API/doc comments from Jerin
> - update devtools/build-dict.sh (Stephen)
> - fix MSVC warnings
> - introduce new test-suite for meson (stress) with
>   ring_stress_autotest and soring_stress_autotest in it
> - enhance error report in tests
> - reorder some sync code in soring and add extra checks
>   (for better debuggability)
> 
> v3 -> v4:
> - fix compilation/doxygen complains (attempt #2)
> - updated release notes
> 
> v2 -> v3:
> - fix compilation/doxygen complains
> - dropped patch:
>   "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
>   As was mentioned in the patch desctiption it was way too big,
>   controversial and incomplete. If the community is ok to introduce
>   pipeline model into the l3fwd, then it is propbably worth to be
>   a separate patch series.
> 
> v1 -> v2:
> - rename 'elmst/objst' to 'meta' (Morten)
> - introduce new data-path APIs set: one with both meta{} and objs[],
>   second with just objs[] (Morten)
> - split data-path APIs into burst/bulk flavours (same as rte_ring)
> - added dump function for te_soring and improved dump() for rte_ring.
> - dropped patch:
>   " ring: minimize reads of the counterpart cache-line"
>   - no performance gain observed
>   - actually it does change behavior of conventional rte_ring
>     enqueue/dequeue APIs -
>     it could return available/free less then actually exist in the ring.
>     As in some other libs we reliy on that information - it will
>     introduce problems.
> 
> The main aim of these series is to extend ring library with
> new API that allows user to create/use Staged-Ordered-Ring (SORING)
> abstraction. In addition to that there are few other patches that serve
> different purposes:
> - first two patches are just code reordering to de-duplicate
>   and generalize existing rte_ring code.
> - patch #3 extends rte_ring_dump() to correctly print head/tail metadata
>   for different sync modes.
> - next two patches introduce SORING API into the ring library and
>   provide UT for it.
> 
> SORING overview
> ===============
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'. It is based on conventional DPDK
> rte_ring, re-uses many of its concepts, and even substantial part of
> its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in between.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> 
> - number of stages, size and number of objects in the ring are
>  configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation. After some objects are 'acquired' -
>   given thread can safely assume that it has exclusive ownership of
>   these objects till it will invoke 'release' for them.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important.
> 
> The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
> but the internals are different.
> In particular, SORING maintains internal array of 'states' for each element
> in the ring that is  shared by all threads/processes that access the ring.
> That allows 'release' to avoid excessive waits on the tail value and helps
> to improve performancei and scalability.
> In terms of performance, with our measurements rte_soring and
> conventional rte_ring provide nearly identical numbers.
> As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
> l3fwd (--lookup=acl) in pipeline mode [2] both
> rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
> number of worker lcores.
> 
> [1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
> [2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/
> 
> Eimear Morrissey (1):
>   ring: make dump function more verbose
> 
> Konstantin Ananyev (6):
>   test/ring: fix failure with custom number of lcores
>   ring: common functions for 'move head' ops
>   ring: make copying functions generic
>   ring/soring: introduce Staged Ordered Ring
>   app/test: add unit tests for soring API
>   test: add stress test suite
> 
>  .mailmap                                  |   1 +
>  app/test/meson.build                      |   3 +
>  app/test/suites/meson.build               |  10 +
>  app/test/test.h                           |   1 +
>  app/test/test_ring_stress.c               |   2 +-
>  app/test/test_ring_stress_impl.h          |   3 +-
>  app/test/test_soring.c                    | 442 ++++++++++++
>  app/test/test_soring_mt_stress.c          |  40 ++
>  app/test/test_soring_stress.c             |  48 ++
>  app/test/test_soring_stress.h             |  35 +
>  app/test/test_soring_stress_impl.h        | 834 ++++++++++++++++++++++
>  devtools/build-dict.sh                    |   1 +
>  doc/api/doxy-api-index.md                 |   1 +
>  doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++
>  doc/guides/prog_guide/ring_lib.rst        | 202 ++++++
>  doc/guides/rel_notes/release_24_11.rst    |   8 +
>  lib/ring/meson.build                      |   4 +-
>  lib/ring/rte_ring.c                       |  87 ++-
>  lib/ring/rte_ring.h                       |  15 +
>  lib/ring/rte_ring_c11_pvt.h               | 156 ++--
>  lib/ring/rte_ring_elem_pvt.h              | 181 +++--
>  lib/ring/rte_ring_generic_pvt.h           | 143 ++--
>  lib/ring/rte_ring_hts_elem_pvt.h          | 107 ++-
>  lib/ring/rte_ring_rts_elem_pvt.h          | 107 ++-
>  lib/ring/rte_soring.c                     | 198 +++++
>  lib/ring/rte_soring.h                     | 556 +++++++++++++++
>  lib/ring/soring.c                         | 613 ++++++++++++++++
>  lib/ring/soring.h                         | 138 ++++
>  lib/ring/version.map                      |  26 +
>  29 files changed, 4217 insertions(+), 380 deletions(-)
>  create mode 100644 app/test/test_soring.c
>  create mode 100644 app/test/test_soring_mt_stress.c
>  create mode 100644 app/test/test_soring_stress.c
>  create mode 100644 app/test/test_soring_stress.h
>  create mode 100644 app/test/test_soring_stress_impl.h
>  create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
>  create mode 100644 lib/ring/rte_soring.c
>  create mode 100644 lib/ring/rte_soring.h
>  create mode 100644 lib/ring/soring.c
>  create mode 100644 lib/ring/soring.h
> 
> --
> 2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 2/7] ring: common functions for 'move head' ops
  2024-10-30 21:22             ` [PATCH v7 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-11-07 11:31               ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-11-07 11:31 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, eimear.morrissey, stephen

Copy-paste bad! Cleaning up good! :-)

Thank you, Konstantin.

Acked-by: Morten Brørup <mb@smartsharesystems.com>

<feature creep>
I wonder if would improve anything marking the head/tail pointers ("d" and "s") in the inner functions __rte_restrict?
And perhaps the ring pointer "r" in the outer functions?
</feature creep>


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 3/7] ring: make copying functions generic
  2024-10-30 21:23             ` [PATCH v7 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-11-07 11:46               ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-11-07 11:46 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, eimear.morrissey, stephen

It should be safe adding __rte_restrict to the "ring_table" and "obj_table" pointers.

I don't know if it makes any real difference; these functions are called from applications via function pointers where the function type doesn't have __rte_restrict to the arrays. If __rte_restrict were added there, it would need to be added below (i.e. here) too.

With or without,
Acked-by: Morten Brørup <mb@smartsharesystems.com>


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 4/7] ring: make dump function more verbose
  2024-10-30 21:23             ` [PATCH v7 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-11-07 11:49               ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-11-07 11:49 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, eimear.morrissey, stephen

> From: Konstantin Ananyev [mailto:konstantin.ananyev@huawei.com]
> Sent: Wednesday, 30 October 2024 22.23
> 
> From: Eimear Morrissey <eimear.morrissey@huawei.com>
> 
> The current rte_ring_dump function uses the generic rte_ring_headtail
> structure to access head/tail positions. This is incorrect for the RTS
> case where the head is stored in a different offset in the union of
> structs. Switching to a separate function for each sync type allows
> to dump correct head/tail values and extra metadata.
> 
> Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
> ---
> +static const char *
> +ring_get_sync_type(const enum rte_ring_sync_type st)
> +{
> +	switch (st) {
> +	case RTE_RING_SYNC_ST:
> +		return "single thread";
> +	case RTE_RING_SYNC_MT:
> +		return "multi thread";
> +	case RTE_RING_SYNC_MT_RTS:
> +		return "multi thread - RTS";
> +	case RTE_RING_SYNC_MT_HTS:
> +		return "multi thread - HTS";
> +	default:
> +		return "unknown";
> +	}

I would prefer "ST", "MT" "MT_RTS" and "MT_HTS" instead of the "human readable" strings.

With or without suggested change,
Acked-by: Morten Brørup <mb@smartsharesystems.com>


^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 1/7] test/ring: fix failure with custom number of lcores
  2024-10-30 21:22             ` [PATCH v7 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-11-07 11:50               ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-11-07 11:50 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, eimear.morrissey, stephen, stable

Simple change,
Acked-by: Morten Brørup <mb@smartsharesystems.com>

^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 5/7] ring/soring: introduce Staged Ordered Ring
  2024-10-30 21:23             ` [PATCH v7 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-11-07 12:07               ` Morten Brørup
  0 siblings, 0 replies; 101+ messages in thread
From: Morten Brørup @ 2024-11-07 12:07 UTC (permalink / raw)
  To: Konstantin Ananyev, dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, eimear.morrissey, stephen

> From: Konstantin Ananyev [mailto:konstantin.ananyev@huawei.com]
> Sent: Wednesday, 30 October 2024 22.23

> +.. code-block:: c
> +
> +    /*
> +     * use pointer to mbuf as soring element, while tx_state
> +     * as a metadata.
> +     * In this example we use a soring with just one stage.
> +     */
> +     union tx_state {
> +         /* negative values for error */
> +         int32_t rc;
> +         /* otherwise contain valid TX port and queue IDs*/
> +         struct {
> +             uint16_t port_id;
> +             uint16_t queue_id;
> +          } tx;

One space too much in indentation before "} tx;".

> +     };
> +     struct rte_soring *soring;
> +
> +
> +producer/consumer part:
> +
> +.. code-block:: c
> +
> +     struct rte_mbuf *pkts[MAX_PKT_BURST];
> +     union tx_state txst[MAX_PKT_BURST];
> +     ...
> +     /* enqueue - writes to soring objects array no need to update
> metadata */
> +     uint32_t num = MAX_PKT_BURST;
> +     num = rte_soring_enqueue_burst(soring, pkts, num, NULL);
> +     ....
> +     /* dequeux - reads both packets and related tx_state */
> +     uint32_t num = MAX_PKT_BURST;
> +     num = rte_soring_dequeux_burst(soring, pkts, txst, num, NULL);
> +
> +     /*
> +      * TX packets out, or drop in case of error.
> +      * Note that we don't need to dereference the soring objects
> itself
> +      * to make a decision.
> +      */
> +     uint32_t i, j, k, n;
> +     struct rte_mbuf *dr[MAX_PKT_BURST];
> +
> +     k = 0;
> +     for (i = 0; i != num; i++) {
> +         /* packet processing reports an error */
> +         if (txst[i].rc < 0)
> +             dr[k++] = pkts[i];
> +         /* valid packet, send it out */
> +         else {
> +             /* group consequitive packets with the same port and
> queue IDs */
> +             for (j = i + 1; j < num; j++)
> +                 if (txst[j].rc != txst[i].rc)
> +                     break;
> +
> +             n = rte_eth_tx_burst(txst[i].tx.port_id,
> txst[i].tx.queue_id,
> +                             pkts + i, j - i);
> +             if (i + n != j) {
> +                 /* decide with unsent packets if any */

Indentation is off in the next few lines.

> +              }
> +         }
> +      }
> +      /* drop errorneous packets */
> +      if (k != 0)
> +          rte_pktmbuf_free_bulk(dr, k);


> +++ b/lib/ring/rte_soring.c
> @@ -0,0 +1,198 @@
> +/* SPDX-License-Identifier: BSD-3-Clause
> + * Copyright(c) 2024 Huawei Technologies Co., Ltd
> + */
> +
> +#include <inttypes.h>
> +
> +#include "soring.h"
> +#include <rte_string_fns.h>
> +
> +RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
> +
> +static uint32_t
> +soring_calc_elem_num(uint32_t count)
> +{
> +	return rte_align32pow2(count + 1);
> +}
> +
> +static int
> +soring_check_param(uint32_t esize, uint32_t stsize, uint32_t count,

stsize -> msize or meta_size
Throughout the code, where stsize is used.

> +	uint32_t stages)
> +{
> +	if (stages == 0) {
> +		SORING_LOG(ERR, "invalid number of stages: %u", stages);
> +		return -EINVAL;
> +	}
> +
> +	/* Check if element size is a multiple of 4B */
> +	if (esize == 0 || esize % 4 != 0) {
> +		SORING_LOG(ERR, "invalid element size: %u", esize);
> +		return -EINVAL;
> +	}
> +
> +	/* Check if ret-code size is a multiple of 4B */

ret-code -> meta

> +	if (stsize % 4 != 0) {
> +		SORING_LOG(ERR, "invalid retcode size: %u", stsize);

retcode -> meta

> +		return -EINVAL;
> +	}
> +
> +	 /* count must be a power of 2 */
> +	if (rte_is_power_of_2(count) == 0 ||
> +			(count > RTE_SORING_ELEM_MAX + 1)) {
> +		SORING_LOG(ERR, "invalid number of elements: %u", count);
> +		return -EINVAL;
> +	}
> +
> +	return 0;
> +}

For the next version with above fixes,
Acked-by: Morten Brørup <mb@smartsharesystems.com>


^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (7 preceding siblings ...)
  2024-11-07 10:41             ` [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-11-07 16:16             ` Stephen Hemminger
  2024-11-07 18:11               ` Konstantin Ananyev
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
  9 siblings, 1 reply; 101+ messages in thread
From: Stephen Hemminger @ 2024-11-07 16:16 UTC (permalink / raw)
  To: Konstantin Ananyev
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, eimear.morrissey

On Wed, 30 Oct 2024 17:22:57 -0400
Konstantin Ananyev <konstantin.ananyev@huawei.com> wrote:

> Testing coverage (passed):
> x86_64, i686, PPC, ARM   
> 
> Would like to express my gratitude to all community members who helped me
> with testing it on different platforms, in particular:
> David Christensen <drc@linux.ibm.com>
> Cody Cheng <ccheng@iol.unh.edu>
> Patrick Robb <probb@iol.unh.edu>
> Phanendra Vukkisala <pvukkisala@marvell.com>
> Chengwen Feng <fengchengwen@huawei.com>
> 
> v6 -> v7
> - updated Programmer Guide (Jerin, Morten, Stephen)
> - fix some functions in public headers without comments (Morten)
> - update debug checks, added new macro for that: RTE_SORING_DEBUG
>   (disabled by default).
> 
> v5 -> v6
> - fix problem with ring_stress_autotest (Phanendra)
> - added more checks and debug output
> 
> v4 -> v5
> - fix public API/doc comments from Jerin
> - update devtools/build-dict.sh (Stephen)
> - fix MSVC warnings
> - introduce new test-suite for meson (stress) with
>   ring_stress_autotest and soring_stress_autotest in it
> - enhance error report in tests
> - reorder some sync code in soring and add extra checks
>   (for better debuggability)
> 
> v3 -> v4:
> - fix compilation/doxygen complains (attempt #2)
> - updated release notes
> 
> v2 -> v3:
> - fix compilation/doxygen complains
> - dropped patch:
>   "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
>   As was mentioned in the patch desctiption it was way too big,
>   controversial and incomplete. If the community is ok to introduce
>   pipeline model into the l3fwd, then it is propbably worth to be
>   a separate patch series.
> 
> v1 -> v2:
> - rename 'elmst/objst' to 'meta' (Morten)
> - introduce new data-path APIs set: one with both meta{} and objs[],
>   second with just objs[] (Morten)
> - split data-path APIs into burst/bulk flavours (same as rte_ring)
> - added dump function for te_soring and improved dump() for rte_ring.
> - dropped patch:
>   " ring: minimize reads of the counterpart cache-line"
>   - no performance gain observed
>   - actually it does change behavior of conventional rte_ring
>     enqueue/dequeue APIs -
>     it could return available/free less then actually exist in the ring.
>     As in some other libs we reliy on that information - it will
>     introduce problems.
> 
> The main aim of these series is to extend ring library with
> new API that allows user to create/use Staged-Ordered-Ring (SORING)
> abstraction. In addition to that there are few other patches that serve
> different purposes:
> - first two patches are just code reordering to de-duplicate
>   and generalize existing rte_ring code.
> - patch #3 extends rte_ring_dump() to correctly print head/tail metadata
>   for different sync modes.
> - next two patches introduce SORING API into the ring library and
>   provide UT for it.
> 
> SORING overview
> ===============
> Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
> with multiple processing 'stages'. It is based on conventional DPDK
> rte_ring, re-uses many of its concepts, and even substantial part of
> its code.
> It can be viewed as an 'extension' of rte_ring functionality.
> In particular, main SORING properties:
> - circular ring buffer with fixed size objects
> - producer, consumer plus multiple processing stages in between.
> - allows to split objects processing into multiple stages.
> - objects remain in the same ring while moving from one stage to the other,
>   initial order is preserved, no extra copying needed.
> - preserves the ingress order of objects within the queue across multiple
>   stages
> - each stage (and producer/consumer) can be served by single and/or
>   multiple threads.
> 
> - number of stages, size and number of objects in the ring are
>  configurable at ring initialization time.
> 
> Data-path API provides four main operations:
> - enqueue/dequeue works in the same manner as for conventional rte_ring,
>   all rte_ring synchronization types are supported.
> - acquire/release - for each stage there is an acquire (start) and
>   release (finish) operation. After some objects are 'acquired' -
>   given thread can safely assume that it has exclusive ownership of
>   these objects till it will invoke 'release' for them.
>   After 'release', objects can be 'acquired' by next stage and/or dequeued
>   by the consumer (in case of last stage).
> 
> Expected use-case: applications that uses pipeline model
> (probably with multiple stages) for packet processing, when preserving
> incoming packet order is important.
> 
> The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
> but the internals are different.
> In particular, SORING maintains internal array of 'states' for each element
> in the ring that is  shared by all threads/processes that access the ring.
> That allows 'release' to avoid excessive waits on the tail value and helps
> to improve performancei and scalability.
> In terms of performance, with our measurements rte_soring and
> conventional rte_ring provide nearly identical numbers.
> As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
> l3fwd (--lookup=acl) in pipeline mode [2] both
> rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
> number of worker lcores.
> 
> [1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
> [2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

One future suggestion. What about having an example (l3fwd-soring?) so
that performance can be compared.

Assuming you get the other minor comments from Morten fixed.

Series-Acked-by: Stephen Hemminger <stephen@networkplumber.org>

^ permalink raw reply	[flat|nested] 101+ messages in thread

* RE: [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library
  2024-11-07 16:16             ` Stephen Hemminger
@ 2024-11-07 18:11               ` Konstantin Ananyev
  0 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:11 UTC (permalink / raw)
  To: Stephen Hemminger
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, Eimear Morrissey



>
> > The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
> > but the internals are different.
> > In particular, SORING maintains internal array of 'states' for each element
> > in the ring that is  shared by all threads/processes that access the ring.
> > That allows 'release' to avoid excessive waits on the tail value and helps
> > to improve performancei and scalability.
> > In terms of performance, with our measurements rte_soring and
> > conventional rte_ring provide nearly identical numbers.
> > As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
> > l3fwd (--lookup=acl) in pipeline mode [2] both
> > rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
> > number of worker lcores.
> >
> > [1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
> > [2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/
> 
> One future suggestion. What about having an example (l3fwd-soring?) so
> that performance can be compared.
> 

On early stages (RFC) I submitted a patch which allows l3fwd (ACL-case) to work
in sort of pipeline mode:
https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/
So user can run it in one of the modes: run-to-completion/eventdev/rte_ring/rte_soring
and measure performance differences.
If there is a interest from the community, then yes we can try to make it a proper patch series for future 
releases.

^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 0/7] Stage-Ordered API and other extensions for ring library
  2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
                               ` (8 preceding siblings ...)
  2024-11-07 16:16             ` Stephen Hemminger
@ 2024-11-07 18:24             ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
                                 ` (8 more replies)
  9 siblings, 9 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Testing coverage (passed):
x86_64, i686, PPC, ARM   

Would like to express my gratitude to all community members who helped
with testing it on different platforms, in particular:
David Christensen <drc@linux.ibm.com>
Cody Cheng <ccheng@iol.unh.edu>
Patrick Robb <probb@iol.unh.edu>
Phanendra Vukkisala <pvukkisala@marvell.com>
Chengwen Feng <fengchengwen@huawei.com>

v7 -> v8
- fixed several nits (spotted by Morten):
  indentation, variable names, comments, etc.

v6 -> v7
- updated Programmer Guide (Jerin, Morten, Stephen)
- fix some functions in public headers without comments (Morten)
- update debug checks, added new macro for that: RTE_SORING_DEBUG
  (disabled by default).

v5 -> v6
- fix problem with ring_stress_autotest (Phanendra)
- added more checks and debug output

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (6):
  test/ring: fix failure with custom number of lcores
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                                  |   1 +
 app/test/meson.build                      |   3 +
 app/test/suites/meson.build               |  10 +
 app/test/test.h                           |   1 +
 app/test/test_ring_stress.c               |   2 +-
 app/test/test_ring_stress_impl.h          |   3 +-
 app/test/test_soring.c                    | 442 ++++++++++++
 app/test/test_soring_mt_stress.c          |  40 ++
 app/test/test_soring_stress.c             |  48 ++
 app/test/test_soring_stress.h             |  35 +
 app/test/test_soring_stress_impl.h        | 834 ++++++++++++++++++++++
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 ++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_ring.c                       |  87 ++-
 lib/ring/rte_ring.h                       |  15 +
 lib/ring/rte_ring_c11_pvt.h               | 156 ++--
 lib/ring/rte_ring_elem_pvt.h              | 181 +++--
 lib/ring/rte_ring_generic_pvt.h           | 143 ++--
 lib/ring/rte_ring_hts_elem_pvt.h          | 107 ++-
 lib/ring/rte_ring_rts_elem_pvt.h          | 107 ++-
 lib/ring/rte_soring.c                     | 198 +++++
 lib/ring/rte_soring.h                     | 556 +++++++++++++++
 lib/ring/soring.c                         | 613 ++++++++++++++++
 lib/ring/soring.h                         | 138 ++++
 lib/ring/version.map                      |  26 +
 29 files changed, 4217 insertions(+), 380 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 1/7] test/ring: fix failure with custom number of lcores
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
                                 ` (7 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen, stable,
	Phanendra Vukkisala

ring_stress_autotest fails to initialize the ring when RTE_MAX_LCORE value
is not a number of 2.
There is a flaw in calculation required number of elements in the ring.
Fix it by aligning number of elements to next power of 2.

Fixes: bf28df24e915 ("test/ring: add contention stress test")
Cc: stable@dpdk.org

Reported-by: Phanendra Vukkisala <pvukkisala@marvell.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/test_ring_stress_impl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..ee5274aeef 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -297,7 +297,7 @@ mt1_init(struct rte_ring **rng, void **data, uint32_t num)
 	*data = elm;
 
 	/* alloc ring */
-	nr = 2 * num;
+	nr = rte_align32pow2(2 * num);
 	sz = rte_ring_get_memsize(nr);
 	r = rte_zmalloc(NULL, sz, alignof(typeof(*r)));
 	if (r == NULL) {
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 2/7] ring: common functions for 'move head' ops
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 3/7] ring: make copying functions generic Konstantin Ananyev
                                 ` (6 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 lib/ring/rte_ring_c11_pvt.h      | 156 ++++++++++---------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 143 +++++++++-------------------
 lib/ring/rte_ring_hts_elem_pvt.h | 107 ++++++++++-----------
 lib/ring/rte_ring_rts_elem_pvt.h | 107 ++++++++++-----------
 5 files changed, 255 insertions(+), 324 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..b9388af0da 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -11,6 +11,17 @@
 #ifndef _RTE_RING_C11_PVT_H_
 #define _RTE_RING_C11_PVT_H_
 
+/**
+ * @file rte_ring_c11_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -29,40 +40,45 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +89,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..affd2d5ba7 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -10,6 +10,17 @@
 #ifndef _RTE_RING_GENERIC_PVT_H_
 #define _RTE_RING_GENERIC_PVT_H_
 
+/**
+ * @file rte_ring_generic_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -30,35 +41,39 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +81,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +91,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..e2b82dd1e6 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -51,19 +51,39 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,20 +94,20 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +120,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +148,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..96825931f8 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -65,19 +65,40 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue.
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,20 +109,20 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +135,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +164,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 3/7] ring: make copying functions generic
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 4/7] ring: make dump function more verbose Konstantin Ananyev
                                 ` (5 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 4/7] ring: make dump function more verbose
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
                                 ` (2 preceding siblings ...)
  2024-11-07 18:24               ` [PATCH v8 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                                 ` (4 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 5290420258..7d6fb51ca8 100644
--- a/.mailmap
+++ b/.mailmap
@@ -388,6 +388,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index ee5274aeef..f99a7ff675 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..a7d9d6b037 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "ST";
+	case RTE_RING_SYNC_MT:
+		return "MT";
+	case RTE_RING_SYNC_MT_RTS:
+		return "MT_RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "MT_HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 5/7] ring/soring: introduce Staged Ordered Ring
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
                                 ` (3 preceding siblings ...)
  2024-11-07 18:24               ` [PATCH v8 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 6/7] app/test: add unit tests for soring API Konstantin Ananyev
                                 ` (3 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 +++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_soring.c                     | 198 +++++++
 lib/ring/rte_soring.h                     | 556 +++++++++++++++++++
 lib/ring/soring.c                         | 613 +++++++++++++++++++++
 lib/ring/soring.h                         | 138 +++++
 lib/ring/version.map                      |  19 +
 11 files changed, 2373 insertions(+), 2 deletions(-)
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index 266c8b90dc..ac7f3cd010 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -173,6 +173,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/prog_guide/img/soring-pic1.svg b/doc/guides/prog_guide/img/soring-pic1.svg
new file mode 100644
index 0000000000..c97e66ca43
--- /dev/null
+++ b/doc/guides/prog_guide/img/soring-pic1.svg
@@ -0,0 +1,635 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<!-- SPDX-License-Identifier: BSD-3-Clause -->
+
+<!-- Copyright(c) 2010 Intel Corporation -->
+
+<svg
+   width="484.05716"
+   height="385.63785"
+   id="svg3388"
+   version="1.1"
+   inkscape:version="1.2.1 (9c6d41e410, 2022-07-14)"
+   sodipodi:docname="soring-pic1.svg"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:dc="http://purl.org/dc/elements/1.1/">
+  <defs
+     id="defs3390">
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend"
+       style="overflow:visible">
+      <path
+         id="path4317"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       sodipodi:type="inkscape:persp3d"
+       inkscape:vp_x="0 : 526.18109 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_z="744.09448 : 526.18109 : 1"
+       inkscape:persp3d-origin="372.04724 : 350.78739 : 1"
+       id="perspective3396" />
+    <inkscape:perspective
+       id="perspective4180"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-6"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-0"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-3"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-06"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-5"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-69"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281-2"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4767"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-7"
+       style="overflow:visible">
+      <path
+         id="path4317-4"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective4799"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4824"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4915"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4937"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4962"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4993"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-0"
+       style="overflow:visible">
+      <path
+         id="path4317-6"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker4999"
+       style="overflow:visible">
+      <path
+         id="path5001"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5091"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-9"
+       style="overflow:visible">
+      <path
+         id="path4317-0"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5121"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-1"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-9"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5710"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-6"
+       style="overflow:visible">
+      <path
+         id="path4317-7"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5738"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="1"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="232.14286"
+     inkscape:cy="168.21429"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     inkscape:window-width="1483"
+     inkscape:window-height="1002"
+     inkscape:window-x="83"
+     inkscape:window-y="16"
+     inkscape:window-maximized="0"
+     inkscape:snap-grids="false"
+     inkscape:snap-to-guides="true"
+     showguides="false"
+     fit-margin-top="0.1"
+     fit-margin-left="0.1"
+     fit-margin-right="0.1"
+     fit-margin-bottom="0.1"
+     inkscape:showpageshadow="2"
+     inkscape:pagecheckerboard="0"
+     inkscape:deskcolor="#d1d1d1">
+    <inkscape:grid
+       type="xygrid"
+       id="grid5162"
+       empspacing="5"
+       visible="true"
+       enabled="true"
+       snapvisiblegridlinesonly="true"
+       originx="-162.97143"
+       originy="-368.03525"
+       spacingx="1"
+       spacingy="1" />
+  </sodipodi:namedview>
+  <metadata
+     id="metadata3393">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-162.97143,-298.68909)">
+    <rect
+       style="fill:#ffd080;fill-opacity:1;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+       id="rect4257"
+       width="439.41635"
+       height="58.588848"
+       x="186.87822"
+       y="463.44324"
+       rx="11.631636"
+       ry="11.631636" />
+    <g
+       id="g4259"
+       transform="translate(108.51492,3.9469318)">
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="83.143028"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="137.00014"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-3"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="190.85725"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-1"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="244.71437"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-6"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="298.57147"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-2"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="352.42859"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-15"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="406.28571"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-4"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="460.14282"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-65"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+    </g>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="528.41443"
+       y="496.36957"
+       id="text4269-5-6-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5-0"
+         x="528.41443"
+         y="496.36957"
+         style="font-size:14px;line-height:1.25">obj5</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="313.90488"
+       y="495.49646"
+       id="text4269"><tspan
+         sodipodi:role="line"
+         id="tspan4271"
+         x="313.90488"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj1</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="368.95203"
+       y="495.49646"
+       id="text4269-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-5"
+         x="368.95203"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj2</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="422.99518"
+       y="495.49646"
+       id="text4269-5"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4"
+         x="422.99518"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj3</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 323.57143,578.07647 0,-42.14286"
+       id="path4309"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="289.85715"
+       y="589.505"
+       id="text4787"><tspan
+         sodipodi:role="line"
+         id="tspan4789"
+         x="289.85715"
+         y="589.505"
+         style="font-size:14px;line-height:1.25">cons_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="293.45334"
+       y="603.41034"
+       id="text4787-3"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0"
+         x="293.45334"
+         y="603.41034"
+         style="font-size:14px;line-height:1.25">cons_tail</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="561.2981"
+       y="602.81482"
+       id="text4787-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8"
+         x="561.2981"
+         y="602.81482"
+         style="font-size:14px;line-height:1.25">prod_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="566.75146"
+       y="587.72028"
+       id="text4787-3-6"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0-8"
+         x="566.75146"
+         y="587.72028"
+         style="font-size:14px;line-height:1.25">prod_tail</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="315.21933"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="172.28571"
+       y="308.93362"
+       id="text4891"><tspan
+         sodipodi:role="line"
+         id="tspan4893"
+         x="172.28571"
+         y="308.93362"
+         style="font-size:14px;line-height:1.25">lstages states</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889-8"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="529.93365"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="170.89287"
+       y="684.09021"
+       id="text4891-4"><tspan
+         sodipodi:role="line"
+         id="tspan4893-3"
+         x="170.89287"
+         y="684.09021"
+         style="font-size:14px;line-height:1.25">producer and consumer states</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 379.25296,407.43361 v 42.14286"
+       id="path4309-8"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 486.68152,407.43361 0,42.14286"
+       id="path4309-4-9"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;white-space:pre;inline-size:99.1521;display:inline;fill:#000000;fill-opacity:1;stroke:none"
+       x="296.992"
+       y="399.48123"
+       id="text4787-3-64"
+       transform="translate(54)"><tspan
+         x="296.992"
+         y="399.48123"
+         id="tspan1631"><tspan
+           style="font-size:14px;line-height:1.25"
+           id="tspan1629">stage[1].tail</tspan></tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="440.26532"
+       y="399.48123"
+       id="text4787-7-5"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0"
+         x="440.26532"
+         y="399.48123"
+         style="font-size:14px;line-height:1.25">stage[1].head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="441.55243"
+       y="377.24719"
+       id="text4787-7-5-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-4"
+         x="441.55243"
+         y="377.24719"
+         style="font-size:14px;line-height:1.25">stage[0].tail</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 593.14285,407.43361 v 42.14286"
+       id="path4309-4-9-9"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="M 594,575.57647 V 533.43361"
+       id="path4309-4-3"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="476.46902"
+       y="495.12097"
+       id="text4269-5-6"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5"
+         x="476.46902"
+         y="495.12097"
+         style="font-size:14px;line-height:1.25">obj4</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="552.98102"
+       y="397.24719"
+       id="text4787-7-5-74"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-8"
+         x="552.98102"
+         y="397.24719"
+         style="font-size:14px;line-height:1.25">stage[0].head</tspan></text>
+  </g>
+</svg>
diff --git a/doc/guides/prog_guide/ring_lib.rst b/doc/guides/prog_guide/ring_lib.rst
index f7dbba0e4e..dbe66993a5 100644
--- a/doc/guides/prog_guide/ring_lib.rst
+++ b/doc/guides/prog_guide/ring_lib.rst
@@ -491,6 +491,208 @@ Following is an example of usage:
 Note that between ``_start_`` and ``_finish_`` no other thread can proceed
 with enqueue(/dequeue) operation till ``_finish_`` completes.
 
+Staged Ordered Ring API
+-----------------------
+
+Staged-Ordered-Ring (SORING) API provides a SW abstraction for *ordered* queues
+with multiple processing *stages*. It is based on conventional DPDK
+``rte_ring`` API, re-uses many of its concepts, and even substantial part of
+its code. It can be viewed as an 'extension' of ``rte_ring`` functionality.
+In particular, main SORING properties:
+
+*   circular ring buffer with fixed size objects and related metadata
+
+*   producer, consumer plus multiple processing stages in between.
+
+*   allows to split objects processing into multiple stages.
+
+*   objects remain in the same ring while moving from one stage to the other,
+    initial order is preserved, no extra copying needed.
+
+*   preserves the ingress order of objects within the queue across multiple
+    stages
+
+*   each stage (and producer/consumer) can be served by single and/or
+    multiple threads.
+
+*   number of stages, size and number of objects and their metadata in the
+    ring are configurable at ring initialization time.
+
+Data-Path API
+~~~~~~~~~~~~~
+
+SORING data-path API provided four main operations:
+
+*   ``enqueue``/``dequeue`` works in the same manner as for conventional
+    ``rte_ring``, all rte_ring synchronization types are supported.
+
+*   ``acquire``/``release`` - for each stage there is an ``acquire`` (start)
+    and ``release`` (finish) operation.
+    After some objects are ``acquired`` - given thread can safely assume that
+    it has exclusive possession of these objects till ``release`` for them is
+    invoked.
+    Note that right now user has to release exactly the same number of
+    objects that was acquired before.
+    After objects are ``released``, given thread loses its possession on them,
+    and they can be either acquired by next stage or dequeued
+    by the consumer (in case of last stage).
+
+A simplified representation of a SORING with two stages is shown below.
+On that picture ``obj5`` and ``obj4`` elements are acquired by stage 0,
+``obj2`` and ``obj3`` are acquired by stage 1, while ``obj11`` was already
+released by stage 1 and is ready to be consumed.
+
+.. _figure_soring1:
+
+.. figure:: img/soring-pic1.*
+
+Along with traditional flavor there are enhanced versions for all these
+data-path operations: ``enqueux``/``dequeux``/``acquirx``/``releasx``.
+All enhanced versions take as extra parameter a pointer to an array of
+metadata values.
+At initialization user can request within the ``soring`` supplementary and
+optional array of metadata associated with each object in the ``soring``.
+While ``soring`` element size is configurable and user can specify it big
+enough to hold both object and its metadata together,
+for performance reasons it might be plausible to access them as separate arrays.
+Note that users are free to mix and match both versions of data-path API in
+a way they like.
+As an example, possible usage scenario when such separation helps:
+
+.. code-block:: c
+
+    /*
+     * use pointer to mbuf as soring element, while tx_state
+     * as a metadata.
+     * In this example we use a soring with just one stage.
+     */
+     union tx_state {
+         /* negative values for error */
+         int32_t rc;
+         /* otherwise contain valid TX port and queue IDs*/
+         struct {
+             uint16_t port_id;
+             uint16_t queue_id;
+         } tx;
+     };
+     struct rte_soring *soring;
+
+
+producer/consumer part:
+
+.. code-block:: c
+
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* enqueue - writes to soring objects array no need to update metadata */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_enqueue_burst(soring, pkts, num, NULL);
+     ....
+     /* dequeux - reads both packets and related tx_state */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_dequeux_burst(soring, pkts, txst, num, NULL);
+
+     /*
+      * TX packets out, or drop in case of error.
+      * Note that we don't need to dereference the soring objects itself
+      * to make a decision.
+      */
+     uint32_t i, j, k, n;
+     struct rte_mbuf *dr[MAX_PKT_BURST];
+
+     k = 0;
+     for (i = 0; i != num; i++) {
+         /* packet processing reports an error */
+         if (txst[i].rc < 0)
+             dr[k++] = pkts[i];
+         /* valid packet, send it out */
+         else {
+             /* group consequitive packets with the same port and queue IDs */
+             for (j = i + 1; j < num; j++)
+                 if (txst[j].rc != txst[i].rc)
+                     break;
+
+             n = rte_eth_tx_burst(txst[i].tx.port_id, txst[i].tx.queue_id,
+                             pkts + i, j - i);
+             if (i + n != j) {
+                 /* decide with unsent packets if any */
+             }
+         }
+     }
+     /* drop errorneous packets */
+     if (k != 0)
+         rte_pktmbuf_free_bulk(dr, k);
+
+acquire/release part:
+
+.. code-block:: c
+
+     uint32_t ftoken;
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* acquire - grab some packets to process */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_acquire_burst(soring, pkts, 0, num, &ftoken, NULL);
+
+     /* process packets, fill txst[] for each */
+     do_process_packets(pkts, txst, num);
+
+     /*
+      * release - assuming that do_process_packets() didn't change
+      * contents of pkts[], we need to update soring metadata array only.
+      */
+     rte_soring_releasx(soring, NULL, txst, 0, num, ftoken);
+
+Use Cases
+~~~~~~~~~~
+
+Expected use-cases include applications that use pipeline model
+(probably with multiple stages) for packet processing, when preserving
+incoming packet order is important. I.E.: IPsec processing, etc.
+
+SORING internals
+~~~~~~~~~~~~~~~~
+
+*   In addition to accessible by the user array of objects (and metadata),
+    ``soirng`` also contains an internal array of states. Each ``state[]``
+    corresponds to exactly one object within the soring. That ``state[]``
+    array is used by ``acquire``/``release``/``dequeue`` operations to
+    store internal information and should not be accessed by the user directly.
+
+*   At ``acquire``, soring  moves stage's head (in a same way as ``rte_ring``
+    ``move_head`` does), plus it saves in ``state[stage.old_head]``
+    information about how many elements were acquired, acquired head position,
+    and special flag value to indicate that given elements are acquired
+    (``SORING_ST_START``).
+    Note that ``acquire`` returns an opaque ``ftoken`` value that user has
+    to provide for ``release`` function.
+
+*   ``release`` extracts old head value from provided by user ``ftoken`` and
+    checks that corresponding ``state[]`` entry contains expected values
+    (mostly for sanity purposes). Then it marks this ``state[]`` entry with
+    ``SORING_ST_FINISH`` flag to indicate that given subset of objects was
+    released. After that, it checks does stage's old ``head`` value equals to
+    its current ``tail`` value. If so, then it performs ``finalize``
+    operation, otherwise ``release`` just returns.
+
+*   As ``state[]`` is shared by all threads, some other thread can perform
+    ``finalize`` operation for given stage. That allows ``release`` to avoid
+    excessive waits on the ``tail`` value.
+    Main purpose of ``finalize`` operation is to walk through ``state[]``
+    array  from current stage's ``tail`` position up to its ``head``,
+    check ``state[]`` and move stage ``tail`` through elements that already
+    are released (in ``SORING_ST_FINISH`` state).
+    Along with that, corresponding ``state[]`` entries are reset back to zero.
+    Note that ``finalize`` for given stage can be called from multiple places:
+    from ``release`` for that stage or from ``acquire`` for next stage, or
+    even from consumer's ``dequeue`` - in case given stage is the last one.
+    So ``finalize`` has to be MT-safe and inside it we have to guarantee that
+    at any given moment only one thread can update stage's ``tail`` and reset
+    corresponding ``state[]`` entries.
+
+
 References
 ----------
 
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index 53a5ffebe5..acaf63e3f3 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -251,6 +251,14 @@ New Features
   Added ability for node to advertise and update multiple xstat counters,
   that can be retrieved using ``rte_graph_cluster_stats_get``.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..c0eae680e7
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,198 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t msize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if size of metadata is a multiple of 4B */
+	if (msize % 4 != 0) {
+		SORING_LOG(ERR, "invalid metadata size: %u", msize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t msize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * msize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+/* compilation-time checks */
+static void
+soring_compilation_checks(void)
+{
+	RTE_BUILD_BUG_ON((sizeof(struct rte_soring) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, cons) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, prod) &
+			RTE_CACHE_LINE_MASK) != 0);
+
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, tail) !=
+		offsetof(struct soring_stage_headtail, tail.pos));
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, sync_type) !=
+		offsetof(struct soring_stage_headtail, unused));
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	soring_compilation_checks();
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..ec1ee813ed
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,556 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of RTE soring (Staged Ordered Ring) public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Extra debugging might be enabled with RTE_SORING_DEBUG macro.
+ */
+
+#include <rte_ring.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elemnts in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided paramer values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and metadata tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation paramers.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become avaialble for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to relase.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..e8fe890597
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,613 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head
+ * plus current stage index).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht, uint32_t stage,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t ftkn, head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/* Ensure the head is read before rstate[] */
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		ftkn = SORING_FTKN_MAKE(tail, stage);
+
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & SORING_ST_MASK) != SORING_ST_FINISH ||
+				st.ftoken != ftkn)
+			break;
+
+		k = st.stnum & ~SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acq_rel,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht, ns,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+/*
+ * Verify internal SORING state.
+ * WARNING: if expected value is not equal to actual one, it means that for
+ * whatever reason SORING data constancy is broken. That is a very serious
+ * problem that most likely will cause race-conditions, memory corruption,
+ * program crash.
+ * To ease debugging it user might rebuild ring library with
+ * RTE_SORING_DEBUG enabled.
+ */
+static __rte_always_inline void
+soring_verify_state(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	const char *msg, union soring_state val,  union soring_state exp)
+{
+	if (val.raw != exp.raw) {
+#ifdef RTE_SORING_DEBUG
+		rte_soring_dump(stderr, r);
+		rte_panic("line:%d from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};\n",
+			__LINE__, msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#else
+		SORING_LOG(EMERG, "from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};",
+			msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#endif
+	}
+}
+
+/* check and update state ring at acquire op*/
+static __rte_always_inline void
+acquire_state_update(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	uint32_t ftoken, uint32_t num)
+{
+	union soring_state st;
+	const union soring_state est = {.raw = 0};
+
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	st.ftoken = ftoken;
+	st.stnum = (SORING_ST_START | num);
+
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, stage - 1,
+				r->state, r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+		*ftoken = SORING_FTKN_MAKE(head, stage);
+
+		/* check and update state value */
+		acquire_state_update(r, stage, idx, *ftoken, n);
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, pos, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	const union soring_state est = {
+		.stnum = (SORING_ST_START | n),
+		.ftoken = ftoken,
+	};
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	pos = SORING_FTKN_POS(ftoken, stage);
+	idx = pos & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == pos)
+		__rte_soring_stage_finalize(&stg->sht, stage, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..b252ae4a61
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,138 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal strctures of RTE soring: Staged Ordered Ring.
+ * Sort of extension of conventional RTE ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+/* logging stuff, register our own tag for SORING */
+#include <rte_log.h>
+
+extern int soring_logtype;
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+/**
+ * SORING internal state for each element
+ */
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define SORING_ST_START		RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+#define SORING_FTKN_MAKE(pos, stg)	((pos) + (stg))
+#define SORING_FTKN_POS(ftk, stg)	((ftk) - (stg))
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * RTE soring internal structure.
+ * As with rte_ring actual elements array supposed to be located direclty
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 6/7] app/test: add unit tests for soring API
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
                                 ` (4 preceding siblings ...)
  2024-11-07 18:24               ` [PATCH v8 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-07 18:24               ` [PATCH v8 7/7] test: add stress test suite Konstantin Ananyev
                                 ` (2 subsequent siblings)
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 834 +++++++++++++++++++++++++++++
 6 files changed, 1402 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index 0f7e11969a..e59f27fd51 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -177,6 +177,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..b2110305a7
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calcuated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at relase */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..df4ca9564c
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,834 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) != WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (__atomic_load_n(&wrk_cmd, __ATOMIC_RELAXED) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_RUN, __ATOMIC_RELEASE);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	__atomic_store_n(&wrk_cmd, WRK_CMD_STOP, __ATOMIC_RELEASE);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v8 7/7] test: add stress test suite
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
                                 ` (5 preceding siblings ...)
  2024-11-07 18:24               ` [PATCH v8 6/7] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-11-07 18:24               ` Konstantin Ananyev
  2024-11-08 21:56               ` [PATCH v8 0/7] Stage-Ordered API and other extensions for ring library Thomas Monjalon
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
  8 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-07 18:24 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add a new test suite which purpose is to run 'stress' tests:
main purpose is put a pressure to dpdk sync algorithms
to flag their misbehaving/slowdown/etc.
Right now it consists from just 2 test-cases:
meson test --suite stress-tests --list
DPDK:stress-tests / ring_stress_autotest
DPDK:stress-tests / soring_stress_autotest

These tests are quite time consuming (~15 mins each),
that's another reason to put them into a separate test-suite.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/suites/meson.build   | 10 ++++++++++
 app/test/test.h               |  1 +
 app/test/test_ring_stress.c   |  2 +-
 app/test/test_soring_stress.c |  2 +-
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/app/test/suites/meson.build b/app/test/suites/meson.build
index 191702cf76..e482373330 100644
--- a/app/test/suites/meson.build
+++ b/app/test/suites/meson.build
@@ -5,6 +5,7 @@
 # to complete, so timeout to 10 minutes
 timeout_seconds = 600
 timeout_seconds_fast = 10
+timeout_seconds_stress = 900
 
 test_no_huge_args = ['--no-huge', '-m', '2048']
 has_hugepage = run_command(has_hugepages_cmd, check: true).stdout().strip() != '0'
@@ -21,6 +22,7 @@ endif
 # - fast_tests
 # - perf_tests
 # - driver_tests
+# - stress_tests
 test_suites = run_command(get_test_suites_cmd, autotest_sources,
          check: true).stdout().strip().split()
 foreach suite:test_suites
@@ -39,6 +41,14 @@ foreach suite:test_suites
                     timeout: timeout_seconds,
                     is_parallel: false)
         endforeach
+    elif suite_name == 'stress-tests'
+        foreach t: suite_tests
+            test(t, dpdk_test,
+                    env: ['DPDK_TEST=' + t],
+                    timeout: timeout_seconds_stress,
+                    is_parallel: false,
+                    suite: suite_name)
+        endforeach
     elif suite_name != 'fast-tests'
         # simple cases - tests without parameters or special handling
         foreach t: suite_tests
diff --git a/app/test/test.h b/app/test/test.h
index 15e23d297f..ebc4864bf8 100644
--- a/app/test/test.h
+++ b/app/test/test.h
@@ -208,5 +208,6 @@ void add_test_command(struct test_command *t);
 #define REGISTER_FAST_TEST(cmd, no_huge, ASan, func)  REGISTER_TEST_COMMAND(cmd, func)
 #define REGISTER_PERF_TEST REGISTER_TEST_COMMAND
 #define REGISTER_DRIVER_TEST REGISTER_TEST_COMMAND
+#define REGISTER_STRESS_TEST REGISTER_TEST_COMMAND
 
 #endif
diff --git a/app/test/test_ring_stress.c b/app/test/test_ring_stress.c
index 1af45e0fc8..82e19b02c3 100644
--- a/app/test/test_ring_stress.c
+++ b/app/test/test_ring_stress.c
@@ -63,4 +63,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(ring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(ring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
index 334af6a29c..e5655d49cb 100644
--- a/app/test/test_soring_stress.c
+++ b/app/test/test_soring_stress.c
@@ -45,4 +45,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(soring_stress_autotest, test_ring_stress);
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* Re: [PATCH v8 0/7] Stage-Ordered API and other extensions for ring library
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
                                 ` (6 preceding siblings ...)
  2024-11-07 18:24               ` [PATCH v8 7/7] test: add stress test suite Konstantin Ananyev
@ 2024-11-08 21:56               ` Thomas Monjalon
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
  8 siblings, 0 replies; 101+ messages in thread
From: Thomas Monjalon @ 2024-11-08 21:56 UTC (permalink / raw)
  To: Konstantin Ananyev
  Cc: dev, honnappa.nagarahalli, jerinj, hemant.agrawal,
	bruce.richardson, drc, ruifeng.wang, mb, eimear.morrissey,
	stephen, David Marchand

> Konstantin Ananyev (6):
>   test/ring: fix failure with custom number of lcores
>   ring: common functions for 'move head' ops
>   ring: make copying functions generic
>   ring/soring: introduce Staged Ordered Ring
>   app/test: add unit tests for soring API
>   test: add stress test suite

This looks too risky to change something in the ring library
at this stage of the release cycle.
Sorry we got too many issues with other features merged recently
to allow more big changes in 24.11.

For next version, please could you fixe those typos in comments?
	avaialble -> available
	calcuated -> calculated
	direclty -> directly
	elemnts -> elements
	errorneous -> erroneous
	paramer -> parameter
	relase -> release
	strctures -> structures

Instead of "RTE soring", please write "DPDK soring",
because RTE is nothing more than a namespace prefix.

One more thing:
Warning in app/test/test_soring_stress_impl.h:
Using __atomic_xxx/__ATOMIC_XXX built-ins, prefer rte_atomic_xxx/rte_memory_order_xxx

Thanks



^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 0/7] Stage-Ordered API and other extensions for ring library
  2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
                                 ` (7 preceding siblings ...)
  2024-11-08 21:56               ` [PATCH v8 0/7] Stage-Ordered API and other extensions for ring library Thomas Monjalon
@ 2024-11-11 12:25               ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
                                   ` (7 more replies)
  8 siblings, 8 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Testing coverage (passed):
x86_64, i686, PPC, ARM   

Would like to express my gratitude to all community members who helped
with testing it on different platforms, in particular:
David Christensen <drc@linux.ibm.com>
Cody Cheng <ccheng@iol.unh.edu>
Patrick Robb <probb@iol.unh.edu>
Phanendra Vukkisala <pvukkisala@marvell.com>
Chengwen Feng <fengchengwen@huawei.com>

v8 -> v9
- fix grammar (Thomas)
- test: use stdatomic API (Thomas)

v7 -> v8
- fixed several nits (spotted by Morten):
  indentation, variable names, comments, etc.

v6 -> v7
- updated Programmer Guide (Jerin, Morten, Stephen)
- fix some functions in public headers without comments (Morten)
- update debug checks, added new macro for that: RTE_SORING_DEBUG
  (disabled by default).

v5 -> v6
- fix problem with ring_stress_autotest (Phanendra)
- added more checks and debug output

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/


Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (6):
  test/ring: fix failure with custom number of lcores
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                                  |   1 +
 app/test/meson.build                      |   3 +
 app/test/suites/meson.build               |  10 +
 app/test/test.h                           |   1 +
 app/test/test_ring_stress.c               |   2 +-
 app/test/test_ring_stress_impl.h          |   3 +-
 app/test/test_soring.c                    | 442 ++++++++++++
 app/test/test_soring_mt_stress.c          |  40 ++
 app/test/test_soring_stress.c             |  48 ++
 app/test/test_soring_stress.h             |  35 +
 app/test/test_soring_stress_impl.h        | 838 ++++++++++++++++++++++
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 ++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_ring.c                       |  87 ++-
 lib/ring/rte_ring.h                       |  15 +
 lib/ring/rte_ring_c11_pvt.h               | 156 ++--
 lib/ring/rte_ring_elem_pvt.h              | 181 +++--
 lib/ring/rte_ring_generic_pvt.h           | 143 ++--
 lib/ring/rte_ring_hts_elem_pvt.h          | 107 ++-
 lib/ring/rte_ring_rts_elem_pvt.h          | 107 ++-
 lib/ring/rte_soring.c                     | 198 +++++
 lib/ring/rte_soring.h                     | 557 ++++++++++++++
 lib/ring/soring.c                         | 613 ++++++++++++++++
 lib/ring/soring.h                         | 138 ++++
 lib/ring/version.map                      |  26 +
 29 files changed, 4222 insertions(+), 380 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 1/7] test/ring: fix failure with custom number of lcores
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
                                   ` (6 subsequent siblings)
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen, stable,
	Phanendra Vukkisala

ring_stress_autotest fails to initialize the ring when RTE_MAX_LCORE value
is not a number of 2.
There is a flaw in calculation required number of elements in the ring.
Fix it by aligning number of elements to next power of 2.

Fixes: bf28df24e915 ("test/ring: add contention stress test")
Cc: stable@dpdk.org

Reported-by: Phanendra Vukkisala <pvukkisala@marvell.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/test_ring_stress_impl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..ee5274aeef 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -297,7 +297,7 @@ mt1_init(struct rte_ring **rng, void **data, uint32_t num)
 	*data = elm;
 
 	/* alloc ring */
-	nr = 2 * num;
+	nr = rte_align32pow2(2 * num);
 	sz = rte_ring_get_memsize(nr);
 	r = rte_zmalloc(NULL, sz, alignof(typeof(*r)));
 	if (r == NULL) {
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 2/7] ring: common functions for 'move head' ops
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 3/7] ring: make copying functions generic Konstantin Ananyev
                                   ` (5 subsequent siblings)
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 lib/ring/rte_ring_c11_pvt.h      | 156 ++++++++++---------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 143 +++++++++-------------------
 lib/ring/rte_ring_hts_elem_pvt.h | 107 ++++++++++-----------
 lib/ring/rte_ring_rts_elem_pvt.h | 107 ++++++++++-----------
 5 files changed, 255 insertions(+), 324 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..b9388af0da 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -11,6 +11,17 @@
 #ifndef _RTE_RING_C11_PVT_H_
 #define _RTE_RING_C11_PVT_H_
 
+/**
+ * @file rte_ring_c11_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -29,40 +40,45 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +89,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..affd2d5ba7 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -10,6 +10,17 @@
 #ifndef _RTE_RING_GENERIC_PVT_H_
 #define _RTE_RING_GENERIC_PVT_H_
 
+/**
+ * @file rte_ring_generic_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -30,35 +41,39 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +81,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +91,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..e2b82dd1e6 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -51,19 +51,39 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,20 +94,20 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +120,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +148,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..96825931f8 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -65,19 +65,40 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue.
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,20 +109,20 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +135,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +164,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 3/7] ring: make copying functions generic
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 4/7] ring: make dump function more verbose Konstantin Ananyev
                                   ` (4 subsequent siblings)
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 4/7] ring: make dump function more verbose
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
                                   ` (2 preceding siblings ...)
  2024-11-11 12:25                 ` [PATCH v9 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                                   ` (3 subsequent siblings)
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 49734b899f..a22d28989c 100644
--- a/.mailmap
+++ b/.mailmap
@@ -388,6 +388,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index ee5274aeef..f99a7ff675 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..a7d9d6b037 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "ST";
+	case RTE_RING_SYNC_MT:
+		return "MT";
+	case RTE_RING_SYNC_MT_RTS:
+		return "MT_RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "MT_HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 5/7] ring/soring: introduce Staged Ordered Ring
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
                                   ` (3 preceding siblings ...)
  2024-11-11 12:25                 ` [PATCH v9 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 6/7] app/test: add unit tests for soring API Konstantin Ananyev
                                   ` (2 subsequent siblings)
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 +++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_soring.c                     | 198 +++++++
 lib/ring/rte_soring.h                     | 557 +++++++++++++++++++
 lib/ring/soring.c                         | 613 +++++++++++++++++++++
 lib/ring/soring.h                         | 138 +++++
 lib/ring/version.map                      |  19 +
 11 files changed, 2374 insertions(+), 2 deletions(-)
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index 266c8b90dc..ac7f3cd010 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -173,6 +173,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/prog_guide/img/soring-pic1.svg b/doc/guides/prog_guide/img/soring-pic1.svg
new file mode 100644
index 0000000000..c97e66ca43
--- /dev/null
+++ b/doc/guides/prog_guide/img/soring-pic1.svg
@@ -0,0 +1,635 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<!-- SPDX-License-Identifier: BSD-3-Clause -->
+
+<!-- Copyright(c) 2010 Intel Corporation -->
+
+<svg
+   width="484.05716"
+   height="385.63785"
+   id="svg3388"
+   version="1.1"
+   inkscape:version="1.2.1 (9c6d41e410, 2022-07-14)"
+   sodipodi:docname="soring-pic1.svg"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:dc="http://purl.org/dc/elements/1.1/">
+  <defs
+     id="defs3390">
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend"
+       style="overflow:visible">
+      <path
+         id="path4317"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       sodipodi:type="inkscape:persp3d"
+       inkscape:vp_x="0 : 526.18109 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_z="744.09448 : 526.18109 : 1"
+       inkscape:persp3d-origin="372.04724 : 350.78739 : 1"
+       id="perspective3396" />
+    <inkscape:perspective
+       id="perspective4180"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-6"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-0"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-3"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-06"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-5"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-69"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281-2"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4767"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-7"
+       style="overflow:visible">
+      <path
+         id="path4317-4"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective4799"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4824"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4915"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4937"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4962"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4993"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-0"
+       style="overflow:visible">
+      <path
+         id="path4317-6"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker4999"
+       style="overflow:visible">
+      <path
+         id="path5001"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5091"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-9"
+       style="overflow:visible">
+      <path
+         id="path4317-0"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5121"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-1"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-9"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5710"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-6"
+       style="overflow:visible">
+      <path
+         id="path4317-7"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5738"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="1"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="232.14286"
+     inkscape:cy="168.21429"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     inkscape:window-width="1483"
+     inkscape:window-height="1002"
+     inkscape:window-x="83"
+     inkscape:window-y="16"
+     inkscape:window-maximized="0"
+     inkscape:snap-grids="false"
+     inkscape:snap-to-guides="true"
+     showguides="false"
+     fit-margin-top="0.1"
+     fit-margin-left="0.1"
+     fit-margin-right="0.1"
+     fit-margin-bottom="0.1"
+     inkscape:showpageshadow="2"
+     inkscape:pagecheckerboard="0"
+     inkscape:deskcolor="#d1d1d1">
+    <inkscape:grid
+       type="xygrid"
+       id="grid5162"
+       empspacing="5"
+       visible="true"
+       enabled="true"
+       snapvisiblegridlinesonly="true"
+       originx="-162.97143"
+       originy="-368.03525"
+       spacingx="1"
+       spacingy="1" />
+  </sodipodi:namedview>
+  <metadata
+     id="metadata3393">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-162.97143,-298.68909)">
+    <rect
+       style="fill:#ffd080;fill-opacity:1;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+       id="rect4257"
+       width="439.41635"
+       height="58.588848"
+       x="186.87822"
+       y="463.44324"
+       rx="11.631636"
+       ry="11.631636" />
+    <g
+       id="g4259"
+       transform="translate(108.51492,3.9469318)">
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="83.143028"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="137.00014"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-3"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="190.85725"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-1"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="244.71437"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-6"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="298.57147"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-2"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="352.42859"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-15"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="406.28571"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-4"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="460.14282"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-65"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+    </g>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="528.41443"
+       y="496.36957"
+       id="text4269-5-6-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5-0"
+         x="528.41443"
+         y="496.36957"
+         style="font-size:14px;line-height:1.25">obj5</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="313.90488"
+       y="495.49646"
+       id="text4269"><tspan
+         sodipodi:role="line"
+         id="tspan4271"
+         x="313.90488"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj1</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="368.95203"
+       y="495.49646"
+       id="text4269-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-5"
+         x="368.95203"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj2</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="422.99518"
+       y="495.49646"
+       id="text4269-5"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4"
+         x="422.99518"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj3</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 323.57143,578.07647 0,-42.14286"
+       id="path4309"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="289.85715"
+       y="589.505"
+       id="text4787"><tspan
+         sodipodi:role="line"
+         id="tspan4789"
+         x="289.85715"
+         y="589.505"
+         style="font-size:14px;line-height:1.25">cons_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="293.45334"
+       y="603.41034"
+       id="text4787-3"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0"
+         x="293.45334"
+         y="603.41034"
+         style="font-size:14px;line-height:1.25">cons_tail</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="561.2981"
+       y="602.81482"
+       id="text4787-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8"
+         x="561.2981"
+         y="602.81482"
+         style="font-size:14px;line-height:1.25">prod_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="566.75146"
+       y="587.72028"
+       id="text4787-3-6"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0-8"
+         x="566.75146"
+         y="587.72028"
+         style="font-size:14px;line-height:1.25">prod_tail</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="315.21933"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="172.28571"
+       y="308.93362"
+       id="text4891"><tspan
+         sodipodi:role="line"
+         id="tspan4893"
+         x="172.28571"
+         y="308.93362"
+         style="font-size:14px;line-height:1.25">lstages states</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889-8"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="529.93365"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="170.89287"
+       y="684.09021"
+       id="text4891-4"><tspan
+         sodipodi:role="line"
+         id="tspan4893-3"
+         x="170.89287"
+         y="684.09021"
+         style="font-size:14px;line-height:1.25">producer and consumer states</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 379.25296,407.43361 v 42.14286"
+       id="path4309-8"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 486.68152,407.43361 0,42.14286"
+       id="path4309-4-9"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;white-space:pre;inline-size:99.1521;display:inline;fill:#000000;fill-opacity:1;stroke:none"
+       x="296.992"
+       y="399.48123"
+       id="text4787-3-64"
+       transform="translate(54)"><tspan
+         x="296.992"
+         y="399.48123"
+         id="tspan1631"><tspan
+           style="font-size:14px;line-height:1.25"
+           id="tspan1629">stage[1].tail</tspan></tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="440.26532"
+       y="399.48123"
+       id="text4787-7-5"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0"
+         x="440.26532"
+         y="399.48123"
+         style="font-size:14px;line-height:1.25">stage[1].head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="441.55243"
+       y="377.24719"
+       id="text4787-7-5-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-4"
+         x="441.55243"
+         y="377.24719"
+         style="font-size:14px;line-height:1.25">stage[0].tail</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 593.14285,407.43361 v 42.14286"
+       id="path4309-4-9-9"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="M 594,575.57647 V 533.43361"
+       id="path4309-4-3"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="476.46902"
+       y="495.12097"
+       id="text4269-5-6"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5"
+         x="476.46902"
+         y="495.12097"
+         style="font-size:14px;line-height:1.25">obj4</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="552.98102"
+       y="397.24719"
+       id="text4787-7-5-74"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-8"
+         x="552.98102"
+         y="397.24719"
+         style="font-size:14px;line-height:1.25">stage[0].head</tspan></text>
+  </g>
+</svg>
diff --git a/doc/guides/prog_guide/ring_lib.rst b/doc/guides/prog_guide/ring_lib.rst
index f7dbba0e4e..70a454972e 100644
--- a/doc/guides/prog_guide/ring_lib.rst
+++ b/doc/guides/prog_guide/ring_lib.rst
@@ -491,6 +491,208 @@ Following is an example of usage:
 Note that between ``_start_`` and ``_finish_`` no other thread can proceed
 with enqueue(/dequeue) operation till ``_finish_`` completes.
 
+Staged Ordered Ring API
+-----------------------
+
+Staged-Ordered-Ring (SORING) API provides a SW abstraction for *ordered* queues
+with multiple processing *stages*. It is based on conventional DPDK
+``rte_ring`` API, re-uses many of its concepts, and even substantial part of
+its code. It can be viewed as an 'extension' of ``rte_ring`` functionality.
+In particular, main SORING properties:
+
+*   circular ring buffer with fixed size objects and related metadata
+
+*   producer, consumer plus multiple processing stages in between.
+
+*   allows to split objects processing into multiple stages.
+
+*   objects remain in the same ring while moving from one stage to the other,
+    initial order is preserved, no extra copying needed.
+
+*   preserves the ingress order of objects within the queue across multiple
+    stages
+
+*   each stage (and producer/consumer) can be served by single and/or
+    multiple threads.
+
+*   number of stages, size and number of objects and their metadata in the
+    ring are configurable at ring initialization time.
+
+Data-Path API
+~~~~~~~~~~~~~
+
+SORING data-path API provided four main operations:
+
+*   ``enqueue``/``dequeue`` works in the same manner as for conventional
+    ``rte_ring``, all rte_ring synchronization types are supported.
+
+*   ``acquire``/``release`` - for each stage there is an ``acquire`` (start)
+    and ``release`` (finish) operation.
+    After some objects are ``acquired`` - given thread can safely assume that
+    it has exclusive possession of these objects till ``release`` for them is
+    invoked.
+    Note that right now user has to release exactly the same number of
+    objects that was acquired before.
+    After objects are ``released``, given thread loses its possession on them,
+    and they can be either acquired by next stage or dequeued
+    by the consumer (in case of last stage).
+
+A simplified representation of a SORING with two stages is shown below.
+On that picture ``obj5`` and ``obj4`` elements are acquired by stage 0,
+``obj2`` and ``obj3`` are acquired by stage 1, while ``obj11`` was already
+released by stage 1 and is ready to be consumed.
+
+.. _figure_soring1:
+
+.. figure:: img/soring-pic1.*
+
+Along with traditional flavor there are enhanced versions for all these
+data-path operations: ``enqueux``/``dequeux``/``acquirx``/``releasx``.
+All enhanced versions take as extra parameter a pointer to an array of
+metadata values.
+At initialization user can request within the ``soring`` supplementary and
+optional array of metadata associated with each object in the ``soring``.
+While ``soring`` element size is configurable and user can specify it big
+enough to hold both object and its metadata together,
+for performance reasons it might be plausible to access them as separate arrays.
+Note that users are free to mix and match both versions of data-path API in
+a way they like.
+As an example, possible usage scenario when such separation helps:
+
+.. code-block:: c
+
+    /*
+     * use pointer to mbuf as soring element, while tx_state
+     * as a metadata.
+     * In this example we use a soring with just one stage.
+     */
+     union tx_state {
+         /* negative values for error */
+         int32_t rc;
+         /* otherwise contain valid TX port and queue IDs*/
+         struct {
+             uint16_t port_id;
+             uint16_t queue_id;
+         } tx;
+     };
+     struct rte_soring *soring;
+
+
+producer/consumer part:
+
+.. code-block:: c
+
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* enqueue - writes to soring objects array no need to update metadata */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_enqueue_burst(soring, pkts, num, NULL);
+     ....
+     /* dequeux - reads both packets and related tx_state */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_dequeux_burst(soring, pkts, txst, num, NULL);
+
+     /*
+      * TX packets out, or drop in case of error.
+      * Note that we don't need to dereference the soring objects itself
+      * to make a decision.
+      */
+     uint32_t i, j, k, n;
+     struct rte_mbuf *dr[MAX_PKT_BURST];
+
+     k = 0;
+     for (i = 0; i != num; i++) {
+         /* packet processing reports an error */
+         if (txst[i].rc < 0)
+             dr[k++] = pkts[i];
+         /* valid packet, send it out */
+         else {
+             /* group consequitive packets with the same port and queue IDs */
+             for (j = i + 1; j < num; j++)
+                 if (txst[j].rc != txst[i].rc)
+                     break;
+
+             n = rte_eth_tx_burst(txst[i].tx.port_id, txst[i].tx.queue_id,
+                             pkts + i, j - i);
+             if (i + n != j) {
+                 /* decide with unsent packets if any */
+             }
+         }
+     }
+     /* drop erroneous packets */
+     if (k != 0)
+         rte_pktmbuf_free_bulk(dr, k);
+
+acquire/release part:
+
+.. code-block:: c
+
+     uint32_t ftoken;
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* acquire - grab some packets to process */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_acquire_burst(soring, pkts, 0, num, &ftoken, NULL);
+
+     /* process packets, fill txst[] for each */
+     do_process_packets(pkts, txst, num);
+
+     /*
+      * release - assuming that do_process_packets() didn't change
+      * contents of pkts[], we need to update soring metadata array only.
+      */
+     rte_soring_releasx(soring, NULL, txst, 0, num, ftoken);
+
+Use Cases
+~~~~~~~~~~
+
+Expected use-cases include applications that use pipeline model
+(probably with multiple stages) for packet processing, when preserving
+incoming packet order is important. I.E.: IPsec processing, etc.
+
+SORING internals
+~~~~~~~~~~~~~~~~
+
+*   In addition to accessible by the user array of objects (and metadata),
+    ``soirng`` also contains an internal array of states. Each ``state[]``
+    corresponds to exactly one object within the soring. That ``state[]``
+    array is used by ``acquire``/``release``/``dequeue`` operations to
+    store internal information and should not be accessed by the user directly.
+
+*   At ``acquire``, soring  moves stage's head (in a same way as ``rte_ring``
+    ``move_head`` does), plus it saves in ``state[stage.old_head]``
+    information about how many elements were acquired, acquired head position,
+    and special flag value to indicate that given elements are acquired
+    (``SORING_ST_START``).
+    Note that ``acquire`` returns an opaque ``ftoken`` value that user has
+    to provide for ``release`` function.
+
+*   ``release`` extracts old head value from provided by user ``ftoken`` and
+    checks that corresponding ``state[]`` entry contains expected values
+    (mostly for sanity purposes). Then it marks this ``state[]`` entry with
+    ``SORING_ST_FINISH`` flag to indicate that given subset of objects was
+    released. After that, it checks does stage's old ``head`` value equals to
+    its current ``tail`` value. If so, then it performs ``finalize``
+    operation, otherwise ``release`` just returns.
+
+*   As ``state[]`` is shared by all threads, some other thread can perform
+    ``finalize`` operation for given stage. That allows ``release`` to avoid
+    excessive waits on the ``tail`` value.
+    Main purpose of ``finalize`` operation is to walk through ``state[]``
+    array  from current stage's ``tail`` position up to its ``head``,
+    check ``state[]`` and move stage ``tail`` through elements that already
+    are released (in ``SORING_ST_FINISH`` state).
+    Along with that, corresponding ``state[]`` entries are reset back to zero.
+    Note that ``finalize`` for given stage can be called from multiple places:
+    from ``release`` for that stage or from ``acquire`` for next stage, or
+    even from consumer's ``dequeue`` - in case given stage is the last one.
+    So ``finalize`` has to be MT-safe and inside it we have to guarantee that
+    at any given moment only one thread can update stage's ``tail`` and reset
+    corresponding ``state[]`` entries.
+
+
 References
 ----------
 
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index 9dc739c4cb..4a94a6c038 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -256,6 +256,14 @@ New Features
   Added ability for node to advertise and update multiple xstat counters,
   that can be retrieved using ``rte_graph_cluster_stats_get``.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..c0eae680e7
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,198 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t msize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if size of metadata is a multiple of 4B */
+	if (msize % 4 != 0) {
+		SORING_LOG(ERR, "invalid metadata size: %u", msize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t msize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * msize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+/* compilation-time checks */
+static void
+soring_compilation_checks(void)
+{
+	RTE_BUILD_BUG_ON((sizeof(struct rte_soring) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, cons) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, prod) &
+			RTE_CACHE_LINE_MASK) != 0);
+
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, tail) !=
+		offsetof(struct soring_stage_headtail, tail.pos));
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, sync_type) !=
+		offsetof(struct soring_stage_headtail, unused));
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	soring_compilation_checks();
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..1c3a798e0f
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,557 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of DPDK soring (Staged Ordered Ring)
+ * public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Extra debugging might be enabled with RTE_SORING_DEBUG macro.
+ */
+
+#include <rte_ring.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elements in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation parameters.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided parameter values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and metadata tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation parameters.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become available for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to release.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become available for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to release.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..e8fe890597
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,613 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head
+ * plus current stage index).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht, uint32_t stage,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t ftkn, head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/* Ensure the head is read before rstate[] */
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		ftkn = SORING_FTKN_MAKE(tail, stage);
+
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & SORING_ST_MASK) != SORING_ST_FINISH ||
+				st.ftoken != ftkn)
+			break;
+
+		k = st.stnum & ~SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acq_rel,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht, ns,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+/*
+ * Verify internal SORING state.
+ * WARNING: if expected value is not equal to actual one, it means that for
+ * whatever reason SORING data constancy is broken. That is a very serious
+ * problem that most likely will cause race-conditions, memory corruption,
+ * program crash.
+ * To ease debugging it user might rebuild ring library with
+ * RTE_SORING_DEBUG enabled.
+ */
+static __rte_always_inline void
+soring_verify_state(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	const char *msg, union soring_state val,  union soring_state exp)
+{
+	if (val.raw != exp.raw) {
+#ifdef RTE_SORING_DEBUG
+		rte_soring_dump(stderr, r);
+		rte_panic("line:%d from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};\n",
+			__LINE__, msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#else
+		SORING_LOG(EMERG, "from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};",
+			msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#endif
+	}
+}
+
+/* check and update state ring at acquire op*/
+static __rte_always_inline void
+acquire_state_update(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	uint32_t ftoken, uint32_t num)
+{
+	union soring_state st;
+	const union soring_state est = {.raw = 0};
+
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	st.ftoken = ftoken;
+	st.stnum = (SORING_ST_START | num);
+
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, stage - 1,
+				r->state, r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+		*ftoken = SORING_FTKN_MAKE(head, stage);
+
+		/* check and update state value */
+		acquire_state_update(r, stage, idx, *ftoken, n);
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, pos, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	const union soring_state est = {
+		.stnum = (SORING_ST_START | n),
+		.ftoken = ftoken,
+	};
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	pos = SORING_FTKN_POS(ftoken, stage);
+	idx = pos & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == pos)
+		__rte_soring_stage_finalize(&stg->sht, stage, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..455cf677a7
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,138 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal structures of DPDK soring: Staged Ordered Ring.
+ * Sort of extension of conventional DPDK ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+/* logging stuff, register our own tag for SORING */
+#include <rte_log.h>
+
+extern int soring_logtype;
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+/**
+ * SORING internal state for each element
+ */
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define SORING_ST_START		RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+#define SORING_FTKN_MAKE(pos, stg)	((pos) + (stg))
+#define SORING_FTKN_POS(ftk, stg)	((ftk) - (stg))
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * soring internal structure.
+ * As with rte_ring actual elements array supposed to be located directly
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 6/7] app/test: add unit tests for soring API
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
                                   ` (4 preceding siblings ...)
  2024-11-11 12:25                 ` [PATCH v9 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 12:25                 ` [PATCH v9 7/7] test: add stress test suite Konstantin Ananyev
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 838 +++++++++++++++++++++++++++++
 6 files changed, 1406 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index 0f7e11969a..e59f27fd51 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -177,6 +177,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..6773a399e6
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at release */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at release */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..99df1838f5
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,838 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static uint32_t wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (rte_atomic_load_explicit(&wrk_cmd, rte_memory_order_relaxed) !=
+			WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (rte_atomic_load_explicit(&wrk_cmd,
+				rte_memory_order_relaxed) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	rte_atomic_store_explicit(&wrk_cmd, WRK_CMD_RUN,
+			rte_memory_order_release);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	rte_atomic_store_explicit(&wrk_cmd, WRK_CMD_STOP,
+			rte_memory_order_release);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v9 7/7] test: add stress test suite
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
                                   ` (5 preceding siblings ...)
  2024-11-11 12:25                 ` [PATCH v9 6/7] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-11-11 12:25                 ` Konstantin Ananyev
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  7 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 12:25 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add a new test suite which purpose is to run 'stress' tests:
main purpose is put a pressure to dpdk sync algorithms
to flag their misbehaving/slowdown/etc.
Right now it consists from just 2 test-cases:
meson test --suite stress-tests --list
DPDK:stress-tests / ring_stress_autotest
DPDK:stress-tests / soring_stress_autotest

These tests are quite time consuming (~15 mins each),
that's another reason to put them into a separate test-suite.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/suites/meson.build   | 10 ++++++++++
 app/test/test.h               |  1 +
 app/test/test_ring_stress.c   |  2 +-
 app/test/test_soring_stress.c |  2 +-
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/app/test/suites/meson.build b/app/test/suites/meson.build
index 191702cf76..e482373330 100644
--- a/app/test/suites/meson.build
+++ b/app/test/suites/meson.build
@@ -5,6 +5,7 @@
 # to complete, so timeout to 10 minutes
 timeout_seconds = 600
 timeout_seconds_fast = 10
+timeout_seconds_stress = 900
 
 test_no_huge_args = ['--no-huge', '-m', '2048']
 has_hugepage = run_command(has_hugepages_cmd, check: true).stdout().strip() != '0'
@@ -21,6 +22,7 @@ endif
 # - fast_tests
 # - perf_tests
 # - driver_tests
+# - stress_tests
 test_suites = run_command(get_test_suites_cmd, autotest_sources,
          check: true).stdout().strip().split()
 foreach suite:test_suites
@@ -39,6 +41,14 @@ foreach suite:test_suites
                     timeout: timeout_seconds,
                     is_parallel: false)
         endforeach
+    elif suite_name == 'stress-tests'
+        foreach t: suite_tests
+            test(t, dpdk_test,
+                    env: ['DPDK_TEST=' + t],
+                    timeout: timeout_seconds_stress,
+                    is_parallel: false,
+                    suite: suite_name)
+        endforeach
     elif suite_name != 'fast-tests'
         # simple cases - tests without parameters or special handling
         foreach t: suite_tests
diff --git a/app/test/test.h b/app/test/test.h
index 15e23d297f..ebc4864bf8 100644
--- a/app/test/test.h
+++ b/app/test/test.h
@@ -208,5 +208,6 @@ void add_test_command(struct test_command *t);
 #define REGISTER_FAST_TEST(cmd, no_huge, ASan, func)  REGISTER_TEST_COMMAND(cmd, func)
 #define REGISTER_PERF_TEST REGISTER_TEST_COMMAND
 #define REGISTER_DRIVER_TEST REGISTER_TEST_COMMAND
+#define REGISTER_STRESS_TEST REGISTER_TEST_COMMAND
 
 #endif
diff --git a/app/test/test_ring_stress.c b/app/test/test_ring_stress.c
index 1af45e0fc8..82e19b02c3 100644
--- a/app/test/test_ring_stress.c
+++ b/app/test/test_ring_stress.c
@@ -63,4 +63,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(ring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(ring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
index 334af6a29c..e5655d49cb 100644
--- a/app/test/test_soring_stress.c
+++ b/app/test/test_soring_stress.c
@@ -45,4 +45,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(soring_stress_autotest, test_ring_stress);
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library
  2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
                                   ` (6 preceding siblings ...)
  2024-11-11 12:25                 ` [PATCH v9 7/7] test: add stress test suite Konstantin Ananyev
@ 2024-11-11 14:19                 ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
                                     ` (6 more replies)
  7 siblings, 7 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Testing coverage (passed):
x86_64, i686, PPC, ARM   

Would like to express my gratitude to all community members who helped
with testing it on different platforms, in particular:
David Christensen <drc@linux.ibm.com>
Cody Cheng <ccheng@iol.unh.edu>
Patrick Robb <probb@iol.unh.edu>
Phanendra Vukkisala <pvukkisala@marvell.com>
Chengwen Feng <fengchengwen@huawei.com>

v9 - v10
- fix clang14 complaints regarding stdatomic (app/test)

v8 -> v9
- fix grammar (Thomas)
- test: use stdatomic API (Thomas)

v7 -> v8
- fixed several nits (spotted by Morten):
  indentation, variable names, comments, etc.

v6 -> v7
- updated Programmer Guide (Jerin, Morten, Stephen)
- fix some functions in public headers without comments (Morten)
- update debug checks, added new macro for that: RTE_SORING_DEBUG
  (disabled by default).

v5 -> v6
- fix problem with ring_stress_autotest (Phanendra)
- added more checks and debug output

v4 -> v5
- fix public API/doc comments from Jerin
- update devtools/build-dict.sh (Stephen)
- fix MSVC warnings
- introduce new test-suite for meson (stress) with
  ring_stress_autotest and soring_stress_autotest in it
- enhance error report in tests
- reorder some sync code in soring and add extra checks
  (for better debuggability)

v3 -> v4:
- fix compilation/doxygen complains (attempt #2)
- updated release notes

v2 -> v3:
- fix compilation/doxygen complains
- dropped patch:
  "examples/l3fwd: make ACL work in pipeline and eventdev modes": [2]
  As was mentioned in the patch desctiption it was way too big,
  controversial and incomplete. If the community is ok to introduce
  pipeline model into the l3fwd, then it is propbably worth to be
  a separate patch series.

v1 -> v2:
- rename 'elmst/objst' to 'meta' (Morten)
- introduce new data-path APIs set: one with both meta{} and objs[],
  second with just objs[] (Morten)
- split data-path APIs into burst/bulk flavours (same as rte_ring)
- added dump function for te_soring and improved dump() for rte_ring.
- dropped patch:
  " ring: minimize reads of the counterpart cache-line"
  - no performance gain observed
  - actually it does change behavior of conventional rte_ring
    enqueue/dequeue APIs -
    it could return available/free less then actually exist in the ring.
    As in some other libs we reliy on that information - it will
    introduce problems.

The main aim of these series is to extend ring library with
new API that allows user to create/use Staged-Ordered-Ring (SORING)
abstraction. In addition to that there are few other patches that serve
different purposes:
- first two patches are just code reordering to de-duplicate
  and generalize existing rte_ring code.
- patch #3 extends rte_ring_dump() to correctly print head/tail metadata
  for different sync modes.
- next two patches introduce SORING API into the ring library and
  provide UT for it.

SORING overview
===============
Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'. It is based on conventional DPDK
rte_ring, re-uses many of its concepts, and even substantial part of
its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in between.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.

- number of stages, size and number of objects in the ring are
 configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation. After some objects are 'acquired' -
  given thread can safely assume that it has exclusive ownership of
  these objects till it will invoke 'release' for them.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important.

The concept of ‘ring with stages’ is similar to DPDK OPDL eventdev PMD [1],
but the internals are different.
In particular, SORING maintains internal array of 'states' for each element
in the ring that is  shared by all threads/processes that access the ring.
That allows 'release' to avoid excessive waits on the tail value and helps
to improve performancei and scalability.
In terms of performance, with our measurements rte_soring and
conventional rte_ring provide nearly identical numbers.
As an example, on our SUT: Intel ICX CPU @ 2.00GHz,
l3fwd (--lookup=acl) in pipeline mode [2] both
rte_ring and rte_soring reach ~20Mpps for single I/O lcore and same
number of worker lcores.

[1] https://www.dpdk.org/wp-content/uploads/sites/35/2018/06/DPDK-China2017-Ma-OPDL.pdf
[2] https://patchwork.dpdk.org/project/dpdk/patch/20240906131348.804-7-konstantin.v.ananyev@yandex.ru/

Eimear Morrissey (1):
  ring: make dump function more verbose

Konstantin Ananyev (6):
  test/ring: fix failure with custom number of lcores
  ring: common functions for 'move head' ops
  ring: make copying functions generic
  ring/soring: introduce Staged Ordered Ring
  app/test: add unit tests for soring API
  test: add stress test suite

 .mailmap                                  |   1 +
 app/test/meson.build                      |   3 +
 app/test/suites/meson.build               |  10 +
 app/test/test.h                           |   1 +
 app/test/test_ring_stress.c               |   2 +-
 app/test/test_ring_stress_impl.h          |   3 +-
 app/test/test_soring.c                    | 442 ++++++++++++
 app/test/test_soring_mt_stress.c          |  40 ++
 app/test/test_soring_stress.c             |  48 ++
 app/test/test_soring_stress.h             |  35 +
 app/test/test_soring_stress_impl.h        | 838 ++++++++++++++++++++++
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 ++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_ring.c                       |  87 ++-
 lib/ring/rte_ring.h                       |  15 +
 lib/ring/rte_ring_c11_pvt.h               | 156 ++--
 lib/ring/rte_ring_elem_pvt.h              | 181 +++--
 lib/ring/rte_ring_generic_pvt.h           | 143 ++--
 lib/ring/rte_ring_hts_elem_pvt.h          | 107 ++-
 lib/ring/rte_ring_rts_elem_pvt.h          | 107 ++-
 lib/ring/rte_soring.c                     | 198 +++++
 lib/ring/rte_soring.h                     | 557 ++++++++++++++
 lib/ring/soring.c                         | 613 ++++++++++++++++
 lib/ring/soring.h                         | 138 ++++
 lib/ring/version.map                      |  26 +
 29 files changed, 4222 insertions(+), 380 deletions(-)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 1/7] test/ring: fix failure with custom number of lcores
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
                                     ` (5 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen, stable,
	Phanendra Vukkisala

ring_stress_autotest fails to initialize the ring when RTE_MAX_LCORE value
is not a number of 2.
There is a flaw in calculation required number of elements in the ring.
Fix it by aligning number of elements to next power of 2.

Fixes: bf28df24e915 ("test/ring: add contention stress test")
Cc: stable@dpdk.org

Reported-by: Phanendra Vukkisala <pvukkisala@marvell.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/test_ring_stress_impl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index 8b0bfb11fe..ee5274aeef 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -297,7 +297,7 @@ mt1_init(struct rte_ring **rng, void **data, uint32_t num)
 	*data = elm;
 
 	/* alloc ring */
-	nr = 2 * num;
+	nr = rte_align32pow2(2 * num);
 	sz = rte_ring_get_memsize(nr);
 	r = rte_zmalloc(NULL, sz, alignof(typeof(*r)));
 	if (r == NULL) {
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 2/7] ring: common functions for 'move head' ops
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 3/7] ring: make copying functions generic Konstantin Ananyev
                                     ` (4 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional or
performance changes.
It is just a code-reordering for:
 - code deduplication
 - ability in future to re-use the same code to introduce new functionality

For each sync mode corresponding move_prod_head() and
move_cons_head() are nearly identical to each other,
the only differences are:
 - do we need to use a @capacity to calculate number of entries or not.
 - what we need to update (prod/cons) and what is used as
   read-only counterpart.
So instead of having 2 copies of nearly identical functions,
introduce a new common one that could be used by both functions:
move_prod_head() and move_cons_head().

As another positive thing - we can get rid of referencing whole rte_ring
structure in that new common sub-function.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 lib/ring/rte_ring_c11_pvt.h      | 156 ++++++++++---------------------
 lib/ring/rte_ring_elem_pvt.h     |  66 +++++++++++++
 lib/ring/rte_ring_generic_pvt.h  | 143 +++++++++-------------------
 lib/ring/rte_ring_hts_elem_pvt.h | 107 ++++++++++-----------
 lib/ring/rte_ring_rts_elem_pvt.h | 107 ++++++++++-----------
 5 files changed, 255 insertions(+), 324 deletions(-)

diff --git a/lib/ring/rte_ring_c11_pvt.h b/lib/ring/rte_ring_c11_pvt.h
index 629b2d9288..b9388af0da 100644
--- a/lib/ring/rte_ring_c11_pvt.h
+++ b/lib/ring/rte_ring_c11_pvt.h
@@ -11,6 +11,17 @@
 #ifndef _RTE_RING_C11_PVT_H_
 #define _RTE_RING_C11_PVT_H_
 
+/**
+ * @file rte_ring_c11_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -29,40 +40,45 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
-	uint32_t cons_tail;
-	unsigned int max = n;
+	uint32_t stail;
 	int success;
+	unsigned int max = n;
 
-	*old_head = rte_atomic_load_explicit(&r->prod.head, rte_memory_order_relaxed);
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
 	do {
 		/* Reset n to the initial burst count */
 		n = max;
@@ -73,112 +89,36 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* load-acquire synchronize with store-release of ht->tail
 		 * in update_tail.
 		 */
-		cons_tail = rte_atomic_load_explicit(&r->cons.tail,
+		stail = rte_atomic_load_explicit(&s->tail,
 					rte_memory_order_acquire);
 
 		/* The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + cons_tail - *old_head);
+		*entries = (capacity + stail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
 			/* on failure, *old_head is updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->prod.head,
-					old_head, *new_head,
+			success = rte_atomic_compare_exchange_strong_explicit(
+					&d->head, old_head, *new_head,
 					rte_memory_order_relaxed,
 					rte_memory_order_relaxed);
 	} while (unlikely(success == 0));
 	return n;
 }
 
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	uint32_t prod_tail;
-	int success;
-
-	/* move cons.head atomically */
-	*old_head = rte_atomic_load_explicit(&r->cons.head, rte_memory_order_relaxed);
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		/* Ensure the head is read before tail */
-		rte_atomic_thread_fence(rte_memory_order_acquire);
-
-		/* this load-acquire synchronize with store-release of ht->tail
-		 * in update_tail.
-		 */
-		prod_tail = rte_atomic_load_explicit(&r->prod.tail,
-					rte_memory_order_acquire);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (prod_tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			success = 1;
-		} else
-			/* on failure, *old_head will be updated */
-			success = rte_atomic_compare_exchange_strong_explicit(&r->cons.head,
-							old_head, *new_head,
-							rte_memory_order_relaxed,
-							rte_memory_order_relaxed);
-	} while (unlikely(success == 0));
-	return n;
-}
-
 #endif /* _RTE_RING_C11_PVT_H_ */
diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 4b80f58980..3a83668a08 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -293,6 +293,72 @@ __rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
 #include "rte_ring_generic_pvt.h"
 #endif
 
+/**
+ * @internal This function updates the producer head for enqueue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sp
+ *   Indicates whether multi-producer path is needed or not
+ * @param n
+ *   The number of elements we will want to enqueue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where enqueue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where enqueue finishes
+ * @param free_entries
+ *   Returns the amount of free space in the ring BEFORE head was moved
+ * @return
+ *   Actual number of objects enqueued.
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *free_entries)
+{
+	return __rte_ring_headtail_move_head(&r->prod, &r->cons, r->capacity,
+			is_sp, n, behavior, old_head, new_head, free_entries);
+}
+
+/**
+ * @internal This function updates the consumer head for dequeue
+ *
+ * @param r
+ *   A pointer to the ring structure
+ * @param is_sc
+ *   Indicates whether multi-consumer path is needed or not
+ * @param n
+ *   The number of elements we will want to dequeue, i.e. how far should the
+ *   head be moved
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
+ *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
+ * @param old_head
+ *   Returns head value as it was before the move, i.e. where dequeue starts
+ * @param new_head
+ *   Returns the current/new head value i.e. where dequeue finishes
+ * @param entries
+ *   Returns the number of entries in the ring BEFORE head was moved
+ * @return
+ *   - Actual number of objects dequeued.
+ *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ */
+static __rte_always_inline unsigned int
+__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
+		unsigned int n, enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head,
+		uint32_t *entries)
+{
+	return __rte_ring_headtail_move_head(&r->cons, &r->prod, 0,
+			is_sc, n, behavior, old_head, new_head, entries);
+}
+
 /**
  * @internal Enqueue several objects on the ring
  *
diff --git a/lib/ring/rte_ring_generic_pvt.h b/lib/ring/rte_ring_generic_pvt.h
index 457f41dab3..affd2d5ba7 100644
--- a/lib/ring/rte_ring_generic_pvt.h
+++ b/lib/ring/rte_ring_generic_pvt.h
@@ -10,6 +10,17 @@
 #ifndef _RTE_RING_GENERIC_PVT_H_
 #define _RTE_RING_GENERIC_PVT_H_
 
+/**
+ * @file rte_ring_generic_pvt.h
+ * It is not recommended to include this file directly,
+ * include <rte_ring.h> instead.
+ * Contains internal helper functions for MP/SP and MC/SC ring modes.
+ * For more information please refer to <rte_ring.h>.
+ */
+
+/**
+ * @internal This function updates tail values.
+ */
 static __rte_always_inline void
 __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 		uint32_t new_val, uint32_t single, uint32_t enqueue)
@@ -30,35 +41,39 @@ __rte_ring_update_tail(struct rte_ring_headtail *ht, uint32_t old_val,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
  *
- * @param r
- *   A pointer to the ring structure
- * @param is_sp
- *   Indicates whether multi-producer path is needed or not
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ * @param is_st
+ *   Indicates whether multi-thread safe path is needed or not
  * @param n
- *   The number of elements we will want to enqueue, i.e. how far should the
- *   head be moved
+ *   The number of elements we want to move head value on
  * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Enqueue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Enqueue as many items as possible from ring
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
  * @param old_head
- *   Returns head value as it was before the move, i.e. where enqueue starts
+ *   Returns head value as it was before the move
  * @param new_head
- *   Returns the current/new head value i.e. where enqueue finishes
- * @param free_entries
- *   Returns the amount of free space in the ring BEFORE head was moved
+ *   Returns the new head value
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
  * @return
- *   Actual number of objects enqueued.
- *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline unsigned int
-__rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *free_entries)
+__rte_ring_headtail_move_head(struct rte_ring_headtail *d,
+		const struct rte_ring_headtail *s, uint32_t capacity,
+		unsigned int is_st, unsigned int n,
+		enum rte_ring_queue_behavior behavior,
+		uint32_t *old_head, uint32_t *new_head, uint32_t *entries)
 {
-	const uint32_t capacity = r->capacity;
 	unsigned int max = n;
 	int success;
 
@@ -66,7 +81,7 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/* Reset n to the initial burst count */
 		n = max;
 
-		*old_head = r->prod.head;
+		*old_head = d->head;
 
 		/* add rmb barrier to avoid load/load reorder in weak
 		 * memory model. It is noop on x86
@@ -76,97 +91,27 @@ __rte_ring_move_prod_head(struct rte_ring *r, unsigned int is_sp,
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > s->tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = (capacity + r->cons.tail - *old_head);
+		*entries = (capacity + s->tail - *old_head);
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			return 0;
 
 		*new_head = *old_head + n;
-		if (is_sp) {
-			r->prod.head = *new_head;
+		if (is_st) {
+			d->head = *new_head;
 			success = 1;
 		} else
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->prod.head,
-					*old_head, *new_head);
-	} while (unlikely(success == 0));
-	return n;
-}
-
-/**
- * @internal This function updates the consumer head for dequeue
- *
- * @param r
- *   A pointer to the ring structure
- * @param is_sc
- *   Indicates whether multi-consumer path is needed or not
- * @param n
- *   The number of elements we will want to dequeue, i.e. how far should the
- *   head be moved
- * @param behavior
- *   RTE_RING_QUEUE_FIXED:    Dequeue a fixed number of items from a ring
- *   RTE_RING_QUEUE_VARIABLE: Dequeue as many items as possible from ring
- * @param old_head
- *   Returns head value as it was before the move, i.e. where dequeue starts
- * @param new_head
- *   Returns the current/new head value i.e. where dequeue finishes
- * @param entries
- *   Returns the number of entries in the ring BEFORE head was moved
- * @return
- *   - Actual number of objects dequeued.
- *     If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only.
- */
-static __rte_always_inline unsigned int
-__rte_ring_move_cons_head(struct rte_ring *r, unsigned int is_sc,
-		unsigned int n, enum rte_ring_queue_behavior behavior,
-		uint32_t *old_head, uint32_t *new_head,
-		uint32_t *entries)
-{
-	unsigned int max = n;
-	int success;
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = max;
-
-		*old_head = r->cons.head;
-
-		/* add rmb barrier to avoid load/load reorder in weak
-		 * memory model. It is noop on x86
-		 */
-		rte_smp_rmb();
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = (r->prod.tail - *old_head);
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			return 0;
-
-		*new_head = *old_head + n;
-		if (is_sc) {
-			r->cons.head = *new_head;
-			rte_smp_rmb();
-			success = 1;
-		} else {
-			success = rte_atomic32_cmpset((uint32_t *)(uintptr_t)&r->cons.head,
+			success = rte_atomic32_cmpset(
+					(uint32_t *)(uintptr_t)&d->head,
 					*old_head, *new_head);
-		}
 	} while (unlikely(success == 0));
 	return n;
 }
diff --git a/lib/ring/rte_ring_hts_elem_pvt.h b/lib/ring/rte_ring_hts_elem_pvt.h
index 91f5eeccb9..e2b82dd1e6 100644
--- a/lib/ring/rte_ring_hts_elem_pvt.h
+++ b/lib/ring/rte_ring_hts_elem_pvt.h
@@ -51,19 +51,39 @@ __rte_ring_hts_head_wait(const struct rte_ring_hts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
-static __rte_always_inline unsigned int
-__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+static __rte_always_inline uint32_t
+__rte_ring_hts_move_head(struct rte_ring_hts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_hts_pos np, op;
 
-	const uint32_t capacity = r->capacity;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_prod.ht.raw, rte_memory_order_acquire);
+	op.raw = rte_atomic_load_explicit(&d->ht.raw, rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -74,20 +94,20 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 		 * make sure that we read prod head/tail *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_hts_head_wait(&r->hts_prod, &op);
+		__rte_ring_hts_head_wait(d, &op);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - op.pos.head;
+		*entries = capacity + s->tail - op.pos.head;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -100,13 +120,25 @@ __rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems from the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_prod.ht.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->ht.raw,
 			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = op.pos.head;
 	return n;
 }
+/**
+ * @internal This function updates the producer head for enqueue
+ */
+static __rte_always_inline unsigned int
+__rte_ring_hts_move_prod_head(struct rte_ring *r, unsigned int num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_hts_move_head(&r->hts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
 
 /**
  * @internal This function updates the consumer head for dequeue
@@ -116,51 +148,8 @@ __rte_ring_hts_move_cons_head(struct rte_ring *r, unsigned int num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_hts_pos np, op;
-
-	op.raw = rte_atomic_load_explicit(&r->hts_cons.ht.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for tail to be equal to head,
-		 * make sure that we read cons head/tail *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_hts_head_wait(&r->hts_cons, &op);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - op.pos.head;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		np.pos.tail = op.pos.tail;
-		np.pos.head = op.pos.head + n;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->hts_cons.ht.raw,
-			(uint64_t *)(uintptr_t)&op.raw, np.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = op.pos.head;
-	return n;
+	return __rte_ring_hts_move_head(&r->hts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
diff --git a/lib/ring/rte_ring_rts_elem_pvt.h b/lib/ring/rte_ring_rts_elem_pvt.h
index 122650346b..96825931f8 100644
--- a/lib/ring/rte_ring_rts_elem_pvt.h
+++ b/lib/ring/rte_ring_rts_elem_pvt.h
@@ -65,19 +65,40 @@ __rte_ring_rts_head_wait(const struct rte_ring_rts_headtail *ht,
 }
 
 /**
- * @internal This function updates the producer head for enqueue.
+ * @internal This is a helper function that moves the producer/consumer head
+ *
+ * @param d
+ *   A pointer to the headtail structure with head value to be moved
+ * @param s
+ *   A pointer to the counter-part headtail structure. Note that this
+ *   function only reads tail value from it
+ * @param capacity
+ *   Either ring capacity value (for producer), or zero (for consumer)
+ *   Indicates whether multi-thread safe path is needed or not
+ * @param num
+ *   The number of elements we want to move head value on
+ * @param behavior
+ *   RTE_RING_QUEUE_FIXED:    Move on a fixed number of items
+ *   RTE_RING_QUEUE_VARIABLE: Move on as many items as possible
+ * @param old_head
+ *   Returns head value as it was before the move
+ * @param entries
+ *   Returns the number of ring entries available BEFORE head was moved
+ * @return
+ *   Actual number of objects the head was moved on
+ *   If behavior == RTE_RING_QUEUE_FIXED, this will be 0 or n only
  */
 static __rte_always_inline uint32_t
-__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+__rte_ring_rts_move_head(struct rte_ring_rts_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
-	uint32_t *free_entries)
+	uint32_t *entries)
 {
 	uint32_t n;
 	union __rte_ring_rts_poscnt nh, oh;
 
-	const uint32_t capacity = r->capacity;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_prod.head.raw, rte_memory_order_acquire);
+	oh.raw = rte_atomic_load_explicit(&d->head.raw,
+			rte_memory_order_acquire);
 
 	do {
 		/* Reset n to the initial burst count */
@@ -88,20 +109,20 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 		 * make sure that we read prod head *before*
 		 * reading cons tail.
 		 */
-		__rte_ring_rts_head_wait(&r->rts_prod, &oh);
+		__rte_ring_rts_head_wait(d, &oh);
 
 		/*
 		 *  The subtraction is done between two unsigned 32bits value
 		 * (the result is always modulo 32 bits even if we have
-		 * *old_head > cons_tail). So 'free_entries' is always between 0
+		 * *old_head > cons_tail). So 'entries' is always between 0
 		 * and capacity (which is < size).
 		 */
-		*free_entries = capacity + r->cons.tail - oh.val.pos;
+		*entries = capacity + s->tail - oh.val.pos;
 
 		/* check that we have enough room in ring */
-		if (unlikely(n > *free_entries))
+		if (unlikely(n > *entries))
 			n = (behavior == RTE_RING_QUEUE_FIXED) ?
-					0 : *free_entries;
+					0 : *entries;
 
 		if (n == 0)
 			break;
@@ -114,14 +135,27 @@ __rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
 	 *  - OOO reads of cons tail value
 	 *  - OOO copy of elems to the ring
 	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_prod.head.raw,
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head.raw,
 			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
+			rte_memory_order_acquire,
+			rte_memory_order_acquire) == 0);
 
 	*old_head = oh.val.pos;
 	return n;
 }
 
+/**
+ * @internal This function updates the producer head for enqueue.
+ */
+static __rte_always_inline uint32_t
+__rte_ring_rts_move_prod_head(struct rte_ring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
+	uint32_t *free_entries)
+{
+	return __rte_ring_rts_move_head(&r->rts_prod, &r->cons,
+			r->capacity, num, behavior, old_head, free_entries);
+}
+
 /**
  * @internal This function updates the consumer head for dequeue
  */
@@ -130,51 +164,8 @@ __rte_ring_rts_move_cons_head(struct rte_ring *r, uint32_t num,
 	enum rte_ring_queue_behavior behavior, uint32_t *old_head,
 	uint32_t *entries)
 {
-	uint32_t n;
-	union __rte_ring_rts_poscnt nh, oh;
-
-	oh.raw = rte_atomic_load_explicit(&r->rts_cons.head.raw, rte_memory_order_acquire);
-
-	/* move cons.head atomically */
-	do {
-		/* Restore n as it may change every loop */
-		n = num;
-
-		/*
-		 * wait for cons head/tail distance,
-		 * make sure that we read cons head *before*
-		 * reading prod tail.
-		 */
-		__rte_ring_rts_head_wait(&r->rts_cons, &oh);
-
-		/* The subtraction is done between two unsigned 32bits value
-		 * (the result is always modulo 32 bits even if we have
-		 * cons_head > prod_tail). So 'entries' is always between 0
-		 * and size(ring)-1.
-		 */
-		*entries = r->prod.tail - oh.val.pos;
-
-		/* Set the actual entries for dequeue */
-		if (n > *entries)
-			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *entries;
-
-		if (unlikely(n == 0))
-			break;
-
-		nh.val.pos = oh.val.pos + n;
-		nh.val.cnt = oh.val.cnt + 1;
-
-	/*
-	 * this CAS(ACQUIRE, ACQUIRE) serves as a hoist barrier to prevent:
-	 *  - OOO reads of prod tail value
-	 *  - OOO copy of elems from the ring
-	 */
-	} while (rte_atomic_compare_exchange_strong_explicit(&r->rts_cons.head.raw,
-			(uint64_t *)(uintptr_t)&oh.raw, nh.raw,
-			rte_memory_order_acquire, rte_memory_order_acquire) == 0);
-
-	*old_head = oh.val.pos;
-	return n;
+	return __rte_ring_rts_move_head(&r->rts_cons, &r->prod,
+			0, num, behavior, old_head, entries);
 }
 
 /**
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 3/7] ring: make copying functions generic
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 4/7] ring: make dump function more verbose Konstantin Ananyev
                                     ` (3 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Note upfront: that change doesn't introduce any functional
or performance changes.
It is just a code-reordering for:
 - improve code modularity and re-usability
 - ability in future to re-use the same code to introduce new functionality

There is no real need for enqueue_elems()/dequeue_elems()
to get pointer to actual rte_ring structure, instead it is enough to pass
a pointer to actual elements buffer inside the ring.
In return, we'll get a copying functions that could be used for other
queueing abstractions that do have circular ring buffer inside.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 lib/ring/rte_ring_elem_pvt.h | 115 ++++++++++++++++++++---------------
 1 file changed, 67 insertions(+), 48 deletions(-)

diff --git a/lib/ring/rte_ring_elem_pvt.h b/lib/ring/rte_ring_elem_pvt.h
index 3a83668a08..6eafae121f 100644
--- a/lib/ring/rte_ring_elem_pvt.h
+++ b/lib/ring/rte_ring_elem_pvt.h
@@ -17,12 +17,14 @@
 #endif
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_32(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
+
+	uint32_t *ring = (uint32_t *)ring_table;
 	const uint32_t *obj = (const uint32_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			ring[idx] = obj[i];
@@ -60,14 +62,14 @@ __rte_ring_enqueue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_64(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
+
+	uint64_t *ring = (uint64_t *)ring_table;
 	const unaligned_uint64_t *obj = (const unaligned_uint64_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			ring[idx] = obj[i];
@@ -93,14 +95,14 @@ __rte_ring_enqueue_elems_64(struct rte_ring *r, uint32_t prod_head,
 }
 
 static __rte_always_inline void
-__rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems_128(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = prod_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
+
+	rte_int128_t *ring = (rte_int128_t *)ring_table;
 	const rte_int128_t *obj = (const rte_int128_t *)obj_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
 			memcpy((void *)(ring + idx),
@@ -126,37 +128,47 @@ __rte_ring_enqueue_elems_128(struct rte_ring *r, uint32_t prod_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
-		const void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_enqueue_elems(void *ring_table, const void *obj_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_enqueue_elems_64(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_64(ring_table, obj_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_enqueue_elems_128(r, prod_head, obj_table, num);
+		__rte_ring_enqueue_elems_128(ring_table, obj_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = prod_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_enqueue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_enqueue_elems_32(ring_table, obj_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
-		uint32_t idx, void *obj_table, uint32_t n)
+__rte_ring_enqueue_elems(struct rte_ring *r, uint32_t prod_head,
+		const void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_enqueue_elems(&r[1], obj_table, r->size,
+			prod_head & r->mask, esize, num);
+}
+
+static __rte_always_inline void
+__rte_ring_dequeue_elems_32(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	uint32_t *ring = (uint32_t *)&r[1];
 	uint32_t *obj = (uint32_t *)obj_table;
+	const uint32_t *ring = (const uint32_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x7); i += 8, idx += 8) {
 			obj[i] = ring[idx];
@@ -194,14 +206,13 @@ __rte_ring_dequeue_elems_32(struct rte_ring *r, const uint32_t size,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_64(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	uint64_t *ring = (uint64_t *)&r[1];
 	unaligned_uint64_t *obj = (unaligned_uint64_t *)obj_table;
+	const uint64_t *ring = (const uint64_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x3); i += 4, idx += 4) {
 			obj[i] = ring[idx];
@@ -227,27 +238,26 @@ __rte_ring_dequeue_elems_64(struct rte_ring *r, uint32_t cons_head,
 }
 
 static __rte_always_inline void
-__rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t n)
+__rte_ring_dequeue_elems_128(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t n)
 {
 	unsigned int i;
-	const uint32_t size = r->size;
-	uint32_t idx = cons_head & r->mask;
-	rte_int128_t *ring = (rte_int128_t *)&r[1];
 	rte_int128_t *obj = (rte_int128_t *)obj_table;
+	const rte_int128_t *ring = (const rte_int128_t *)ring_table;
+
 	if (likely(idx + n <= size)) {
 		for (i = 0; i < (n & ~0x1); i += 2, idx += 2)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 32);
+			memcpy((obj + i), (const void *)(ring + idx), 32);
 		switch (n & 0x1) {
 		case 1:
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		}
 	} else {
 		for (i = 0; idx < size; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 		/* Start at the beginning */
 		for (idx = 0; i < n; i++, idx++)
-			memcpy((void *)(obj + i), (void *)(ring + idx), 16);
+			memcpy((obj + i), (const void *)(ring + idx), 16);
 	}
 }
 
@@ -256,30 +266,39 @@ __rte_ring_dequeue_elems_128(struct rte_ring *r, uint32_t cons_head,
  * single and multi producer enqueue functions.
  */
 static __rte_always_inline void
-__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
-		void *obj_table, uint32_t esize, uint32_t num)
+__rte_ring_do_dequeue_elems(void *obj_table, const void *ring_table,
+	uint32_t size, uint32_t idx, uint32_t esize, uint32_t num)
 {
 	/* 8B and 16B copies implemented individually to retain
 	 * the current performance.
 	 */
 	if (esize == 8)
-		__rte_ring_dequeue_elems_64(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_64(obj_table, ring_table, size,
+				idx, num);
 	else if (esize == 16)
-		__rte_ring_dequeue_elems_128(r, cons_head, obj_table, num);
+		__rte_ring_dequeue_elems_128(obj_table, ring_table, size,
+				idx, num);
 	else {
-		uint32_t idx, scale, nr_idx, nr_num, nr_size;
+		uint32_t scale, nr_idx, nr_num, nr_size;
 
 		/* Normalize to uint32_t */
 		scale = esize / sizeof(uint32_t);
 		nr_num = num * scale;
-		idx = cons_head & r->mask;
 		nr_idx = idx * scale;
-		nr_size = r->size * scale;
-		__rte_ring_dequeue_elems_32(r, nr_size, nr_idx,
-				obj_table, nr_num);
+		nr_size = size * scale;
+		__rte_ring_dequeue_elems_32(obj_table, ring_table, nr_size,
+				nr_idx, nr_num);
 	}
 }
 
+static __rte_always_inline void
+__rte_ring_dequeue_elems(struct rte_ring *r, uint32_t cons_head,
+		void *obj_table, uint32_t esize, uint32_t num)
+{
+	__rte_ring_do_dequeue_elems(obj_table, &r[1], r->size,
+			cons_head & r->mask, esize, num);
+}
+
 /* Between load and load. there might be cpu reorder in weak model
  * (powerpc/arm).
  * There are 2 choices for the users
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 4/7] ring: make dump function more verbose
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                                     ` (2 preceding siblings ...)
  2024-11-11 14:19                   ` [PATCH v10 3/7] ring: make copying functions generic Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
                                     ` (2 subsequent siblings)
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

From: Eimear Morrissey <eimear.morrissey@huawei.com>

The current rte_ring_dump function uses the generic rte_ring_headtail
structure to access head/tail positions. This is incorrect for the RTS
case where the head is stored in a different offset in the union of
structs. Switching to a separate function for each sync type allows
to dump correct head/tail values and extra metadata.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 .mailmap                         |  1 +
 app/test/test_ring_stress_impl.h |  1 +
 lib/ring/rte_ring.c              | 87 ++++++++++++++++++++++++++++++--
 lib/ring/rte_ring.h              | 15 ++++++
 lib/ring/version.map             |  7 +++
 5 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/.mailmap b/.mailmap
index 4894219f2f..dbffe91d8a 100644
--- a/.mailmap
+++ b/.mailmap
@@ -388,6 +388,7 @@ Eduard Serra <eserra@vmware.com>
 Edward Makarov <makarov@kraftway.ru>
 Edwin Brossette <edwin.brossette@6wind.com>
 Eelco Chaudron <echaudro@redhat.com>
+Eimear Morrissey <eimear.morrissey@huawei.com>
 Elad Nachman <eladv6@gmail.com>
 Elad Persiko <eladpe@mellanox.com>
 Elena Agostini <eagostini@nvidia.com>
diff --git a/app/test/test_ring_stress_impl.h b/app/test/test_ring_stress_impl.h
index ee5274aeef..f99a7ff675 100644
--- a/app/test/test_ring_stress_impl.h
+++ b/app/test/test_ring_stress_impl.h
@@ -380,6 +380,7 @@ test_mt1(int (*test)(void *))
 	}
 
 	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_ring_dump(stdout, r);
 	mt1_fini(r, data);
 	return rc;
 }
diff --git a/lib/ring/rte_ring.c b/lib/ring/rte_ring.c
index aebb6d6728..a7d9d6b037 100644
--- a/lib/ring/rte_ring.c
+++ b/lib/ring/rte_ring.c
@@ -364,20 +364,99 @@ rte_ring_free(struct rte_ring *r)
 	rte_free(te);
 }
 
+static const char *
+ring_get_sync_type(const enum rte_ring_sync_type st)
+{
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+		return "ST";
+	case RTE_RING_SYNC_MT:
+		return "MT";
+	case RTE_RING_SYNC_MT_RTS:
+		return "MT_RTS";
+	case RTE_RING_SYNC_MT_HTS:
+		return "MT_HTS";
+	default:
+		return "unknown";
+	}
+}
+
+static void
+ring_dump_ht_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *ht)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(ht->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, ht->head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, ht->tail);
+}
+
+static void
+ring_dump_rts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_rts_headtail *rts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(rts->sync_type));
+	fprintf(f, "%shead.pos=%"PRIu32"\n", prefix, rts->head.val.pos);
+	fprintf(f, "%shead.cnt=%"PRIu32"\n", prefix, rts->head.val.cnt);
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, rts->tail.val.pos);
+	fprintf(f, "%stail.cnt=%"PRIu32"\n", prefix, rts->tail.val.cnt);
+	fprintf(f, "%shtd_max=%"PRIu32"\n", prefix, rts->htd_max);
+}
+
+static void
+ring_dump_hts_headtail(FILE *f, const char *prefix,
+		const struct rte_ring_hts_headtail *hts)
+{
+	fprintf(f, "%ssync_type=%s\n", prefix,
+			ring_get_sync_type(hts->sync_type));
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, hts->ht.pos.head);
+	fprintf(f, "%stail=%"PRIu32"\n", prefix, hts->ht.pos.tail);
+}
+
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r)
+{
+	if (f == NULL || r == NULL)
+		return;
+
+	prefix = (prefix != NULL) ? prefix : "";
+
+	switch (r->sync_type) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		ring_dump_ht_headtail(f, prefix, r);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		ring_dump_rts_headtail(f, prefix,
+				(const struct rte_ring_rts_headtail *)r);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		ring_dump_hts_headtail(f, prefix,
+				(const struct rte_ring_hts_headtail *)r);
+		break;
+	default:
+		RING_LOG(ERR, "Invalid ring sync type detected");
+	}
+}
+
 /* dump the status of the ring on the console */
 void
 rte_ring_dump(FILE *f, const struct rte_ring *r)
 {
+	if (f == NULL || r == NULL)
+		return;
+
 	fprintf(f, "ring <%s>@%p\n", r->name, r);
 	fprintf(f, "  flags=%x\n", r->flags);
 	fprintf(f, "  size=%"PRIu32"\n", r->size);
 	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
-	fprintf(f, "  ct=%"PRIu32"\n", r->cons.tail);
-	fprintf(f, "  ch=%"PRIu32"\n", r->cons.head);
-	fprintf(f, "  pt=%"PRIu32"\n", r->prod.tail);
-	fprintf(f, "  ph=%"PRIu32"\n", r->prod.head);
 	fprintf(f, "  used=%u\n", rte_ring_count(r));
 	fprintf(f, "  avail=%u\n", rte_ring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod));
 }
 
 /* dump the status of all rings on the console */
diff --git a/lib/ring/rte_ring.h b/lib/ring/rte_ring.h
index 11ca69c73d..33ac5e4423 100644
--- a/lib/ring/rte_ring.h
+++ b/lib/ring/rte_ring.h
@@ -204,6 +204,21 @@ void rte_ring_free(struct rte_ring *r);
  */
 void rte_ring_dump(FILE *f, const struct rte_ring *r);
 
+/**
+ * Dump the status of a headtail to a file.
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param prefix
+ *   A string to prefix each output line with
+ * @param r
+ *   A pointer to a ring headtail structure.
+ */
+__rte_experimental
+void
+rte_ring_headtail_dump(FILE *f, const char *prefix,
+		const struct rte_ring_headtail *r);
+
 /**
  * Enqueue several objects on the ring (multi-producers safe).
  *
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 8da094a69a..61f7464f5a 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -14,3 +14,10 @@ DPDK_25 {
 
 	local: *;
 };
+
+EXPERIMENTAL {
+	global:
+
+	# added in 24.11
+	rte_ring_headtail_dump;
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 5/7] ring/soring: introduce Staged Ordered Ring
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                                     ` (3 preceding siblings ...)
  2024-11-11 14:19                   ` [PATCH v10 4/7] ring: make dump function more verbose Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 6/7] app/test: add unit tests for soring API Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 7/7] test: add stress test suite Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Staged-Ordered-Ring (SORING) provides a SW abstraction for 'ordered' queues
with multiple processing 'stages'.
It is based on conventional DPDK rte_ring, re-uses many of its concepts,
and even substantial part of its code.
It can be viewed as an 'extension' of rte_ring functionality.
In particular, main SORING properties:
- circular ring buffer with fixed size objects
- producer, consumer plus multiple processing stages in the middle.
- allows to split objects processing into multiple stages.
- objects remain in the same ring while moving from one stage to the other,
  initial order is preserved, no extra copying needed.
- preserves the ingress order of objects within the queue across multiple
  stages, i.e.:
  at the same stage multiple threads can process objects from the ring in
  any order, but for the next stage objects will always appear in the
  original order.
- each stage (and producer/consumer) can be served by single and/or
  multiple threads.
- number of stages, size and number of objects in the ring are
  configurable at ring initialization time.

Data-path API provides four main operations:
- enqueue/dequeue works in the same manner as for conventional rte_ring,
  all rte_ring synchronization types are supported.
- acquire/release - for each stage there is an acquire (start) and
  release (finish) operation.
  after some objects are 'acquired' - given thread can safely assume that
  it has exclusive possession of these objects till 'release' for them is
  invoked.
  Note that right now user has to release exactly the same number of
  objects that was acquired before.
  After 'release', objects can be 'acquired' by next stage and/or dequeued
  by the consumer (in case of last stage).

Expected use-case: applications that uses pipeline model
(probably with multiple stages) for packet processing, when preserving
incoming packet order is important. I.E.: IPsec processing, etc.

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Morten Brørup <mb@smartsharesystems.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 devtools/build-dict.sh                    |   1 +
 doc/api/doxy-api-index.md                 |   1 +
 doc/guides/prog_guide/img/soring-pic1.svg | 635 ++++++++++++++++++++++
 doc/guides/prog_guide/ring_lib.rst        | 202 +++++++
 doc/guides/rel_notes/release_24_11.rst    |   8 +
 lib/ring/meson.build                      |   4 +-
 lib/ring/rte_soring.c                     | 198 +++++++
 lib/ring/rte_soring.h                     | 557 +++++++++++++++++++
 lib/ring/soring.c                         | 613 +++++++++++++++++++++
 lib/ring/soring.h                         | 138 +++++
 lib/ring/version.map                      |  19 +
 11 files changed, 2374 insertions(+), 2 deletions(-)
 create mode 100644 doc/guides/prog_guide/img/soring-pic1.svg
 create mode 100644 lib/ring/rte_soring.c
 create mode 100644 lib/ring/rte_soring.h
 create mode 100644 lib/ring/soring.c
 create mode 100644 lib/ring/soring.h

diff --git a/devtools/build-dict.sh b/devtools/build-dict.sh
index a8cac49029..5eb592abfa 100755
--- a/devtools/build-dict.sh
+++ b/devtools/build-dict.sh
@@ -17,6 +17,7 @@ sed '/^..->/d' |
 sed '/^uint->/d' |
 sed "/^doesn'->/d" |
 sed '/^wasn->/d' |
+sed '/^soring->/d' |
 
 # print to stdout
 cat
diff --git a/doc/api/doxy-api-index.md b/doc/api/doxy-api-index.md
index f0193502bc..b2fc24b3e4 100644
--- a/doc/api/doxy-api-index.md
+++ b/doc/api/doxy-api-index.md
@@ -174,6 +174,7 @@ The public API headers are grouped by topics:
   [mbuf](@ref rte_mbuf.h),
   [mbuf pool ops](@ref rte_mbuf_pool_ops.h),
   [ring](@ref rte_ring.h),
+  [soring](@ref rte_soring.h),
   [stack](@ref rte_stack.h),
   [tailq](@ref rte_tailq.h),
   [bitset](@ref rte_bitset.h),
diff --git a/doc/guides/prog_guide/img/soring-pic1.svg b/doc/guides/prog_guide/img/soring-pic1.svg
new file mode 100644
index 0000000000..c97e66ca43
--- /dev/null
+++ b/doc/guides/prog_guide/img/soring-pic1.svg
@@ -0,0 +1,635 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<!-- SPDX-License-Identifier: BSD-3-Clause -->
+
+<!-- Copyright(c) 2010 Intel Corporation -->
+
+<svg
+   width="484.05716"
+   height="385.63785"
+   id="svg3388"
+   version="1.1"
+   inkscape:version="1.2.1 (9c6d41e410, 2022-07-14)"
+   sodipodi:docname="soring-pic1.svg"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:dc="http://purl.org/dc/elements/1.1/">
+  <defs
+     id="defs3390">
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend"
+       style="overflow:visible">
+      <path
+         id="path4317"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       sodipodi:type="inkscape:persp3d"
+       inkscape:vp_x="0 : 526.18109 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_z="744.09448 : 526.18109 : 1"
+       inkscape:persp3d-origin="372.04724 : 350.78739 : 1"
+       id="perspective3396" />
+    <inkscape:perspective
+       id="perspective4180"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-6"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-0"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-3"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-06"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-5"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4180-69"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4281-2"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4767"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-7"
+       style="overflow:visible">
+      <path
+         id="path4317-4"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective4799"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4824"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4915"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4937"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4962"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective4993"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-0"
+       style="overflow:visible">
+      <path
+         id="path4317-6"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker4999"
+       style="overflow:visible">
+      <path
+         id="path5001"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5091"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-9"
+       style="overflow:visible">
+      <path
+         id="path4317-0"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5121"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-7"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-1"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5121-9"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <inkscape:perspective
+       id="perspective5710"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend-6"
+       style="overflow:visible">
+      <path
+         id="path4317-7"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt;marker-start:none"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <inkscape:perspective
+       id="perspective5738"
+       inkscape:persp3d-origin="0.5 : 0.33333333 : 1"
+       inkscape:vp_z="1 : 0.5 : 1"
+       inkscape:vp_y="0 : 1000 : 0"
+       inkscape:vp_x="0 : 0.5 : 1"
+       sodipodi:type="inkscape:persp3d" />
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="1"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="232.14286"
+     inkscape:cy="168.21429"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     inkscape:window-width="1483"
+     inkscape:window-height="1002"
+     inkscape:window-x="83"
+     inkscape:window-y="16"
+     inkscape:window-maximized="0"
+     inkscape:snap-grids="false"
+     inkscape:snap-to-guides="true"
+     showguides="false"
+     fit-margin-top="0.1"
+     fit-margin-left="0.1"
+     fit-margin-right="0.1"
+     fit-margin-bottom="0.1"
+     inkscape:showpageshadow="2"
+     inkscape:pagecheckerboard="0"
+     inkscape:deskcolor="#d1d1d1">
+    <inkscape:grid
+       type="xygrid"
+       id="grid5162"
+       empspacing="5"
+       visible="true"
+       enabled="true"
+       snapvisiblegridlinesonly="true"
+       originx="-162.97143"
+       originy="-368.03525"
+       spacingx="1"
+       spacingy="1" />
+  </sodipodi:namedview>
+  <metadata
+     id="metadata3393">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-162.97143,-298.68909)">
+    <rect
+       style="fill:#ffd080;fill-opacity:1;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+       id="rect4257"
+       width="439.41635"
+       height="58.588848"
+       x="186.87822"
+       y="463.44324"
+       rx="11.631636"
+       ry="11.631636" />
+    <g
+       id="g4259"
+       transform="translate(108.51492,3.9469318)">
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="83.143028"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="137.00014"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-3"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="190.85725"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-1"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="244.71437"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-6"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="298.57147"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-2"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="352.42859"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-15"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="406.28571"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-4"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+      <rect
+         ry="11.631636"
+         rx="11.631636"
+         y="463.79074"
+         x="460.14282"
+         height="49.999996"
+         width="52.857113"
+         id="rect3398-65"
+         style="fill:#ff8080;fill-opacity:1;stroke:#000000;stroke-opacity:1" />
+    </g>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="528.41443"
+       y="496.36957"
+       id="text4269-5-6-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5-0"
+         x="528.41443"
+         y="496.36957"
+         style="font-size:14px;line-height:1.25">obj5</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="313.90488"
+       y="495.49646"
+       id="text4269"><tspan
+         sodipodi:role="line"
+         id="tspan4271"
+         x="313.90488"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj1</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="368.95203"
+       y="495.49646"
+       id="text4269-4"><tspan
+         sodipodi:role="line"
+         id="tspan4271-5"
+         x="368.95203"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj2</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="422.99518"
+       y="495.49646"
+       id="text4269-5"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4"
+         x="422.99518"
+         y="495.49646"
+         style="font-size:14px;line-height:1.25">obj3</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 323.57143,578.07647 0,-42.14286"
+       id="path4309"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="289.85715"
+       y="589.505"
+       id="text4787"><tspan
+         sodipodi:role="line"
+         id="tspan4789"
+         x="289.85715"
+         y="589.505"
+         style="font-size:14px;line-height:1.25">cons_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="293.45334"
+       y="603.41034"
+       id="text4787-3"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0"
+         x="293.45334"
+         y="603.41034"
+         style="font-size:14px;line-height:1.25">cons_tail</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="561.2981"
+       y="602.81482"
+       id="text4787-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8"
+         x="561.2981"
+         y="602.81482"
+         style="font-size:14px;line-height:1.25">prod_head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="566.75146"
+       y="587.72028"
+       id="text4787-3-6"><tspan
+         sodipodi:role="line"
+         id="tspan4789-0-8"
+         x="566.75146"
+         y="587.72028"
+         style="font-size:14px;line-height:1.25">prod_tail</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="315.21933"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="172.28571"
+       y="308.93362"
+       id="text4891"><tspan
+         sodipodi:role="line"
+         id="tspan4893"
+         x="172.28571"
+         y="308.93362"
+         style="font-size:14px;line-height:1.25">lstages states</tspan></text>
+    <rect
+       style="fill:none;stroke:#000000;stroke-width:1;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:1, 1;stroke-dashoffset:0"
+       id="rect4889-8"
+       width="482.85715"
+       height="138.57147"
+       x="163.57143"
+       y="529.93365"
+       rx="11.631636"
+       ry="11.631636" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="170.89287"
+       y="684.09021"
+       id="text4891-4"><tspan
+         sodipodi:role="line"
+         id="tspan4893-3"
+         x="170.89287"
+         y="684.09021"
+         style="font-size:14px;line-height:1.25">producer and consumer states</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 379.25296,407.43361 v 42.14286"
+       id="path4309-8"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 486.68152,407.43361 0,42.14286"
+       id="path4309-4-9"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;white-space:pre;inline-size:99.1521;display:inline;fill:#000000;fill-opacity:1;stroke:none"
+       x="296.992"
+       y="399.48123"
+       id="text4787-3-64"
+       transform="translate(54)"><tspan
+         x="296.992"
+         y="399.48123"
+         id="tspan1631"><tspan
+           style="font-size:14px;line-height:1.25"
+           id="tspan1629">stage[1].tail</tspan></tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="440.26532"
+       y="399.48123"
+       id="text4787-7-5"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0"
+         x="440.26532"
+         y="399.48123"
+         style="font-size:14px;line-height:1.25">stage[1].head</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="441.55243"
+       y="377.24719"
+       id="text4787-7-5-7"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-4"
+         x="441.55243"
+         y="377.24719"
+         style="font-size:14px;line-height:1.25">stage[0].tail</tspan></text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="m 593.14285,407.43361 v 42.14286"
+       id="path4309-4-9-9"
+       inkscape:connector-curvature="0" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1px;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;marker-end:url(#Arrow1Lend)"
+       d="M 594,575.57647 V 533.43361"
+       id="path4309-4-3"
+       inkscape:connector-curvature="0" />
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="476.46902"
+       y="495.12097"
+       id="text4269-5-6"><tspan
+         sodipodi:role="line"
+         id="tspan4271-4-5"
+         x="476.46902"
+         y="495.12097"
+         style="font-size:14px;line-height:1.25">obj4</tspan></text>
+    <text
+       xml:space="preserve"
+       style="font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:0%;font-family:Arial;-inkscape-font-specification:Arial;text-align:start;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none"
+       x="552.98102"
+       y="397.24719"
+       id="text4787-7-5-74"><tspan
+         sodipodi:role="line"
+         id="tspan4789-8-0-8"
+         x="552.98102"
+         y="397.24719"
+         style="font-size:14px;line-height:1.25">stage[0].head</tspan></text>
+  </g>
+</svg>
diff --git a/doc/guides/prog_guide/ring_lib.rst b/doc/guides/prog_guide/ring_lib.rst
index f7dbba0e4e..70a454972e 100644
--- a/doc/guides/prog_guide/ring_lib.rst
+++ b/doc/guides/prog_guide/ring_lib.rst
@@ -491,6 +491,208 @@ Following is an example of usage:
 Note that between ``_start_`` and ``_finish_`` no other thread can proceed
 with enqueue(/dequeue) operation till ``_finish_`` completes.
 
+Staged Ordered Ring API
+-----------------------
+
+Staged-Ordered-Ring (SORING) API provides a SW abstraction for *ordered* queues
+with multiple processing *stages*. It is based on conventional DPDK
+``rte_ring`` API, re-uses many of its concepts, and even substantial part of
+its code. It can be viewed as an 'extension' of ``rte_ring`` functionality.
+In particular, main SORING properties:
+
+*   circular ring buffer with fixed size objects and related metadata
+
+*   producer, consumer plus multiple processing stages in between.
+
+*   allows to split objects processing into multiple stages.
+
+*   objects remain in the same ring while moving from one stage to the other,
+    initial order is preserved, no extra copying needed.
+
+*   preserves the ingress order of objects within the queue across multiple
+    stages
+
+*   each stage (and producer/consumer) can be served by single and/or
+    multiple threads.
+
+*   number of stages, size and number of objects and their metadata in the
+    ring are configurable at ring initialization time.
+
+Data-Path API
+~~~~~~~~~~~~~
+
+SORING data-path API provided four main operations:
+
+*   ``enqueue``/``dequeue`` works in the same manner as for conventional
+    ``rte_ring``, all rte_ring synchronization types are supported.
+
+*   ``acquire``/``release`` - for each stage there is an ``acquire`` (start)
+    and ``release`` (finish) operation.
+    After some objects are ``acquired`` - given thread can safely assume that
+    it has exclusive possession of these objects till ``release`` for them is
+    invoked.
+    Note that right now user has to release exactly the same number of
+    objects that was acquired before.
+    After objects are ``released``, given thread loses its possession on them,
+    and they can be either acquired by next stage or dequeued
+    by the consumer (in case of last stage).
+
+A simplified representation of a SORING with two stages is shown below.
+On that picture ``obj5`` and ``obj4`` elements are acquired by stage 0,
+``obj2`` and ``obj3`` are acquired by stage 1, while ``obj11`` was already
+released by stage 1 and is ready to be consumed.
+
+.. _figure_soring1:
+
+.. figure:: img/soring-pic1.*
+
+Along with traditional flavor there are enhanced versions for all these
+data-path operations: ``enqueux``/``dequeux``/``acquirx``/``releasx``.
+All enhanced versions take as extra parameter a pointer to an array of
+metadata values.
+At initialization user can request within the ``soring`` supplementary and
+optional array of metadata associated with each object in the ``soring``.
+While ``soring`` element size is configurable and user can specify it big
+enough to hold both object and its metadata together,
+for performance reasons it might be plausible to access them as separate arrays.
+Note that users are free to mix and match both versions of data-path API in
+a way they like.
+As an example, possible usage scenario when such separation helps:
+
+.. code-block:: c
+
+    /*
+     * use pointer to mbuf as soring element, while tx_state
+     * as a metadata.
+     * In this example we use a soring with just one stage.
+     */
+     union tx_state {
+         /* negative values for error */
+         int32_t rc;
+         /* otherwise contain valid TX port and queue IDs*/
+         struct {
+             uint16_t port_id;
+             uint16_t queue_id;
+         } tx;
+     };
+     struct rte_soring *soring;
+
+
+producer/consumer part:
+
+.. code-block:: c
+
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* enqueue - writes to soring objects array no need to update metadata */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_enqueue_burst(soring, pkts, num, NULL);
+     ....
+     /* dequeux - reads both packets and related tx_state */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_dequeux_burst(soring, pkts, txst, num, NULL);
+
+     /*
+      * TX packets out, or drop in case of error.
+      * Note that we don't need to dereference the soring objects itself
+      * to make a decision.
+      */
+     uint32_t i, j, k, n;
+     struct rte_mbuf *dr[MAX_PKT_BURST];
+
+     k = 0;
+     for (i = 0; i != num; i++) {
+         /* packet processing reports an error */
+         if (txst[i].rc < 0)
+             dr[k++] = pkts[i];
+         /* valid packet, send it out */
+         else {
+             /* group consequitive packets with the same port and queue IDs */
+             for (j = i + 1; j < num; j++)
+                 if (txst[j].rc != txst[i].rc)
+                     break;
+
+             n = rte_eth_tx_burst(txst[i].tx.port_id, txst[i].tx.queue_id,
+                             pkts + i, j - i);
+             if (i + n != j) {
+                 /* decide with unsent packets if any */
+             }
+         }
+     }
+     /* drop erroneous packets */
+     if (k != 0)
+         rte_pktmbuf_free_bulk(dr, k);
+
+acquire/release part:
+
+.. code-block:: c
+
+     uint32_t ftoken;
+     struct rte_mbuf *pkts[MAX_PKT_BURST];
+     union tx_state txst[MAX_PKT_BURST];
+     ...
+     /* acquire - grab some packets to process */
+     uint32_t num = MAX_PKT_BURST;
+     num = rte_soring_acquire_burst(soring, pkts, 0, num, &ftoken, NULL);
+
+     /* process packets, fill txst[] for each */
+     do_process_packets(pkts, txst, num);
+
+     /*
+      * release - assuming that do_process_packets() didn't change
+      * contents of pkts[], we need to update soring metadata array only.
+      */
+     rte_soring_releasx(soring, NULL, txst, 0, num, ftoken);
+
+Use Cases
+~~~~~~~~~~
+
+Expected use-cases include applications that use pipeline model
+(probably with multiple stages) for packet processing, when preserving
+incoming packet order is important. I.E.: IPsec processing, etc.
+
+SORING internals
+~~~~~~~~~~~~~~~~
+
+*   In addition to accessible by the user array of objects (and metadata),
+    ``soirng`` also contains an internal array of states. Each ``state[]``
+    corresponds to exactly one object within the soring. That ``state[]``
+    array is used by ``acquire``/``release``/``dequeue`` operations to
+    store internal information and should not be accessed by the user directly.
+
+*   At ``acquire``, soring  moves stage's head (in a same way as ``rte_ring``
+    ``move_head`` does), plus it saves in ``state[stage.old_head]``
+    information about how many elements were acquired, acquired head position,
+    and special flag value to indicate that given elements are acquired
+    (``SORING_ST_START``).
+    Note that ``acquire`` returns an opaque ``ftoken`` value that user has
+    to provide for ``release`` function.
+
+*   ``release`` extracts old head value from provided by user ``ftoken`` and
+    checks that corresponding ``state[]`` entry contains expected values
+    (mostly for sanity purposes). Then it marks this ``state[]`` entry with
+    ``SORING_ST_FINISH`` flag to indicate that given subset of objects was
+    released. After that, it checks does stage's old ``head`` value equals to
+    its current ``tail`` value. If so, then it performs ``finalize``
+    operation, otherwise ``release`` just returns.
+
+*   As ``state[]`` is shared by all threads, some other thread can perform
+    ``finalize`` operation for given stage. That allows ``release`` to avoid
+    excessive waits on the ``tail`` value.
+    Main purpose of ``finalize`` operation is to walk through ``state[]``
+    array  from current stage's ``tail`` position up to its ``head``,
+    check ``state[]`` and move stage ``tail`` through elements that already
+    are released (in ``SORING_ST_FINISH`` state).
+    Along with that, corresponding ``state[]`` entries are reset back to zero.
+    Note that ``finalize`` for given stage can be called from multiple places:
+    from ``release`` for that stage or from ``acquire`` for next stage, or
+    even from consumer's ``dequeue`` - in case given stage is the last one.
+    So ``finalize`` has to be MT-safe and inside it we have to guarantee that
+    at any given moment only one thread can update stage's ``tail`` and reset
+    corresponding ``state[]`` entries.
+
+
 References
 ----------
 
diff --git a/doc/guides/rel_notes/release_24_11.rst b/doc/guides/rel_notes/release_24_11.rst
index b8f0c58294..4bab0c796a 100644
--- a/doc/guides/rel_notes/release_24_11.rst
+++ b/doc/guides/rel_notes/release_24_11.rst
@@ -291,6 +291,14 @@ New Features
   Added ability for node to advertise and update multiple xstat counters,
   that can be retrieved using ``rte_graph_cluster_stats_get``.
 
+* **Add Staged-Ordered-Ring (SORING) API to the rte_ring library.**
+
+     New API to the ring library to provide a SW abstraction for
+     'ordered' queues with multiple processing 'stages'.
+     It is based on conventional DPDK rte_ring, re-uses many of its concepts,
+     and even substantial part of its code.
+     It can be viewed as an 'extension' of rte_ring functionality.
+
 
 Removed Items
 -------------
diff --git a/lib/ring/meson.build b/lib/ring/meson.build
index 7fca958ed7..21f2c12989 100644
--- a/lib/ring/meson.build
+++ b/lib/ring/meson.build
@@ -1,8 +1,8 @@
 # SPDX-License-Identifier: BSD-3-Clause
 # Copyright(c) 2017 Intel Corporation
 
-sources = files('rte_ring.c')
-headers = files('rte_ring.h')
+sources = files('rte_ring.c', 'rte_soring.c', 'soring.c')
+headers = files('rte_ring.h', 'rte_soring.h')
 # most sub-headers are not for direct inclusion
 indirect_headers += files (
         'rte_ring_core.h',
diff --git a/lib/ring/rte_soring.c b/lib/ring/rte_soring.c
new file mode 100644
index 0000000000..c0eae680e7
--- /dev/null
+++ b/lib/ring/rte_soring.c
@@ -0,0 +1,198 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <inttypes.h>
+
+#include "soring.h"
+#include <rte_string_fns.h>
+
+RTE_LOG_REGISTER_DEFAULT(soring_logtype, INFO);
+
+static uint32_t
+soring_calc_elem_num(uint32_t count)
+{
+	return rte_align32pow2(count + 1);
+}
+
+static int
+soring_check_param(uint32_t esize, uint32_t msize, uint32_t count,
+	uint32_t stages)
+{
+	if (stages == 0) {
+		SORING_LOG(ERR, "invalid number of stages: %u", stages);
+		return -EINVAL;
+	}
+
+	/* Check if element size is a multiple of 4B */
+	if (esize == 0 || esize % 4 != 0) {
+		SORING_LOG(ERR, "invalid element size: %u", esize);
+		return -EINVAL;
+	}
+
+	/* Check if size of metadata is a multiple of 4B */
+	if (msize % 4 != 0) {
+		SORING_LOG(ERR, "invalid metadata size: %u", msize);
+		return -EINVAL;
+	}
+
+	 /* count must be a power of 2 */
+	if (rte_is_power_of_2(count) == 0 ||
+			(count > RTE_SORING_ELEM_MAX + 1)) {
+		SORING_LOG(ERR, "invalid number of elements: %u", count);
+		return -EINVAL;
+	}
+
+	return 0;
+}
+
+/*
+ * Calculate size offsets for SORING internal data layout.
+ */
+static size_t
+soring_get_szofs(uint32_t esize, uint32_t msize, uint32_t count,
+	uint32_t stages, size_t *elst_ofs, size_t *state_ofs,
+	size_t *stage_ofs)
+{
+	size_t sz;
+	const struct rte_soring * const r = NULL;
+
+	sz = sizeof(r[0]) + (size_t)count * esize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (elst_ofs != NULL)
+		*elst_ofs = sz;
+
+	sz = sz + (size_t)count * msize;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (state_ofs != NULL)
+		*state_ofs = sz;
+
+	sz += sizeof(r->state[0]) * count;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	if (stage_ofs != NULL)
+		*stage_ofs = sz;
+
+	sz += sizeof(r->stage[0]) * stages;
+	sz = RTE_ALIGN(sz, RTE_CACHE_LINE_SIZE);
+
+	return sz;
+}
+
+static void
+soring_dump_stage_headtail(FILE *f, const char *prefix,
+		struct soring_stage *st)
+{
+	fprintf(f, "%stail.pos=%"PRIu32"\n", prefix, st->sht.tail.pos);
+	fprintf(f, "%stail.sync=%"PRIu32"\n", prefix, st->sht.tail.sync);
+	fprintf(f, "%shead=%"PRIu32"\n", prefix, st->sht.head);
+}
+
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r)
+{
+	uint32_t i;
+	char buf[32];
+
+	if (f == NULL || r == NULL)
+		return;
+
+	fprintf(f, "soring <%s>@%p\n", r->name, r);
+	fprintf(f, "  size=%"PRIu32"\n", r->size);
+	fprintf(f, "  capacity=%"PRIu32"\n", r->capacity);
+	fprintf(f, "  esize=%"PRIu32"\n", r->esize);
+	fprintf(f, "  msize=%"PRIu32"\n", r->msize);
+	fprintf(f, "  used=%u\n", rte_soring_count(r));
+	fprintf(f, "  avail=%u\n", rte_soring_free_count(r));
+
+	rte_ring_headtail_dump(f, "  cons.", &(r->cons.ht));
+	rte_ring_headtail_dump(f, "  prod.", &(r->prod.ht));
+
+	fprintf(f, "  nb_stage=%"PRIu32"\n", r->nb_stage);
+	for (i = 0; i < r->nb_stage; i++) {
+		snprintf(buf, sizeof(buf), "  stage[%u].", i);
+		soring_dump_stage_headtail(f, buf, r->stage + i);
+	}
+}
+
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t count;
+
+	count = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, count,
+			prm->stages);
+	if (rc != 0)
+		return rc;
+
+	return soring_get_szofs(prm->elem_size, prm->meta_size, count,
+			prm->stages, NULL, NULL, NULL);
+}
+
+/* compilation-time checks */
+static void
+soring_compilation_checks(void)
+{
+	RTE_BUILD_BUG_ON((sizeof(struct rte_soring) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, cons) &
+			RTE_CACHE_LINE_MASK) != 0);
+	RTE_BUILD_BUG_ON((offsetof(struct rte_soring, prod) &
+			RTE_CACHE_LINE_MASK) != 0);
+
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, tail) !=
+		offsetof(struct soring_stage_headtail, tail.pos));
+	RTE_BUILD_BUG_ON(offsetof(struct rte_ring_headtail, sync_type) !=
+		offsetof(struct soring_stage_headtail, unused));
+}
+
+int
+rte_soring_init(struct rte_soring *r, const struct rte_soring_param *prm)
+{
+	int32_t rc;
+	uint32_t n;
+	size_t meta_ofs, stage_ofs, state_ofs;
+
+	soring_compilation_checks();
+
+	if (r == NULL || prm == NULL)
+		return -EINVAL;
+
+	n = soring_calc_elem_num(prm->elems);
+	rc = soring_check_param(prm->elem_size, prm->meta_size, n, prm->stages);
+	if (rc != 0)
+		return rc;
+
+	soring_get_szofs(prm->elem_size, prm->meta_size, n, prm->stages,
+			&meta_ofs, &state_ofs, &stage_ofs);
+
+	memset(r, 0, sizeof(*r));
+	rc = strlcpy(r->name, prm->name, sizeof(r->name));
+	if (rc < 0 || rc >= (int)sizeof(r->name))
+		return -ENAMETOOLONG;
+
+	r->size = n;
+	r->mask = r->size - 1;
+	r->capacity = prm->elems;
+	r->esize = prm->elem_size;
+	r->msize = prm->meta_size;
+
+	r->prod.ht.sync_type = prm->prod_synt;
+	r->cons.ht.sync_type = prm->cons_synt;
+
+	r->state = (union soring_state *)((uintptr_t)r + state_ofs);
+	memset(r->state, 0, sizeof(r->state[0]) * r->size);
+
+	r->stage = (struct soring_stage *)((uintptr_t)r + stage_ofs);
+	r->nb_stage = prm->stages;
+	memset(r->stage, 0, r->nb_stage * sizeof(r->stage[0]));
+
+	if (r->msize != 0)
+		r->meta = (void *)((uintptr_t)r + meta_ofs);
+
+	return 0;
+}
diff --git a/lib/ring/rte_soring.h b/lib/ring/rte_soring.h
new file mode 100644
index 0000000000..1c3a798e0f
--- /dev/null
+++ b/lib/ring/rte_soring.h
@@ -0,0 +1,557 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _RTE_SORING_H_
+#define _RTE_SORING_H_
+
+/**
+ * @file
+ * This file contains definition of DPDK soring (Staged Ordered Ring)
+ * public API.
+ * Brief description:
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * after some elems are 'acquired' - user  can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Extra debugging might be enabled with RTE_SORING_DEBUG macro.
+ */
+
+#include <rte_ring.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/** upper 2 bits are used for status */
+#define RTE_SORING_ST_BIT       30
+
+/** max possible number of elements in the soring */
+#define RTE_SORING_ELEM_MAX	(RTE_BIT32(RTE_SORING_ST_BIT) - 1)
+
+struct rte_soring_param {
+	/** expected name of the soring */
+	const char *name;
+	/** number of elements in the soring */
+	uint32_t elems;
+	/** size of elements in the soring, must be a multiple of 4 */
+	uint32_t elem_size;
+	/**
+	 * size of metadata for each elem, must be a multiple of 4.
+	 * This parameter defines a size of supplementary and optional
+	 * array of metadata associated with each object in the soring.
+	 * While element size is configurable (see 'elem_size' parameter above),
+	 * so user can specify it big enough to hold both object and its
+	 * metadata together, for performance reasons it might be plausible
+	 * to access them as separate arrays.
+	 * Common usage scenario when such separation helps:
+	 * enqueue() - writes to objects array
+	 * acquire() - reads from objects array
+	 * release() - writes to metadata array (as an example: return code)
+	 * dequeue() - reads both objects and metadata array
+	 */
+	uint32_t meta_size;
+	/** number of stages in the soring */
+	uint32_t stages;
+	/** sync type for producer */
+	enum rte_ring_sync_type prod_synt;
+	/** sync type for consumer */
+	enum rte_ring_sync_type cons_synt;
+};
+
+struct rte_soring;
+
+/**
+ * Calculate the memory size needed for a soring
+ *
+ * This function returns the number of bytes needed for a soring, given
+ * the expected parameters for it. This value is the sum of the size of
+ * the internal metadata and the size of the memory needed by the
+ * actual soring elements and their metadata. The value is aligned to a cache
+ * line size.
+ *
+ * @param prm
+ *   Pointer to the structure that contains soring creation parameters.
+ * @return
+ *   - The memory size needed for the soring on success.
+ *   - -EINVAL if provided parameter values are invalid.
+ */
+__rte_experimental
+ssize_t
+rte_soring_get_memsize(const struct rte_soring_param *prm);
+
+/**
+ * Initialize a soring structure.
+ *
+ * Initialize a soring structure in memory pointed by "r".
+ * The size of the memory area must be large enough to store the soring
+ * internal structures plus the objects and metadata tables.
+ * It is strongly advised to use @ref rte_soring_get_memsize() to get the
+ * appropriate size.
+ *
+ * @param r
+ *   Pointer to the soring structure.
+ * @param prm
+ *   Pointer to the structure that contains soring creation parameters.
+ * @return
+ *   - 0 on success, or a negative error code.
+ */
+__rte_experimental
+int
+rte_soring_init(struct rte_soring *r,  const struct rte_soring_param *prm);
+
+/**
+ * Return the total number of filled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_count(const struct rte_soring *r);
+
+/**
+ * Return the total number of unfilled entries in a soring.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @return
+ *   The number of free entries in the soring.
+ */
+__rte_experimental
+unsigned int
+rte_soring_free_count(const struct rte_soring *r);
+
+/**
+ * Dump the status of the soring
+ *
+ * @param f
+ *   A pointer to a file for output
+ * @param r
+ *   Pointer to the soring structure.
+ */
+__rte_experimental
+void
+rte_soring_dump(FILE *f, const struct rte_soring *r);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued, either 0 or n.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs,
+	uint32_t n, uint32_t *free_space);
+
+/**
+ * Enqueue several objects plus metadata on the soring.
+ * Enqueues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to enqueue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to enqueue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param n
+ *   The number of objects to add in the soring from the 'objs'.
+ * @param free_space
+ *   if non-NULL, returns the amount of space in the soring after the
+ *   enqueue operation has finished.
+ * @return
+ *   - Actual number of objects enqueued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Dequeue several objects plus metadata from the soring.
+ * Dequeues up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to dequeue.
+ *   Size of objects to enqueue must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to array of metadata values for each object to dequeue.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param num
+ *   The number of objects to dequeue from the soring into the objs.
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries after the
+ *   dequeue has finished.
+ * @return
+ *   - Actual number of objects dequeued.
+ */
+__rte_experimental
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires exactly requested number of objects or none.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired, either 0 or 'num'.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Acquire several objects plus metadata from the soring for given stage.
+ * Acquires up to requested number of objects.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to acquire.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each for each acquired object.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then 'meta' parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Stage to acquire objects for.
+ * @param num
+ *   The number of objects to acquire.
+ * @param ftoken
+ *   Pointer to the opaque 'token' value used by release() op.
+ *   User has to store this value somewhere, and later provide to the
+ *   release().
+ * @param available
+ *   If non-NULL, returns the number of remaining soring entries for given stage
+ *   after the acquire has finished.
+ * @return
+ *   - Actual number of objects acquired.
+ */
+__rte_experimental
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available);
+
+/**
+ * Release several objects for given stage back to the soring.
+ * Note that it means these objects become available for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to release.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken);
+
+/**
+ * Release several objects plus metadata for given stage back to the soring.
+ * Note that it means these objects become available for next stage or
+ * dequeue.
+ *
+ * @param r
+ *   A pointer to the soring structure.
+ * @param objs
+ *   A pointer to an array of objects to release.
+ *   Note that unless user needs to overwrite soring objects this parameter
+ *   can be NULL.
+ *   Size of objects must be the same value as 'elem_size' parameter
+ *   used while creating the soring. Otherwise the results are undefined.
+ * @param meta
+ *   A pointer to an array of metadata values for each object to release.
+ *   Note that if user not using object metadata values, then this parameter
+ *   can be NULL.
+ *   Size of elements in this array must be the same value as 'meta_size'
+ *   parameter used while creating the soring. If user created the soring with
+ *   'meta_size' value equals zero, then meta parameter should be NULL.
+ *   Otherwise the results are undefined.
+ * @param stage
+ *   Current stage.
+ * @param n
+ *   The number of objects to release.
+ *   Has to be the same value as returned by acquire() op.
+ * @param ftoken
+ *   Opaque 'token' value obtained from acquire() op.
+ */
+__rte_experimental
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* _RTE_SORING_H_ */
diff --git a/lib/ring/soring.c b/lib/ring/soring.c
new file mode 100644
index 0000000000..e8fe890597
--- /dev/null
+++ b/lib/ring/soring.c
@@ -0,0 +1,613 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+/**
+ * @file
+ * This file contains implementation of SORING 'datapath' functions.
+ * Brief description:
+ * ==================
+ * enqueue/dequeue works the same as for conventional rte_ring:
+ * any rte_ring sync types can be used, etc.
+ * Plus there could be multiple 'stages'.
+ * For each stage there is an acquire (start) and release (finish) operation.
+ * After some elems are 'acquired' - user can safely assume that he has
+ * exclusive possession of these elems till 'release' for them is done.
+ * Note that right now user has to release exactly the same number of elems
+ * he acquired before.
+ * After 'release', elems can be 'acquired' by next stage and/or dequeued
+ * (in case of last stage).
+ * Internal structure:
+ * ===================
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * How it works:
+ * =============
+ * 'acquire()' just moves stage's head (same as rte_ring move_head does),
+ * plus it saves in state[stage.cur_head] information about how many elems
+ * were acquired, current head position and special flag value to indicate
+ * that elems are acquired (SORING_ST_START).
+ * Note that 'acquire()' returns to the user a special 'ftoken' that user has
+ * to provide for 'release()' (in fact it is just a position for current head
+ * plus current stage index).
+ * 'release()' extracts old head value from provided ftoken and checks that
+ * corresponding 'state[]' contains expected values(mostly for sanity
+ * purposes).
+ * * Then it marks this state[] with 'SORING_ST_FINISH' flag to indicate
+ * that given subset of objects was released.
+ * After that, it checks does old head value equals to current tail value?
+ * If yes, then it performs  'finalize()' operation, otherwise 'release()'
+ * just returns (without spinning on stage tail value).
+ * As updated state[] is shared by all threads, some other thread can do
+ * 'finalize()' for given stage.
+ * That allows 'release()' to avoid excessive waits on the tail value.
+ * Main purpose of 'finalize()' operation is to walk through 'state[]'
+ * from current stage tail up to its head, check state[] and move stage tail
+ * through elements that already are in SORING_ST_FINISH state.
+ * Along with that, corresponding state[] values are reset to zero.
+ * Note that 'finalize()' for given stage can be done from multiple places:
+ * 'release()' for that stage or from 'acquire()' for next stage
+ * even from consumer's 'dequeue()' - in case given stage is the last one.
+ * So 'finalize()' has to be MT-safe and inside it we have to
+ * guarantee that only one thread will update state[] and stage's tail values.
+ */
+
+#include "soring.h"
+
+/*
+ * Inline functions (fastpath) start here.
+ */
+static __rte_always_inline uint32_t
+__rte_soring_stage_finalize(struct soring_stage_headtail *sht, uint32_t stage,
+	union soring_state *rstate, uint32_t rmask, uint32_t maxn)
+{
+	int32_t rc;
+	uint32_t ftkn, head, i, idx, k, n, tail;
+	union soring_stage_tail nt, ot;
+	union soring_state st;
+
+	/* try to grab exclusive right to update tail value */
+	ot.raw = rte_atomic_load_explicit(&sht->tail.raw,
+			rte_memory_order_acquire);
+
+	/* other thread already finalizing it for us */
+	if (ot.sync != 0)
+		return 0;
+
+	nt.pos = ot.pos;
+	nt.sync = 1;
+	rc = rte_atomic_compare_exchange_strong_explicit(&sht->tail.raw,
+		(uint64_t *)(uintptr_t)&ot.raw, nt.raw,
+		rte_memory_order_release, rte_memory_order_relaxed);
+
+	/* other thread won the race */
+	if (rc == 0)
+		return 0;
+
+	/* Ensure the head is read before rstate[] */
+	head = rte_atomic_load_explicit(&sht->head, rte_memory_order_relaxed);
+	rte_atomic_thread_fence(rte_memory_order_acquire);
+
+	/*
+	 * start with current tail and walk through states that are
+	 * already finished.
+	 */
+
+	n = RTE_MIN(head - ot.pos, maxn);
+	for (i = 0, tail = ot.pos; i < n; i += k, tail += k) {
+
+		idx = tail & rmask;
+		ftkn = SORING_FTKN_MAKE(tail, stage);
+
+		st.raw = rte_atomic_load_explicit(&rstate[idx].raw,
+			rte_memory_order_relaxed);
+		if ((st.stnum & SORING_ST_MASK) != SORING_ST_FINISH ||
+				st.ftoken != ftkn)
+			break;
+
+		k = st.stnum & ~SORING_ST_MASK;
+		rte_atomic_store_explicit(&rstate[idx].raw, 0,
+				rte_memory_order_relaxed);
+	}
+
+
+	/* release exclusive right to update along with new tail value */
+	ot.pos = tail;
+	rte_atomic_store_explicit(&sht->tail.raw, ot.raw,
+			rte_memory_order_release);
+
+	return i;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_prod_head(struct rte_soring *r, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *free)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->prod.ht, &r->cons.ht,
+			r->capacity, st, num, behavior, head, next, free);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->prod.rts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->prod.hts, &r->cons.ht,
+			r->capacity, num, behavior, head, free);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*free = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_move_cons_head(struct rte_soring *r, uint32_t stage, uint32_t num,
+	enum rte_ring_queue_behavior behavior, enum rte_ring_sync_type st,
+	uint32_t *head, uint32_t *next, uint32_t *avail)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		n = __rte_ring_headtail_move_head(&r->cons.ht,
+			&r->stage[stage].ht, 0, st, num, behavior,
+			head, next, avail);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		n = __rte_ring_rts_move_head(&r->cons.rts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = __rte_ring_hts_move_head(&r->cons.hts, &r->stage[stage].ht,
+			0, num, behavior, head, avail);
+		*next = *head + n;
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+		*avail = 0;
+		n = 0;
+	}
+
+	return n;
+}
+
+static __rte_always_inline void
+__rte_soring_update_tail(struct __rte_ring_headtail *rht,
+	enum rte_ring_sync_type st, uint32_t head, uint32_t next, uint32_t enq)
+{
+	uint32_t n;
+
+	switch (st) {
+	case RTE_RING_SYNC_ST:
+	case RTE_RING_SYNC_MT:
+		__rte_ring_update_tail(&rht->ht, head, next, st, enq);
+		break;
+	case RTE_RING_SYNC_MT_RTS:
+		__rte_ring_rts_update_tail(&rht->rts);
+		break;
+	case RTE_RING_SYNC_MT_HTS:
+		n = next - head;
+		__rte_ring_hts_update_tail(&rht->hts, head, n, enq);
+		break;
+	default:
+		/* unsupported mode, shouldn't be here */
+		RTE_ASSERT(0);
+	}
+}
+
+static __rte_always_inline uint32_t
+__rte_soring_stage_move_head(struct soring_stage_headtail *d,
+	const struct rte_ring_headtail *s, uint32_t capacity, uint32_t num,
+	enum rte_ring_queue_behavior behavior,
+	uint32_t *old_head, uint32_t *new_head, uint32_t *avail)
+{
+	uint32_t n, tail;
+
+	*old_head = rte_atomic_load_explicit(&d->head,
+			rte_memory_order_relaxed);
+
+	do {
+		n = num;
+
+		/* Ensure the head is read before tail */
+		rte_atomic_thread_fence(rte_memory_order_acquire);
+
+		tail = rte_atomic_load_explicit(&s->tail,
+				rte_memory_order_acquire);
+		*avail = capacity + tail - *old_head;
+		if (n > *avail)
+			n = (behavior == RTE_RING_QUEUE_FIXED) ? 0 : *avail;
+		if (n == 0)
+			return 0;
+		*new_head = *old_head + n;
+	} while (rte_atomic_compare_exchange_strong_explicit(&d->head,
+			old_head, *new_head, rte_memory_order_acq_rel,
+			rte_memory_order_relaxed) == 0);
+
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_enqueue(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, enum rte_ring_queue_behavior behavior,
+	uint32_t *free_space)
+{
+	enum rte_ring_sync_type st;
+	uint32_t nb_free, prod_head, prod_next;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	st = r->prod.ht.sync_type;
+
+	n = __rte_soring_move_prod_head(r, n, behavior, st,
+			&prod_head, &prod_next, &nb_free);
+	if (n != 0) {
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size,
+			prod_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_enqueue_elems(r->meta, meta, r->size,
+				prod_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->prod, st, prod_head, prod_next, 1);
+	}
+
+	if (free_space != NULL)
+		*free_space = nb_free - n;
+	return n;
+}
+
+static __rte_always_inline uint32_t
+soring_dequeue(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *available)
+{
+	enum rte_ring_sync_type st;
+	uint32_t entries, cons_head, cons_next, n, ns, reqn;
+
+	RTE_ASSERT(r != NULL && r->nb_stage > 0);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	ns = r->nb_stage - 1;
+	st = r->cons.ht.sync_type;
+
+	/* try to grab exactly @num elems first */
+	n = __rte_soring_move_cons_head(r, ns, num, RTE_RING_QUEUE_FIXED, st,
+			&cons_head, &cons_next, &entries);
+	if (n == 0) {
+		/* try to finalize some elems from previous stage */
+		n = __rte_soring_stage_finalize(&r->stage[ns].sht, ns,
+			r->state, r->mask, 2 * num);
+		entries += n;
+
+		/* repeat attempt to grab elems */
+		reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+		if (entries >= reqn)
+			n = __rte_soring_move_cons_head(r, ns, num, behavior,
+				st, &cons_head, &cons_next, &entries);
+		else
+			n = 0;
+	}
+
+	/* we have some elems to consume */
+	if (n != 0) {
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size,
+			cons_head & r->mask, r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta, r->size,
+				cons_head & r->mask, r->msize, n);
+		__rte_soring_update_tail(&r->cons, st, cons_head, cons_next, 0);
+	}
+
+	if (available != NULL)
+		*available = entries - n;
+	return n;
+}
+
+/*
+ * Verify internal SORING state.
+ * WARNING: if expected value is not equal to actual one, it means that for
+ * whatever reason SORING data constancy is broken. That is a very serious
+ * problem that most likely will cause race-conditions, memory corruption,
+ * program crash.
+ * To ease debugging it user might rebuild ring library with
+ * RTE_SORING_DEBUG enabled.
+ */
+static __rte_always_inline void
+soring_verify_state(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	const char *msg, union soring_state val,  union soring_state exp)
+{
+	if (val.raw != exp.raw) {
+#ifdef RTE_SORING_DEBUG
+		rte_soring_dump(stderr, r);
+		rte_panic("line:%d from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};\n",
+			__LINE__, msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#else
+		SORING_LOG(EMERG, "from:%s: soring=%p, stage=%#x, idx=%#x, "
+			"expected={.stnum=%#x, .ftoken=%#x}, "
+			"actual={.stnum=%#x, .ftoken=%#x};",
+			msg, r, stage, idx,
+			exp.stnum, exp.ftoken,
+			val.stnum, val.ftoken);
+#endif
+	}
+}
+
+/* check and update state ring at acquire op*/
+static __rte_always_inline void
+acquire_state_update(const struct rte_soring *r, uint32_t stage, uint32_t idx,
+	uint32_t ftoken, uint32_t num)
+{
+	union soring_state st;
+	const union soring_state est = {.raw = 0};
+
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	st.ftoken = ftoken;
+	st.stnum = (SORING_ST_START | num);
+
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+}
+
+static __rte_always_inline uint32_t
+soring_acquire(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, enum rte_ring_queue_behavior behavior,
+	uint32_t *ftoken, uint32_t *available)
+{
+	uint32_t avail, head, idx, n, next, reqn;
+	struct soring_stage *pstg;
+	struct soring_stage_headtail *cons;
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	cons = &r->stage[stage].sht;
+
+	if (stage == 0)
+		n = __rte_soring_stage_move_head(cons, &r->prod.ht, 0, num,
+			behavior, &head, &next, &avail);
+	else {
+		pstg = r->stage + stage - 1;
+
+		/* try to grab exactly @num elems */
+		n = __rte_soring_stage_move_head(cons, &pstg->ht, 0, num,
+			RTE_RING_QUEUE_FIXED, &head, &next, &avail);
+		if (n == 0) {
+			/* try to finalize some elems from previous stage */
+			n = __rte_soring_stage_finalize(&pstg->sht, stage - 1,
+				r->state, r->mask, 2 * num);
+			avail += n;
+
+			/* repeat attempt to grab elems */
+			reqn = (behavior == RTE_RING_QUEUE_FIXED) ? num : 0;
+			if (avail >= reqn)
+				n = __rte_soring_stage_move_head(cons,
+					&pstg->ht, 0, num, behavior, &head,
+					&next, &avail);
+			else
+				n = 0;
+		}
+	}
+
+	if (n != 0) {
+
+		idx = head & r->mask;
+		*ftoken = SORING_FTKN_MAKE(head, stage);
+
+		/* check and update state value */
+		acquire_state_update(r, stage, idx, *ftoken, n);
+
+		/* copy elems that are ready for given stage */
+		__rte_ring_do_dequeue_elems(objs, &r[1], r->size, idx,
+				r->esize, n);
+		if (meta != NULL)
+			__rte_ring_do_dequeue_elems(meta, r->meta,
+				r->size, idx, r->msize, n);
+	}
+
+	if (available != NULL)
+		*available = avail - n;
+	return n;
+}
+
+static __rte_always_inline void
+soring_release(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	uint32_t idx, pos, tail;
+	struct soring_stage *stg;
+	union soring_state st;
+
+	const union soring_state est = {
+		.stnum = (SORING_ST_START | n),
+		.ftoken = ftoken,
+	};
+
+	RTE_ASSERT(r != NULL && stage < r->nb_stage);
+	RTE_ASSERT(meta == NULL || r->meta != NULL);
+
+	stg = r->stage + stage;
+
+	pos = SORING_FTKN_POS(ftoken, stage);
+	idx = pos & r->mask;
+	st.raw = rte_atomic_load_explicit(&r->state[idx].raw,
+			rte_memory_order_relaxed);
+
+	/* check state ring contents */
+	soring_verify_state(r, stage, idx, __func__, st, est);
+
+	/* update contents of the ring, if necessary */
+	if (objs != NULL)
+		__rte_ring_do_enqueue_elems(&r[1], objs, r->size, idx,
+			r->esize, n);
+	if (meta != NULL)
+		__rte_ring_do_enqueue_elems(r->meta, meta, r->size, idx,
+			r->msize, n);
+
+	/* set state to FINISH, make sure it is not reordered */
+	rte_atomic_thread_fence(rte_memory_order_release);
+
+	st.stnum = SORING_ST_FINISH | n;
+	rte_atomic_store_explicit(&r->state[idx].raw, st.raw,
+			rte_memory_order_relaxed);
+
+	/* try to do finalize(), if appropriate */
+	tail = rte_atomic_load_explicit(&stg->sht.tail.pos,
+			rte_memory_order_relaxed);
+	if (tail == pos)
+		__rte_soring_stage_finalize(&stg->sht, stage, r->state, r->mask,
+				r->capacity);
+}
+
+/*
+ * Public functions (data-path) start here.
+ */
+
+void
+rte_soring_release(struct rte_soring *r, const void *objs,
+	uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, NULL, stage, n, ftoken);
+}
+
+
+void
+rte_soring_releasx(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t stage, uint32_t n, uint32_t ftoken)
+{
+	soring_release(r, objs, meta, stage, n, ftoken);
+}
+
+uint32_t
+rte_soring_enqueue_bulk(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_bulk(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_FIXED,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueue_burst(struct rte_soring *r, const void *objs, uint32_t n,
+	uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, NULL, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_enqueux_burst(struct rte_soring *r, const void *objs,
+	const void *meta, uint32_t n, uint32_t *free_space)
+{
+	return soring_enqueue(r, objs, meta, n, RTE_RING_QUEUE_VARIABLE,
+			free_space);
+}
+
+uint32_t
+rte_soring_dequeue_bulk(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_FIXED,
+			available);
+}
+
+uint32_t
+rte_soring_dequeue_burst(struct rte_soring *r, void *objs, uint32_t num,
+	uint32_t *available)
+{
+	return soring_dequeue(r, objs, NULL, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_dequeux_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t num, uint32_t *available)
+{
+	return soring_dequeue(r, objs, meta, num, RTE_RING_QUEUE_VARIABLE,
+			available);
+}
+
+uint32_t
+rte_soring_acquire_bulk(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_bulk(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_FIXED, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquire_burst(struct rte_soring *r, void *objs,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, NULL, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+uint32_t
+rte_soring_acquirx_burst(struct rte_soring *r, void *objs, void *meta,
+	uint32_t stage, uint32_t num, uint32_t *ftoken, uint32_t *available)
+{
+	return soring_acquire(r, objs, meta, stage, num,
+			RTE_RING_QUEUE_VARIABLE, ftoken, available);
+}
+
+unsigned int
+rte_soring_count(const struct rte_soring *r)
+{
+	uint32_t prod_tail = r->prod.ht.tail;
+	uint32_t cons_tail = r->cons.ht.tail;
+	uint32_t count = (prod_tail - cons_tail) & r->mask;
+	return (count > r->capacity) ? r->capacity : count;
+}
+
+unsigned int
+rte_soring_free_count(const struct rte_soring *r)
+{
+	return r->capacity - rte_soring_count(r);
+}
diff --git a/lib/ring/soring.h b/lib/ring/soring.h
new file mode 100644
index 0000000000..455cf677a7
--- /dev/null
+++ b/lib/ring/soring.h
@@ -0,0 +1,138 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#ifndef _SORING_H_
+#define _SORING_H_
+
+/**
+ * @file
+ * This file contains internal structures of DPDK soring: Staged Ordered Ring.
+ * Sort of extension of conventional DPDK ring.
+ * Internal structure:
+ * In addition to 'normal' ring of elems, it also has a ring of states of the
+ * same size. Each state[] corresponds to exactly one elem[].
+ * state[] will be used by acquire/release/dequeue functions to store internal
+ * information and should not be accessed by the user directly.
+ * For actual implementation details, please refer to soring.c
+ */
+
+#include <rte_soring.h>
+
+/* logging stuff, register our own tag for SORING */
+#include <rte_log.h>
+
+extern int soring_logtype;
+#define RTE_LOGTYPE_SORING soring_logtype
+#define SORING_LOG(level, ...) \
+	RTE_LOG_LINE(level, SORING, "" __VA_ARGS__)
+
+/**
+ * SORING internal state for each element
+ */
+union soring_state {
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) ftoken;
+		RTE_ATOMIC(uint32_t) stnum;
+	};
+};
+
+/* upper 2 bits are used for status */
+#define SORING_ST_START		RTE_SHIFT_VAL32(1, RTE_SORING_ST_BIT)
+#define SORING_ST_FINISH	RTE_SHIFT_VAL32(2, RTE_SORING_ST_BIT)
+
+#define SORING_ST_MASK	\
+	RTE_GENMASK32(sizeof(uint32_t) * CHAR_BIT - 1, RTE_SORING_ST_BIT)
+
+#define SORING_FTKN_MAKE(pos, stg)	((pos) + (stg))
+#define SORING_FTKN_POS(ftk, stg)	((ftk) - (stg))
+
+/**
+ * SORING re-uses rte_ring internal structures and implementation
+ * for enqueue/dequeue operations.
+ */
+struct __rte_ring_headtail {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct rte_ring_hts_headtail hts;
+		struct rte_ring_rts_headtail rts;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+union soring_stage_tail {
+	/** raw 8B value to read/write *sync* and *pos* as one atomic op */
+	alignas(sizeof(uint64_t)) RTE_ATOMIC(uint64_t) raw;
+	struct {
+		RTE_ATOMIC(uint32_t) sync;
+		RTE_ATOMIC(uint32_t) pos;
+	};
+};
+
+struct soring_stage_headtail {
+	volatile union soring_stage_tail tail;
+	enum rte_ring_sync_type unused;  /**< unused */
+	volatile RTE_ATOMIC(uint32_t) head;
+};
+
+/**
+ * SORING stage head_tail structure is 'compatible' with rte_ring ones.
+ * rte_ring internal functions for moving producer/consumer head
+ * can work transparently with stage head_tail and visa-versa
+ * (no modifications required).
+ */
+struct soring_stage {
+
+	union __rte_cache_aligned {
+		struct rte_ring_headtail ht;
+		struct soring_stage_headtail sht;
+	};
+
+	RTE_CACHE_GUARD;
+};
+
+/**
+ * soring internal structure.
+ * As with rte_ring actual elements array supposed to be located directly
+ * after the rte_soring structure.
+ */
+struct  __rte_cache_aligned rte_soring {
+	uint32_t size;           /**< Size of ring. */
+	uint32_t mask;           /**< Mask (size-1) of ring. */
+	uint32_t capacity;       /**< Usable size of ring */
+	uint32_t esize;
+	/**< size of elements in the ring, must be a multiple of 4*/
+	uint32_t msize;
+	/**< size of metadata value for each elem, must be a multiple of 4 */
+
+	/** Ring stages */
+	struct soring_stage *stage;
+	uint32_t nb_stage;
+
+	/** Ring of states (one per element) */
+	union soring_state *state;
+
+	/** Pointer to the buffer where metadata values for each elements
+	 * are stored. This is supplementary and optional information that
+	 * user can attach to each element of the ring.
+	 * While it is possible to incorporate this information inside
+	 * user-defined element, in many cases it is plausible to maintain it
+	 * as a separate array (mainly for performance reasons).
+	 */
+	void *meta;
+
+	RTE_CACHE_GUARD;
+
+	/** Ring producer status. */
+	struct __rte_ring_headtail prod;
+
+	/** Ring consumer status. */
+	struct __rte_ring_headtail cons;
+
+	char name[RTE_RING_NAMESIZE];  /**< Name of the ring. */
+};
+
+#endif /* _SORING_H_ */
diff --git a/lib/ring/version.map b/lib/ring/version.map
index 61f7464f5a..081034c30e 100644
--- a/lib/ring/version.map
+++ b/lib/ring/version.map
@@ -20,4 +20,23 @@ EXPERIMENTAL {
 
 	# added in 24.11
 	rte_ring_headtail_dump;
+	rte_soring_acquire_bulk;
+	rte_soring_acquire_burst;
+	rte_soring_acquirx_bulk;
+	rte_soring_acquirx_burst;
+	rte_soring_count;
+	rte_soring_dequeue_bulk;
+	rte_soring_dequeue_burst;
+	rte_soring_dequeux_bulk;
+	rte_soring_dequeux_burst;
+	rte_soring_enqueue_bulk;
+	rte_soring_enqueue_burst;
+	rte_soring_enqueux_bulk;
+	rte_soring_enqueux_burst;
+	rte_soring_dump;
+	rte_soring_free_count;
+	rte_soring_get_memsize;
+	rte_soring_init;
+	rte_soring_release;
+	rte_soring_releasx;
 };
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 6/7] app/test: add unit tests for soring API
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                                     ` (4 preceding siblings ...)
  2024-11-11 14:19                   ` [PATCH v10 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  2024-11-11 14:19                   ` [PATCH v10 7/7] test: add stress test suite Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add both functional and stess test-cases for soring API.
Stress test serves as both functional and performance test of soring
enqueue/dequeue/acquire/release operations under high contention
(for both over committed and non-over committed scenarios).

Signed-off-by: Eimear Morrissey <eimear.morrissey@huawei.com>
Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/meson.build               |   3 +
 app/test/test_soring.c             | 442 +++++++++++++++
 app/test/test_soring_mt_stress.c   |  40 ++
 app/test/test_soring_stress.c      |  48 ++
 app/test/test_soring_stress.h      |  35 ++
 app/test/test_soring_stress_impl.h | 838 +++++++++++++++++++++++++++++
 6 files changed, 1406 insertions(+)
 create mode 100644 app/test/test_soring.c
 create mode 100644 app/test/test_soring_mt_stress.c
 create mode 100644 app/test/test_soring_stress.c
 create mode 100644 app/test/test_soring_stress.h
 create mode 100644 app/test/test_soring_stress_impl.h

diff --git a/app/test/meson.build b/app/test/meson.build
index 40f22a54d5..b11da1bbc7 100644
--- a/app/test/meson.build
+++ b/app/test/meson.build
@@ -179,6 +179,9 @@ source_file_deps = {
     'test_security_proto.c' : ['cryptodev', 'security'],
     'test_seqlock.c': [],
     'test_service_cores.c': [],
+    'test_soring.c': [],
+    'test_soring_mt_stress.c': [],
+    'test_soring_stress.c': [],
     'test_spinlock.c': [],
     'test_stack.c': ['stack'],
     'test_stack_perf.c': ['stack'],
diff --git a/app/test/test_soring.c b/app/test/test_soring.c
new file mode 100644
index 0000000000..6773a399e6
--- /dev/null
+++ b/app/test/test_soring.c
@@ -0,0 +1,442 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include <string.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <inttypes.h>
+#include <errno.h>
+#include <sys/queue.h>
+
+#include <rte_common.h>
+#include <rte_log.h>
+#include <rte_memory.h>
+#include <rte_launch.h>
+#include <rte_cycles.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_branch_prediction.h>
+#include <rte_malloc.h>
+#include <rte_random.h>
+#include <rte_errno.h>
+#include <rte_hexdump.h>
+
+#include <rte_soring.h>
+
+#include "test.h"
+
+#define MAX_ACQUIRED 20
+
+#define SORING_TEST_ASSERT(val, expected) do { \
+	RTE_TEST_ASSERT(expected == val, \
+			"%s: expected %u got %u\n", #val, expected, val); \
+} while (0)
+
+static void
+set_soring_init_param(struct rte_soring_param *prm,
+		const char *name, uint32_t esize, uint32_t elems,
+		uint32_t stages, uint32_t stsize,
+		enum rte_ring_sync_type rst_prod,
+		enum rte_ring_sync_type rst_cons)
+{
+	prm->name = name;
+	prm->elem_size = esize;
+	prm->elems = elems;
+	prm->stages = stages;
+	prm->meta_size = stsize;
+	prm->prod_synt = rst_prod;
+	prm->cons_synt = rst_cons;
+}
+
+static int
+move_forward_stage(struct rte_soring *sor,
+		uint32_t num_packets, uint32_t stage)
+{
+	uint32_t acquired;
+	uint32_t ftoken;
+	uint32_t *acquired_objs[MAX_ACQUIRED];
+
+	acquired = rte_soring_acquire_bulk(sor, acquired_objs, stage,
+			num_packets, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, num_packets);
+	rte_soring_release(sor, NULL, stage, num_packets,
+			ftoken);
+
+	return 0;
+}
+
+/*
+ * struct rte_soring_param param checking.
+ */
+static int
+test_soring_init(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	size_t sz;
+	memset(&prm, 0, sizeof(prm));
+
+/*init memory*/
+	set_soring_init_param(&prm, "alloc_memory", sizeof(uintptr_t),
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "could not allocate memory for soring");
+
+	set_soring_init_param(&prm, "test_invalid_stages", sizeof(uintptr_t),
+			4, 0, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num stages");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 0,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with 0 esize");
+
+	set_soring_init_param(&prm, "test_invalid_esize", 9,
+			4, 1, 4, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with esize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_rsize", sizeof(uintptr_t),
+			4, 1, 3, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with rcsize not multiple of 4");
+
+	set_soring_init_param(&prm, "test_invalid_elems", sizeof(uintptr_t),
+			RTE_SORING_ELEM_MAX + 1, 1, 4, RTE_RING_SYNC_MT,
+			RTE_RING_SYNC_MT);
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_FAIL(rc, "initted soring with invalid num elements");
+
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_get_memsize(void)
+{
+
+	struct rte_soring_param prm;
+	ssize_t sz;
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(sz > 0, "failed to calculate size");
+
+	set_soring_init_param(&prm, "memsize", sizeof(uint8_t),
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	set_soring_init_param(&prm, "memsize", 0,
+			16, UINT32_MAX, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT_EQUAL(sz, -EINVAL, "calculated size incorrect");
+
+	return 0;
+
+}
+
+static int
+test_soring_stages(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	uint32_t objs[32];
+	uint32_t rcs[32];
+	uint32_t acquired_objs[32];
+	uint32_t acquired_rcs[32];
+	uint32_t dequeued_rcs[32];
+	uint32_t dequeued_objs[32];
+	size_t ssz;
+	uint32_t stage, enqueued, dequeued, acquired;
+	uint32_t i, ftoken;
+
+	memset(&prm, 0, sizeof(prm));
+	set_soring_init_param(&prm, "stages", sizeof(uint32_t), 32,
+			10000, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	ssz = rte_soring_get_memsize(&prm);
+	RTE_TEST_ASSERT(ssz > 0, "parameter error calculating ring size");
+	sor = rte_zmalloc(NULL, ssz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "couldn't allocate memory for soring");
+	rte_soring_init(sor, &prm);
+
+	for (i = 0; i < 32; i++) {
+		rcs[i] = i;
+		objs[i] = i + 32;
+	}
+
+	enqueued = rte_soring_enqueux_burst(sor, objs, rcs, 32, NULL);
+	SORING_TEST_ASSERT(enqueued, 32);
+
+	for (stage = 0; stage < 10000; stage++) {
+		int j;
+		dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs,
+				32, NULL);
+		/* check none at end stage */
+		SORING_TEST_ASSERT(dequeued, 0);
+
+		acquired = rte_soring_acquirx_bulk(sor, acquired_objs,
+				acquired_rcs, stage, 32, &ftoken, NULL);
+		SORING_TEST_ASSERT(acquired, 32);
+
+		for (j = 0; j < 32; j++) {
+			SORING_TEST_ASSERT(acquired_rcs[j], j + stage);
+			SORING_TEST_ASSERT(acquired_objs[j], j + stage + 32);
+			/* modify both queue object and rc */
+			acquired_objs[j]++;
+			acquired_rcs[j]++;
+		}
+
+		rte_soring_releasx(sor, acquired_objs,
+				acquired_rcs, stage, 32,
+				ftoken);
+	}
+
+	dequeued = rte_soring_dequeux_bulk(sor, dequeued_objs, dequeued_rcs,
+			32, NULL);
+	SORING_TEST_ASSERT(dequeued, 32);
+		for (i = 0; i < 32; i++) {
+			/* ensure we ended up with the expected values in order*/
+			SORING_TEST_ASSERT(dequeued_rcs[i], i + 10000);
+			SORING_TEST_ASSERT(dequeued_objs[i], i + 32 + 10000);
+		}
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_enqueue_dequeue(void)
+{
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc;
+	uint32_t i;
+	size_t sz;
+	uint32_t queue_objs[10];
+	uint32_t *queue_objs_p[10];
+	uint32_t free_space;
+	uint32_t enqueued, dequeued;
+	uint32_t *dequeued_objs[10];
+
+	memset(&prm, 0, sizeof(prm));
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 1;
+		queue_objs_p[i] = &queue_objs[i];
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "enqueue/dequeue", sizeof(uint32_t *),
+			10, 1, 0, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	RTE_TEST_ASSERT_NOT_NULL(sor, "alloc failed for soring");
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "Failed to init soring");
+
+	free_space = 0;
+
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p, 5, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* fixed amount enqueue */
+	enqueued = rte_soring_enqueue_bulk(sor, queue_objs_p, 7, &free_space);
+
+	SORING_TEST_ASSERT(free_space, 5);
+	SORING_TEST_ASSERT(enqueued, 0);
+
+	/* variable amount enqueue */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs_p + 5, 7,
+				&free_space);
+	SORING_TEST_ASSERT(free_space, 0);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* test no dequeue while stage 0 has not completed */
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 8, 0);
+
+	/* can only dequeue as many as have completed stage*/
+	dequeued = rte_soring_dequeue_bulk(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 10, NULL);
+	SORING_TEST_ASSERT(dequeued, 8);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i], "dequeued != enqueued");
+	}
+
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 1, NULL);
+	SORING_TEST_ASSERT(dequeued, 0);
+
+	move_forward_stage(sor, 2, 0);
+	dequeued = rte_soring_dequeue_burst(sor, dequeued_objs, 2, NULL);
+	SORING_TEST_ASSERT(dequeued, 2);
+	/* packets remain in order */
+	for (i = 0; i < dequeued; i++) {
+		RTE_TEST_ASSERT_EQUAL(dequeued_objs[i],
+				queue_objs_p[i + 8], "dequeued != enqueued");
+	}
+
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring_acquire_release(void)
+{
+
+	struct rte_soring *sor = NULL;
+	struct rte_soring_param prm;
+	int rc, i;
+	size_t sz;
+
+	memset(&prm, 0, sizeof(prm));
+	uint32_t queue_objs[10];
+	uint32_t rc_objs[10];
+	uint32_t acquired_objs[10];
+	uint32_t dequeued_objs[10];
+	uint32_t dequeued_rcs[10];
+	uint32_t s1_acquired_rcs[10];
+	uint32_t free_space, enqueued, ftoken, acquired, dequeued;
+
+	for (i = 0; i < 10; i++) {
+		queue_objs[i] = i + 5;
+		rc_objs[i] = i + 10;
+	}
+
+/*init memory*/
+	set_soring_init_param(&prm, "test_acquire_release", sizeof(uint32_t),
+			20, 2, sizeof(uint32_t), RTE_RING_SYNC_MT, RTE_RING_SYNC_MT);
+	sz = rte_soring_get_memsize(&prm);
+	sor = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (sor == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	/* init ring */
+	rc = rte_soring_init(sor, &prm);
+	RTE_TEST_ASSERT_SUCCESS(rc, "failed to init soring");
+
+	/* enqueue with associated rc */
+	enqueued = rte_soring_enqueux_burst(sor, queue_objs, rc_objs, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+	/* enqueue without associated rc */
+	enqueued = rte_soring_enqueue_burst(sor, queue_objs + 5, 5,
+			&free_space);
+	SORING_TEST_ASSERT(enqueued, 5);
+
+	/* acquire the objects with rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_release(sor, NULL, 0, 5, ftoken);
+
+	/* acquire the objects without rc's and ensure they are as expected */
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 0, 5, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 5);
+	for (i = 0; i < 5; i++) {
+		/* as the rc area of memory is zero'd at init this is true
+		 * but this is a detail of implementation rather than
+		 * a guarantee.
+		 */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], 0,
+				"acquired rc not empty");
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i + 5],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i + 5]);
+	}
+	/*release the objects, adding rc's */
+	rte_soring_releasx(sor, NULL, rc_objs + 5, 0, 5,
+			ftoken);
+
+	acquired = rte_soring_acquirx_burst(sor, acquired_objs,
+			s1_acquired_rcs, 1, 10, &ftoken, NULL);
+	SORING_TEST_ASSERT(acquired, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at release */
+		RTE_TEST_ASSERT_EQUAL(s1_acquired_rcs[i], rc_objs[i],
+				"acquired rc[%d]: %u != enqueued rc: %u",
+				i, s1_acquired_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	/*release the objects, with rc's set to NULL */
+	rte_soring_release(sor, NULL, 1, 10, ftoken);
+
+	dequeued = rte_soring_dequeux_burst(sor, dequeued_objs, dequeued_rcs,
+			10, NULL);
+	SORING_TEST_ASSERT(dequeued, 10);
+	for (i = 0; i < 10; i++) {
+		/* ensure the associated rc's are the ones added at release */
+		RTE_TEST_ASSERT_EQUAL(dequeued_rcs[i], rc_objs[i],
+				"dequeued rc[%d]: %u != enqueued rc: %u",
+				i, dequeued_rcs[i], rc_objs[i]);
+		RTE_TEST_ASSERT_EQUAL(acquired_objs[i], queue_objs[i],
+				"acquired obj[%d]: %u != enqueued obj %u",
+				i, acquired_objs[i], queue_objs[i]);
+	}
+	rte_soring_dump(stdout, sor);
+	rte_free(sor);
+	return 0;
+}
+
+static int
+test_soring(void)
+{
+
+	/* Negative test cases */
+	if (test_soring_init() < 0)
+		goto test_fail;
+
+	/* Memory calculations */
+	if (test_soring_get_memsize() < 0)
+		goto test_fail;
+
+	/* Basic enqueue/dequeue operations */
+	if (test_soring_enqueue_dequeue() < 0)
+		goto test_fail;
+
+	/* Acquire/release */
+	if (test_soring_acquire_release() < 0)
+		goto test_fail;
+
+	/* Test large number of stages */
+	if (test_soring_stages() < 0)
+		goto test_fail;
+
+	return 0;
+
+test_fail:
+	return -1;
+}
+
+REGISTER_FAST_TEST(soring_autotest, true, true, test_soring);
+
diff --git a/app/test/test_soring_mt_stress.c b/app/test/test_soring_mt_stress.c
new file mode 100644
index 0000000000..2f90bb4598
--- /dev/null
+++ b/app/test/test_soring_mt_stress.c
@@ -0,0 +1,40 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress_impl.h"
+
+static inline uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail)
+{
+	return rte_soring_dequeue_burst(r, obj, n, avail);
+}
+
+static inline uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free)
+{
+	return rte_soring_enqueue_bulk(r, obj, n, free);
+}
+
+static inline uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail)
+{
+	return rte_soring_acquire_burst(r, obj, stage, num, token, avail);
+}
+
+static inline void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num)
+{
+	RTE_SET_USED(obj);
+	rte_soring_release(r, NULL, stage, num, token);
+}
+
+const struct test test_soring_mt_stress = {
+	.name = "MT",
+	.nb_case = RTE_DIM(tests),
+	.cases = tests,
+};
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
new file mode 100644
index 0000000000..334af6a29c
--- /dev/null
+++ b/app/test/test_soring_stress.c
@@ -0,0 +1,48 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+#include "test_soring_stress.h"
+
+static int
+run_test(const struct test *test)
+{
+	int32_t rc;
+	uint32_t i, k;
+
+	for (i = 0, k = 0; i != test->nb_case; i++) {
+
+		printf("TEST-CASE %s %s START\n",
+			test->name, test->cases[i].name);
+
+		rc = test->cases[i].func(test->cases[i].wfunc);
+		k += (rc == 0);
+
+		if (rc != 0)
+			printf("TEST-CASE %s %s FAILED\n",
+				test->name, test->cases[i].name);
+		else
+			printf("TEST-CASE %s %s OK\n",
+				test->name, test->cases[i].name);
+	}
+
+	return k;
+}
+
+static int
+test_ring_stress(void)
+{
+	uint32_t n, k;
+
+	n = 0;
+	k = 0;
+
+	n += test_soring_mt_stress.nb_case;
+	k += run_test(&test_soring_mt_stress);
+
+	printf("Number of tests:\t%u\nSuccess:\t%u\nFailed:\t%u\n",
+		n, k, n - k);
+	return (k != n);
+}
+
+REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.h b/app/test/test_soring_stress.h
new file mode 100644
index 0000000000..6190e96117
--- /dev/null
+++ b/app/test/test_soring_stress.h
@@ -0,0 +1,35 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ * Copyright(c) 2024 Huawei Technologies Co., Ltd
+ */
+
+
+#include <inttypes.h>
+#include <stddef.h>
+#include <stdalign.h>
+#include <string.h>
+#include <stdio.h>
+#include <unistd.h>
+
+#include <rte_soring.h>
+#include <rte_cycles.h>
+#include <rte_launch.h>
+#include <rte_pause.h>
+#include <rte_random.h>
+#include <rte_malloc.h>
+#include <rte_spinlock.h>
+
+#include "test.h"
+
+struct test_case {
+	const char *name;
+	int (*func)(int (*)(void *));
+	int (*wfunc)(void *arg);
+};
+
+struct test {
+	const char *name;
+	uint32_t nb_case;
+	const struct test_case *cases;
+};
+
+extern const struct test test_soring_mt_stress;
diff --git a/app/test/test_soring_stress_impl.h b/app/test/test_soring_stress_impl.h
new file mode 100644
index 0000000000..5e80b6dd02
--- /dev/null
+++ b/app/test/test_soring_stress_impl.h
@@ -0,0 +1,838 @@
+/* SPDX-License-Identifier: BSD-3-Clause
+ */
+
+#include <stdalign.h>
+
+#include "test_soring_stress.h"
+
+/**
+ * Stress test for soring enqueue/dequeue/acquire/release operations.
+ * Depending on the role, performs at least one of the following patterns
+ * on each worker:
+ * - dequeue/read-write data from/to the dequeued objects/enqueue.
+ * - acquire/read-write data from/to the acquired objects/release.
+ * Serves as both functional and performance test of soring
+ * data-path API under high contention
+ * (for both over committed and non-over committed scenarios).
+ */
+
+#define RING_NAME	"SORING_STRESS"
+#define BULK_NUM	32
+#define RING_SIZE	(2 * BULK_NUM * RTE_MAX_LCORE)
+
+#define MAX_STAGES	16
+
+enum {
+	WRK_CMD_STOP,
+	WRK_CMD_RUN,
+};
+
+static RTE_ATOMIC(uint32_t) wrk_cmd __rte_cache_aligned = WRK_CMD_STOP;
+
+/* test run-time in seconds */
+static const uint32_t run_time = 60;
+static const uint32_t verbose;
+
+static rte_spinlock_t dump_lock;
+
+struct lcore_op_stat {
+	uint64_t nb_lcore;
+	uint64_t nb_call;
+	uint64_t nb_obj;
+	uint64_t nb_cycle;
+	uint64_t max_cycle;
+	uint64_t min_cycle;
+};
+
+struct lcore_stat {
+	uint64_t nb_cycle;
+	struct lcore_op_stat deqenq;
+	uint32_t role_mask;
+	uint32_t nb_stage;
+	struct lcore_op_stat stage[MAX_STAGES];
+};
+
+#define	ROLE_DEQENQ	RTE_BIT32(0)
+#define	ROLE_STAGE(n)	RTE_BIT32(n + 1)
+
+struct lcore_arg {
+	struct rte_soring *rng;
+	struct lcore_stat stats;
+} __rte_cache_aligned;
+
+struct ring_elem {
+	uint32_t cnt[RTE_CACHE_LINE_SIZE / sizeof(uint32_t)];
+} __rte_cache_aligned;
+
+/*
+ * redefinable functions
+ */
+
+static uint32_t
+_st_ring_dequeue_burst(struct rte_soring *r, void **obj, uint32_t n,
+	uint32_t *avail);
+
+static uint32_t
+_st_ring_enqueue_bulk(struct rte_soring *r, void * const *obj, uint32_t n,
+	uint32_t *free);
+
+static uint32_t
+_st_ring_acquire_burst(struct rte_soring *r, uint32_t stage, void **obj,
+	uint32_t num, uint32_t *token, uint32_t *avail);
+
+static void
+_st_ring_release(struct rte_soring *r, uint32_t stage, uint32_t token,
+	void * const *obj, uint32_t num);
+
+static void
+lcore_op_stat_update(struct lcore_op_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	ls->nb_call += call;
+	ls->nb_obj += obj;
+	ls->nb_cycle += tm;
+	if (prcs) {
+		ls->max_cycle = RTE_MAX(ls->max_cycle, tm);
+		ls->min_cycle = RTE_MIN(ls->min_cycle, tm);
+	}
+}
+
+static void
+lcore_stat_update(struct lcore_stat *ls, uint64_t call, uint64_t obj,
+	uint64_t tm, int32_t prcs)
+{
+	uint32_t i;
+
+	ls->nb_cycle += tm;
+	lcore_op_stat_update(&ls->deqenq, call, obj, tm, prcs);
+	for (i = 0; i != ls->nb_stage; i++)
+		lcore_op_stat_update(ls->stage + i, call, obj, tm, prcs);
+}
+
+static void
+lcore_op_stat_aggr(struct lcore_op_stat *ms, const struct lcore_op_stat *ls)
+{
+	ms->nb_call += ls->nb_call;
+	ms->nb_obj += ls->nb_obj;
+	ms->nb_cycle += ls->nb_cycle;
+	ms->max_cycle = RTE_MAX(ms->max_cycle, ls->max_cycle);
+	ms->min_cycle = RTE_MIN(ms->min_cycle, ls->min_cycle);
+}
+
+static void
+lcore_stat_aggr(struct lcore_stat *ms, const struct lcore_stat *ls)
+{
+	uint32_t i;
+
+	ms->nb_cycle = RTE_MAX(ms->nb_cycle, ls->nb_cycle);
+	lcore_op_stat_aggr(&ms->deqenq, &ls->deqenq);
+	ms->deqenq.nb_lcore += ((ls->role_mask & ROLE_DEQENQ) != 0);
+	for (i = 0; i != ms->nb_stage; i++) {
+		lcore_op_stat_aggr(ms->stage + i, ls->stage + i);
+		ms->stage[i].nb_lcore += ((ls->role_mask & ROLE_STAGE(i)) != 0);
+	}
+}
+
+static void
+lcore_op_stat_dump(FILE *f, const struct lcore_op_stat *ls, const char *cap,
+	long double st)
+{
+	fprintf(f, "\t%s={\n", cap);
+
+	fprintf(f, "\t\tnb_lcore=%" PRIu64 ",\n", ls->nb_lcore);
+	fprintf(f, "\t\tnb_call=%" PRIu64 ",\n", ls->nb_call);
+	fprintf(f, "\t\tnb_obj=%" PRIu64 ",\n", ls->nb_obj);
+	fprintf(f, "\t\tnb_cycle=%" PRIu64 ",\n", ls->nb_cycle);
+	fprintf(f, "\t\tobj/call(avg): %.2Lf\n",
+		(long double)ls->nb_obj / ls->nb_call);
+	fprintf(f, "\t\tcycles/obj(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_obj);
+	fprintf(f, "\t\tcycles/call(avg): %.2Lf\n",
+		(long double)ls->nb_cycle / ls->nb_call);
+
+	/* if min/max cycles per call stats was collected */
+	if (ls->min_cycle != UINT64_MAX) {
+		fprintf(f, "\t\tmax cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->max_cycle,
+			(long double)ls->max_cycle / st);
+		fprintf(f, "\t\tmin cycles/call=%" PRIu64 "(%.2Lf usec),\n",
+			ls->min_cycle,
+			(long double)ls->min_cycle / st);
+	}
+
+	fprintf(f, "\t},\n");
+}
+
+static void
+lcore_stat_dump(FILE *f, uint32_t lc, const struct lcore_stat *ls)
+{
+	uint32_t i;
+	long double st;
+	char cap[64];
+
+	st = (long double)rte_get_timer_hz() / US_PER_S;
+
+	if (lc == UINT32_MAX)
+		fprintf(f, "%s(AGGREGATE)={\n", __func__);
+	else
+		fprintf(f, "%s(lcore=%u)={\n", __func__, lc);
+
+	fprintf(f, "\tnb_cycle=%" PRIu64 "(%.2Lf usec),\n",
+		ls->nb_cycle, (long double)ls->nb_cycle / st);
+
+	lcore_op_stat_dump(f, &ls->deqenq, "DEQ+ENQ", st);
+	for (i = 0; i != ls->nb_stage; i++) {
+		snprintf(cap, sizeof(cap), "%s#%u", "STAGE", i);
+		lcore_op_stat_dump(f, ls->stage + i, cap, st);
+	}
+
+	fprintf(f, "};\n");
+}
+
+static void
+fill_ring_elm(struct ring_elem *elm, uint32_t fill)
+{
+	uint32_t i;
+
+	for (i = 0; i != RTE_DIM(elm->cnt); i++)
+		elm->cnt[i] = fill;
+}
+
+static int32_t
+check_updt_elem(struct ring_elem *elm[], uint32_t num,
+	const struct ring_elem *check, const struct ring_elem *fill,
+	const char *fname, const char *opname, const struct rte_soring *sor)
+{
+	uint32_t i;
+
+	for (i = 0; i != num; i++) {
+		if (memcmp(check, elm[i], sizeof(*check)) != 0) {
+			rte_spinlock_lock(&dump_lock);
+			printf("%s:%s: %s(lc=%u, num=%u) failed at %u-th iter, "
+				"offending object: %p\n",
+				fname, opname, __func__, rte_lcore_id(), num, i,
+				elm[i]);
+			rte_memdump(stdout, "expected", check, sizeof(*check));
+			rte_memdump(stdout, "result", elm[i], sizeof(*elm[i]));
+			rte_soring_dump(stdout, sor);
+			rte_spinlock_unlock(&dump_lock);
+			return -EINVAL;
+		}
+		memcpy(elm[i], fill, sizeof(*elm[i]));
+	}
+
+	return 0;
+}
+
+static int
+check_ring_op(uint32_t exp, uint32_t res, uint32_t lc,
+	enum rte_ring_queue_behavior bhv, const char *fname, const char *opname,
+	const struct rte_soring *sor)
+{
+	if ((bhv == RTE_RING_QUEUE_FIXED && exp != res) ||
+			(bhv == RTE_RING_QUEUE_VARIABLE && exp < res)) {
+		rte_spinlock_lock(&dump_lock);
+		printf("%s(lc=%u) failure: %s expected: %u, returned %u\n",
+			fname, lc, opname, exp, res);
+		rte_soring_dump(stdout, sor);
+		rte_spinlock_unlock(&dump_lock);
+		return -ENOSPC;
+	}
+	return 0;
+}
+
+/* num in interval [7/8, 11/8] of BULK_NUM */
+static inline uint32_t
+rand_elem_num(void)
+{
+	uint32_t num;
+
+	num = 7 * BULK_NUM / 8 + rte_rand() % (BULK_NUM / 2);
+	return num;
+}
+
+/*
+ * for each enabled stage do:
+ *   acquire burst of objects
+ *   read and check their contents
+ *   update and check their contents
+ *   release burst of objects
+ * done
+ */
+static int32_t
+test_worker_stages(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem stg_elm[MAX_STAGES], int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, n, num, tkn;
+	uint64_t tm0, tm1;
+	const struct ring_elem *celm, *pelm;
+
+	num = rand_elem_num();
+
+	rc = 0;
+	tkn = 0;
+	for (i = 0, pelm = def_elm; i != la->stats.nb_stage; pelm = celm, i++) {
+
+		celm = stg_elm + i;
+
+		/* given stage is not enabled on that lcore */
+		if ((la->stats.role_mask & ROLE_STAGE(i)) == 0)
+			continue;
+
+		/* reset all pointer values */
+		memset(obj, 0, sizeof(*obj) * num);
+
+		/* acquire num elems */
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+		n = _st_ring_acquire_burst(la->rng, i, (void **)obj, num,
+				&tkn, NULL);
+		tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+		/* check return value and objects */
+		rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc == 0)
+			rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+				RTE_STR(_st_ring_stage_acquire), la->rng);
+		if (rc != 0)
+			break;
+
+		/* release num elems */
+		rte_compiler_barrier();
+		rc = check_updt_elem(obj, n, loc_elm, celm, fname,
+			RTE_STR(_st_ring_stage_release), la->rng);
+		if (rc != 0)
+			break;
+
+		if (n == 0)
+			tm1 = 0;
+		else {
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+			_st_ring_release(la->rng, i, tkn,
+					(void **)obj, n);
+			tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+		}
+		lcore_op_stat_update(la->stats.stage + i, 1, n, tm0 + tm1,
+				prcs);
+	}
+
+	return rc;
+}
+
+static int32_t
+test_worker_deqenq(struct lcore_arg *la, uint32_t lc, const char *fname,
+	struct ring_elem *obj[2 * BULK_NUM],
+	const struct ring_elem *def_elm, const struct ring_elem *loc_elm,
+	const struct ring_elem *pelm, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t k, n, num;
+	uint64_t tm0, tm1;
+
+	num = rand_elem_num();
+
+	/* reset all pointer values */
+	memset(obj, 0, sizeof(*obj) * num);
+
+	/* dequeue num elems */
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	n = _st_ring_dequeue_burst(la->rng, (void **)obj, num, NULL);
+
+	tm0 = (prcs != 0) ? rte_rdtsc_precise() - tm0 : 0;
+
+	/* check return value and objects */
+	rc = check_ring_op(num, n, lc, RTE_RING_QUEUE_VARIABLE, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc == 0)
+		rc = check_updt_elem(obj, n, pelm, loc_elm, fname,
+			RTE_STR(_st_ring_dequeue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	/* enqueue n elems */
+	rte_compiler_barrier();
+	rc = check_updt_elem(obj, n, loc_elm, def_elm, fname,
+		RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() : 0;
+	k = _st_ring_enqueue_bulk(la->rng, (void **)obj, n, NULL);
+	tm1 = (prcs != 0) ? rte_rdtsc_precise() - tm1 : 0;
+
+	/* check return value */
+	rc = check_ring_op(n, k, lc, RTE_RING_QUEUE_FIXED, fname,
+			RTE_STR(_st_ring_enqueue_bulk), la->rng);
+	if (rc != 0)
+		return rc;
+
+	lcore_op_stat_update(&la->stats.deqenq, 1, n, tm0 + tm1, prcs);
+	return 0;
+}
+
+static int
+test_worker(void *arg, const char *fname, int32_t prcs)
+{
+	int32_t rc;
+	uint32_t i, lc;
+	uint64_t cl;
+	struct lcore_arg *la;
+	struct ring_elem *obj[2 * BULK_NUM];
+	struct ring_elem *pelm, def_elm, loc_elm, stg_elm[MAX_STAGES];
+
+	la = arg;
+	lc = rte_lcore_id();
+
+	fill_ring_elm(&def_elm, UINT32_MAX);
+	fill_ring_elm(&loc_elm, lc);
+
+	for (i = 0; i != RTE_DIM(stg_elm); i++)
+		fill_ring_elm(stg_elm + i, (i + 1) << 24);
+
+	pelm = stg_elm + la->stats.nb_stage - 1;
+
+	/* Acquire ordering is not required as the main is not
+	 * really releasing any data through 'wrk_cmd' to
+	 * the worker.
+	 */
+	while (rte_atomic_load_explicit(&wrk_cmd, rte_memory_order_relaxed) !=
+			WRK_CMD_RUN)
+		rte_pause();
+
+	cl = rte_rdtsc_precise();
+
+	do {
+		if ((la->stats.role_mask & ~ROLE_DEQENQ) != 0) {
+			rc = test_worker_stages(la, lc, fname, obj,
+				&def_elm, &loc_elm, stg_elm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+		if ((la->stats.role_mask & ROLE_DEQENQ) != 0) {
+			rc = test_worker_deqenq(la, lc, fname, obj,
+				&def_elm, &loc_elm, pelm, prcs);
+			if (rc != 0)
+				break;
+		}
+
+	} while (rte_atomic_load_explicit(&wrk_cmd,
+				rte_memory_order_relaxed) == WRK_CMD_RUN);
+
+	cl = rte_rdtsc_precise() - cl;
+	if (prcs == 0)
+		lcore_stat_update(&la->stats, 0, 0, cl, 0);
+	la->stats.nb_cycle = cl;
+	return rc;
+}
+static int
+test_worker_prcs(void *arg)
+{
+	return test_worker(arg, __func__, 1);
+}
+
+static int
+test_worker_avg(void *arg)
+{
+	return test_worker(arg, __func__, 0);
+}
+
+static void
+mt1_fini(struct rte_soring *rng, void *data)
+{
+	rte_free(rng);
+	rte_free(data);
+}
+
+static int
+mt1_init(struct rte_soring **rng, void **data, uint32_t num,
+	enum rte_ring_sync_type prod_synt, enum rte_ring_sync_type cons_synt,
+	uint32_t nb_stages)
+{
+	int32_t rc;
+	size_t sz;
+	uint32_t i;
+	struct rte_soring *r;
+	struct ring_elem *elm;
+	void *p;
+	struct rte_soring_param prm;
+
+	*rng = NULL;
+	*data = NULL;
+
+	sz = num * sizeof(*elm);
+	elm = rte_zmalloc(NULL, sz, alignof(typeof(*elm)));
+	if (elm == NULL) {
+		printf("%s: alloc(%zu) for %u elems data failed",
+			__func__, sz, num);
+		return -ENOMEM;
+	}
+
+	*data = elm;
+
+	/* alloc soring */
+	memset(&prm, 0, sizeof(prm));
+
+	prm.name = __func__;
+	prm.elems = num;
+	prm.elem_size = sizeof(uintptr_t);
+	prm.stages = nb_stages;
+	prm.prod_synt = prod_synt;
+	prm.cons_synt = cons_synt;
+
+	sz = rte_soring_get_memsize(&prm);
+	r = rte_zmalloc(NULL, sz, RTE_CACHE_LINE_SIZE);
+	if (r == NULL) {
+		printf("%s: alloc(%zu) for FIFO with %u elems failed",
+			__func__, sz, prm.elems);
+		return -ENOMEM;
+	}
+
+	*rng = r;
+
+	rc = rte_soring_init(r, &prm);
+	if (rc != 0) {
+		printf("%s: rte_soring_init(r=%p,elems=%u,stages=%u) failed, "
+			"error: %d(%s)\n",
+			__func__, r, prm.elems, prm.stages, rc, strerror(-rc));
+		return rc;
+	}
+
+	for (i = 0; i != num; i++) {
+		fill_ring_elm(elm + i, UINT32_MAX);
+		p = elm + i;
+		if (_st_ring_enqueue_bulk(r, &p, 1, NULL) != 1)
+			break;
+	}
+
+	if (i != num) {
+		printf("%s: _st_ring_enqueue(%p, %u) returned %u\n",
+			__func__, r, num, i);
+		return -ENOSPC;
+	}
+
+	return 0;
+}
+
+static int
+test_mt(int (*test)(void *), enum rte_ring_sync_type prod_synt,
+	enum rte_ring_sync_type cons_synt, uint32_t nb_stage,
+	const uint32_t role_mask[RTE_MAX_LCORE])
+{
+	int32_t rc;
+	uint32_t i, lc, mc;
+	struct rte_soring *r;
+	void *data;
+	struct lcore_arg arg[RTE_MAX_LCORE];
+
+	static const struct lcore_op_stat init_stat = {
+		.min_cycle = UINT64_MAX,
+	};
+
+	rc = mt1_init(&r, &data, RING_SIZE, prod_synt, cons_synt, nb_stage);
+
+	if (rc != 0) {
+		mt1_fini(r, data);
+		return rc;
+	}
+
+	memset(arg, 0, sizeof(arg));
+
+	/* launch on all workers */
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		arg[lc].rng = r;
+		arg[lc].stats.deqenq = init_stat;
+		arg[lc].stats.nb_stage = nb_stage;
+		arg[lc].stats.role_mask = role_mask[lc];
+		for (i = 0; i != arg[lc].stats.nb_stage; i++)
+			arg[lc].stats.stage[i] = init_stat;
+		rte_eal_remote_launch(test, &arg[lc], lc);
+	}
+
+	/* signal workers to start test */
+	rte_atomic_store_explicit(&wrk_cmd, WRK_CMD_RUN,
+			rte_memory_order_release);
+
+	rte_delay_us(run_time * US_PER_S);
+
+	/* signal workers to stop test */
+	rte_atomic_store_explicit(&wrk_cmd, WRK_CMD_STOP,
+			rte_memory_order_release);
+
+	/* wait for workers and collect stats. */
+	mc = rte_lcore_id();
+	arg[mc].stats.deqenq = init_stat;
+	arg[mc].stats.nb_stage = nb_stage;
+	for (i = 0; i != arg[mc].stats.nb_stage; i++)
+		arg[mc].stats.stage[i] = init_stat;
+
+	rc = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		rc |= rte_eal_wait_lcore(lc);
+		lcore_stat_aggr(&arg[mc].stats, &arg[lc].stats);
+		if (verbose != 0)
+			lcore_stat_dump(stdout, lc, &arg[lc].stats);
+	}
+
+	lcore_stat_dump(stdout, UINT32_MAX, &arg[mc].stats);
+	rte_soring_dump(stdout, r);
+	mt1_fini(r, data);
+	return rc;
+}
+
+/*
+ * launch all stages and deq+enq on all worker lcores
+ */
+static void
+role_mask_sym(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t lc;
+	const uint32_t msk =  RTE_BIT32(nb_stage + 2) - 1;
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+	RTE_LCORE_FOREACH_WORKER(lc)
+		role_mask[lc] = msk;
+}
+
+/*
+ * Divide all workers in two (nearly) equal groups:
+ * - workers from 'even' group do deque+enque
+ * - workers from 'odd' group do acquire/release (for all stages)
+ */
+static void
+role_mask_even_odd(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i & 1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[i & 1];
+	}
+}
+
+/*
+ * Divide all workers (nearly) evenly among all possible stages
+ */
+static void
+role_mask_div(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	uint32_t msk[nb_stage + 1];
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	for (i = 0; i != RTE_DIM(msk); i++) {
+		msk[i] = RTE_BIT32(i);
+	};
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		role_mask[lc] = msk[i % RTE_DIM(msk)];
+		i++;
+	}
+	if (i < RTE_DIM(msk)) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		for (; i != RTE_DIM(msk); i++)
+			role_mask[lc] |= msk[i % RTE_DIM(msk)];
+	}
+}
+
+/*
+ * one worker does ST enqueue+dequeue, while all others - stages processing.
+ */
+static void
+role_mask_denq_st(uint32_t nb_stage, uint32_t role_mask[RTE_MAX_LCORE])
+{
+	uint32_t i, lc;
+	const uint32_t msk[2] = {
+		[0] = ROLE_DEQENQ,
+		[1] =  RTE_GENMASK32(nb_stage + 1, 1),
+	};
+
+	memset(role_mask, 0, sizeof(role_mask[0]) * RTE_MAX_LCORE);
+
+	i = 0;
+	RTE_LCORE_FOREACH_WORKER(lc) {
+		if (i == 0)
+			role_mask[lc] = msk[0];
+		else
+			role_mask[lc] = msk[1];
+		i++;
+	}
+	if (i == 1) {
+		lc = rte_get_next_lcore(-1, 1, 0);
+		role_mask[lc] |= msk[1];
+	}
+}
+
+
+static int
+test_sym_mt1(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+	const uint32_t nb_stage = 1;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_rts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_RTS, RTE_RING_SYNC_MT_RTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_sym_mt_hts4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_sym(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT_HTS, RTE_RING_SYNC_MT_HTS,
+			nb_stage, role_mask);
+}
+
+static int
+test_stdenq_stage4(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 4;
+
+	role_mask_denq_st(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_ST, RTE_RING_SYNC_ST,
+			nb_stage, role_mask);
+}
+
+
+static int
+test_even_odd_mt5(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 5;
+
+	role_mask_even_odd(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static int
+test_div_mt3(int (*test)(void *))
+{
+	uint32_t role_mask[RTE_MAX_LCORE];
+
+	const uint32_t nb_stage = 3;
+
+	role_mask_div(nb_stage, role_mask);
+	return test_mt(test, RTE_RING_SYNC_MT, RTE_RING_SYNC_MT,
+			nb_stage, role_mask);
+}
+
+static const struct test_case tests[] = {
+	{
+		.name = "MT_DEQENQ-MT_STG1-PRCS",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG1-AVG",
+		.func = test_sym_mt1,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTRTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_rts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-PRCS",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MTHTS_DEQENQ-MT_STG4-AVG",
+		.func = test_sym_mt_hts4,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-PRCS",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG5-1:1-AVG",
+		.func = test_even_odd_mt5,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "MT_DEQENQ-MT_STG3-1:3-PRCS",
+		.func = test_div_mt3,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "MT_DEQENQ_MT_STG3-1:3-AVG",
+		.func = test_div_mt3,
+		.wfunc = test_worker_avg,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-PRCS",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_prcs,
+	},
+	{
+		.name = "ST_DEQENQ-MT_STG4-AVG",
+		.func = test_stdenq_stage4,
+		.wfunc = test_worker_avg,
+	},
+};
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

* [PATCH v10 7/7] test: add stress test suite
  2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
                                     ` (5 preceding siblings ...)
  2024-11-11 14:19                   ` [PATCH v10 6/7] app/test: add unit tests for soring API Konstantin Ananyev
@ 2024-11-11 14:19                   ` Konstantin Ananyev
  6 siblings, 0 replies; 101+ messages in thread
From: Konstantin Ananyev @ 2024-11-11 14:19 UTC (permalink / raw)
  To: dev
  Cc: honnappa.nagarahalli, jerinj, hemant.agrawal, bruce.richardson,
	drc, ruifeng.wang, mb, eimear.morrissey, stephen

Add a new test suite which purpose is to run 'stress' tests:
main purpose is put a pressure to dpdk sync algorithms
to flag their misbehaving/slowdown/etc.
Right now it consists from just 2 test-cases:
meson test --suite stress-tests --list
DPDK:stress-tests / ring_stress_autotest
DPDK:stress-tests / soring_stress_autotest

These tests are quite time consuming (~15 mins each),
that's another reason to put them into a separate test-suite.

Signed-off-by: Konstantin Ananyev <konstantin.ananyev@huawei.com>
Acked-by: Stephen Hemminger <stephen@networkplumber.org>
---
 app/test/suites/meson.build   | 10 ++++++++++
 app/test/test.h               |  1 +
 app/test/test_ring_stress.c   |  2 +-
 app/test/test_soring_stress.c |  2 +-
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/app/test/suites/meson.build b/app/test/suites/meson.build
index 191702cf76..e482373330 100644
--- a/app/test/suites/meson.build
+++ b/app/test/suites/meson.build
@@ -5,6 +5,7 @@
 # to complete, so timeout to 10 minutes
 timeout_seconds = 600
 timeout_seconds_fast = 10
+timeout_seconds_stress = 900
 
 test_no_huge_args = ['--no-huge', '-m', '2048']
 has_hugepage = run_command(has_hugepages_cmd, check: true).stdout().strip() != '0'
@@ -21,6 +22,7 @@ endif
 # - fast_tests
 # - perf_tests
 # - driver_tests
+# - stress_tests
 test_suites = run_command(get_test_suites_cmd, autotest_sources,
          check: true).stdout().strip().split()
 foreach suite:test_suites
@@ -39,6 +41,14 @@ foreach suite:test_suites
                     timeout: timeout_seconds,
                     is_parallel: false)
         endforeach
+    elif suite_name == 'stress-tests'
+        foreach t: suite_tests
+            test(t, dpdk_test,
+                    env: ['DPDK_TEST=' + t],
+                    timeout: timeout_seconds_stress,
+                    is_parallel: false,
+                    suite: suite_name)
+        endforeach
     elif suite_name != 'fast-tests'
         # simple cases - tests without parameters or special handling
         foreach t: suite_tests
diff --git a/app/test/test.h b/app/test/test.h
index 15e23d297f..ebc4864bf8 100644
--- a/app/test/test.h
+++ b/app/test/test.h
@@ -208,5 +208,6 @@ void add_test_command(struct test_command *t);
 #define REGISTER_FAST_TEST(cmd, no_huge, ASan, func)  REGISTER_TEST_COMMAND(cmd, func)
 #define REGISTER_PERF_TEST REGISTER_TEST_COMMAND
 #define REGISTER_DRIVER_TEST REGISTER_TEST_COMMAND
+#define REGISTER_STRESS_TEST REGISTER_TEST_COMMAND
 
 #endif
diff --git a/app/test/test_ring_stress.c b/app/test/test_ring_stress.c
index 1af45e0fc8..82e19b02c3 100644
--- a/app/test/test_ring_stress.c
+++ b/app/test/test_ring_stress.c
@@ -63,4 +63,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(ring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(ring_stress_autotest, test_ring_stress);
diff --git a/app/test/test_soring_stress.c b/app/test/test_soring_stress.c
index 334af6a29c..e5655d49cb 100644
--- a/app/test/test_soring_stress.c
+++ b/app/test/test_soring_stress.c
@@ -45,4 +45,4 @@ test_ring_stress(void)
 	return (k != n);
 }
 
-REGISTER_TEST_COMMAND(soring_stress_autotest, test_ring_stress);
+REGISTER_STRESS_TEST(soring_stress_autotest, test_ring_stress);
-- 
2.35.3


^ permalink raw reply	[flat|nested] 101+ messages in thread

end of thread, other threads:[~2024-11-11 13:29 UTC | newest]

Thread overview: 101+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2024-08-15  8:53 [RFC 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-08-15  8:53 ` [RFC 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
2024-08-15  8:53 ` [RFC 2/6] ring: make copying functions generic Konstantin Ananyev
2024-08-15  8:53 ` [RFC 3/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-08-15 11:11   ` Morten Brørup
2024-08-15 12:41     ` Konstantin Ananyev
2024-08-15 13:22       ` Morten Brørup
2024-08-26 19:04   ` Mattias Rönnblom
2024-09-03 13:55     ` Konstantin Ananyev
2024-08-15  8:53 ` [RFC 4/6] app/test: add unit tests for soring API Konstantin Ananyev
2024-08-15  8:53 ` [RFC 5/6] examples/l3fwd: make ACL work in pipeline and eventdev modes Konstantin Ananyev
2024-08-15  8:53 ` [RFC 6/6] ring: minimize reads of the counterpart cache-line Konstantin Ananyev
2024-09-06 13:13 ` [RFCv2 0/6] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-09-06 13:13   ` [RFCv2 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
2024-09-06 13:13   ` [RFCv2 2/6] ring: make copying functions generic Konstantin Ananyev
2024-09-06 13:13   ` [RFCv2 3/6] ring: make dump function more verbose Konstantin Ananyev
2024-09-06 13:13   ` [RFCv2 4/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-09-06 13:13   ` [RFCv2 5/6] app/test: add unit tests for soring API Konstantin Ananyev
2024-09-06 13:13   ` [RFCv2 6/6] examples/l3fwd: make ACL work in pipeline and eventdev modes Konstantin Ananyev
2024-09-16 12:37   ` [PATCH v3 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-09-16 12:37     ` [PATCH v3 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
2024-09-16 12:37     ` [PATCH v3 2/5] ring: make copying functions generic Konstantin Ananyev
2024-09-16 12:37     ` [PATCH v3 3/5] ring: make dump function more verbose Konstantin Ananyev
2024-09-16 12:37     ` [PATCH v3 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-09-16 12:37     ` [PATCH v3 5/5] app/test: add unit tests for soring API Konstantin Ananyev
2024-09-17 12:09     ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-09-17 12:09       ` [PATCH v4 1/5] ring: common functions for 'move head' ops Konstantin Ananyev
2024-09-17 12:09       ` [PATCH v4 2/5] ring: make copying functions generic Konstantin Ananyev
2024-09-17 12:09       ` [PATCH v4 3/5] ring: make dump function more verbose Konstantin Ananyev
2024-09-17 12:09       ` [PATCH v4 4/5] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-09-19 17:03         ` Jerin Jacob
2024-09-17 12:09       ` [PATCH v4 5/5] app/test: add unit tests for soring API Konstantin Ananyev
2024-10-12 18:09       ` [PATCH v4 0/5] Stage-Ordered API and other extensions for ring library Stephen Hemminger
2024-10-15 13:01       ` [PATCH v5 0/6] " Konstantin Ananyev
2024-10-15 13:01         ` [PATCH v5 1/6] ring: common functions for 'move head' ops Konstantin Ananyev
2024-10-15 15:04           ` Morten Brørup
2024-10-15 13:01         ` [PATCH v5 2/6] ring: make copying functions generic Konstantin Ananyev
2024-10-15 13:01         ` [PATCH v5 3/6] ring: make dump function more verbose Konstantin Ananyev
2024-10-15 13:01         ` [PATCH v5 4/6] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-10-15 13:01         ` [PATCH v5 5/6] app/test: add unit tests for soring API Konstantin Ananyev
2024-10-15 13:01         ` [PATCH v5 6/6] test: add stress test suite Konstantin Ananyev
2024-10-15 15:59         ` [PATCH v5 0/6] Stage-Ordered API and other extensions for ring library Stephen Hemminger
2024-10-15 16:02         ` Stephen Hemminger
2024-10-21 16:08         ` [PATCH v6 0/7] " Konstantin Ananyev
2024-10-21 16:08           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
2024-10-21 16:08           ` [PATCH v6 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
2024-10-21 16:08           ` [PATCH v6 3/7] ring: make copying functions generic Konstantin Ananyev
2024-10-21 16:08           ` [PATCH v6 4/7] ring: make dump function more verbose Konstantin Ananyev
2024-10-21 16:08           ` [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-10-21 16:08           ` [PATCH v6 6/7] app/test: add unit tests for soring API Konstantin Ananyev
2024-10-21 17:47         ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 3/7] ring: make copying functions generic Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 4/7] ring: make dump function more verbose Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 6/7] app/test: add unit tests for soring API Konstantin Ananyev
2024-10-21 17:47           ` [PATCH v6 7/7] test: add stress test suite Konstantin Ananyev
2024-10-28 17:18           ` [PATCH v6 0/7] Stage-Ordered API and other extensions for ring library David Christensen
2024-10-29 14:32             ` Konstantin Ananyev
2024-10-30 21:22           ` [PATCH v7 " Konstantin Ananyev
2024-10-30 21:22             ` [PATCH v7 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
2024-11-07 11:50               ` Morten Brørup
2024-10-30 21:22             ` [PATCH v7 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
2024-11-07 11:31               ` Morten Brørup
2024-10-30 21:23             ` [PATCH v7 3/7] ring: make copying functions generic Konstantin Ananyev
2024-11-07 11:46               ` Morten Brørup
2024-10-30 21:23             ` [PATCH v7 4/7] ring: make dump function more verbose Konstantin Ananyev
2024-11-07 11:49               ` Morten Brørup
2024-10-30 21:23             ` [PATCH v7 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-11-07 12:07               ` Morten Brørup
2024-10-30 21:23             ` [PATCH v7 6/7] app/test: add unit tests for soring API Konstantin Ananyev
2024-10-30 21:23             ` [PATCH v7 7/7] test: add stress test suite Konstantin Ananyev
2024-11-07 10:41             ` [PATCH v7 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-11-07 16:16             ` Stephen Hemminger
2024-11-07 18:11               ` Konstantin Ananyev
2024-11-07 18:24             ` [PATCH v8 " Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 3/7] ring: make copying functions generic Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 4/7] ring: make dump function more verbose Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 6/7] app/test: add unit tests for soring API Konstantin Ananyev
2024-11-07 18:24               ` [PATCH v8 7/7] test: add stress test suite Konstantin Ananyev
2024-11-08 21:56               ` [PATCH v8 0/7] Stage-Ordered API and other extensions for ring library Thomas Monjalon
2024-11-11 12:25               ` [PATCH v9 " Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 3/7] ring: make copying functions generic Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 4/7] ring: make dump function more verbose Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 6/7] app/test: add unit tests for soring API Konstantin Ananyev
2024-11-11 12:25                 ` [PATCH v9 7/7] test: add stress test suite Konstantin Ananyev
2024-11-11 14:19                 ` [PATCH v10 0/7] Stage-Ordered API and other extensions for ring library Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 1/7] test/ring: fix failure with custom number of lcores Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 2/7] ring: common functions for 'move head' ops Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 3/7] ring: make copying functions generic Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 4/7] ring: make dump function more verbose Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 5/7] ring/soring: introduce Staged Ordered Ring Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 6/7] app/test: add unit tests for soring API Konstantin Ananyev
2024-11-11 14:19                   ` [PATCH v10 7/7] test: add stress test suite Konstantin Ananyev

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).