From: Timothy McDaniel <timothy.mcdaniel@intel.com>
Cc: dev@dpdk.org, erik.g.carrillo@intel.com, gage.eads@intel.com,
harry.van.haaren@intel.com, jerinj@marvell.com
Subject: [dpdk-dev] [PATCH 16/22] event/dlb2: add dequeue and its burst variants
Date: Fri, 11 Sep 2020 15:26:21 -0500 [thread overview]
Message-ID: <1599855987-25976-17-git-send-email-timothy.mcdaniel@intel.com> (raw)
In-Reply-To: <1599855987-25976-1-git-send-email-timothy.mcdaniel@intel.com>
Add support for dequeue, dequeue_burst, ...
Signed-off-by: Timothy McDaniel <timothy.mcdaniel@intel.com>
---
drivers/event/dlb2/dlb2.c | 786 ++++++++++++++++++++++++++++++++++++++++++++++
1 file changed, 786 insertions(+)
diff --git a/drivers/event/dlb2/dlb2.c b/drivers/event/dlb2/dlb2.c
index ac4cf19..8911c55 100644
--- a/drivers/event/dlb2/dlb2.c
+++ b/drivers/event/dlb2/dlb2.c
@@ -2751,9 +2751,786 @@ dlb2_event_enqueue_forward_burst(void *event_port,
return dlb2_event_enqueue_burst(event_port, events, num);
}
+static inline void
+dlb2_port_credits_inc(struct dlb2_port *qm_port, int num)
+{
+ uint32_t batch_size = DLB2_SW_CREDIT_BATCH_SZ;
+
+ /* increment port credits, and return to pool if exceeds threshold */
+ if (!qm_port->is_directed) {
+ qm_port->cached_ldb_credits += num;
+ if (qm_port->cached_ldb_credits >= 2 * batch_size) {
+ __atomic_fetch_add(
+ qm_port->credit_pool[DLB2_LDB_QUEUE],
+ batch_size, __ATOMIC_SEQ_CST);
+ qm_port->cached_ldb_credits -= batch_size;
+ }
+ } else {
+ qm_port->cached_dir_credits += num;
+ if (qm_port->cached_dir_credits >= 2 * batch_size) {
+ __atomic_fetch_add(
+ qm_port->credit_pool[DLB2_DIR_QUEUE],
+ batch_size, __ATOMIC_SEQ_CST);
+ qm_port->cached_dir_credits -= batch_size;
+ }
+ }
+}
+
+static inline bool
+dlb2_cq_is_empty(struct dlb2_port *qm_port)
+{
+ volatile struct dlb2_dequeue_qe *qe_ptr;
+ struct dlb2_dequeue_qe qe;
+
+ qe_ptr = dlb2_port[qm_port->id][PORT_TYPE(qm_port)].cq_base;
+ qe = qe_ptr[qm_port->cq_idx];
+
+ return (qe.cq_gen != qm_port->gen_bit);
+}
+
+static inline int
+dlb2_dequeue_wait(struct dlb2_eventdev *dlb2,
+ struct dlb2_eventdev_port *ev_port,
+ struct dlb2_port *qm_port,
+ uint64_t timeout,
+ uint64_t start_ticks)
+{
+ struct process_local_port_data *port_data;
+ uint64_t elapsed_ticks;
+
+ port_data = &dlb2_port[qm_port->id][PORT_TYPE(qm_port)];
+
+ elapsed_ticks = rte_get_timer_cycles() - start_ticks;
+
+ /* Wait/poll time expired */
+ if (elapsed_ticks >= timeout) {
+ return 1;
+ } else if (dlb2->umwait_allowed) {
+ volatile struct dlb2_dequeue_qe *cq_base;
+
+ cq_base = port_data->cq_base;
+
+ /* Block on cache line write to CQ. Note: it's
+ * safe to access the per-process cq_base
+ * address here, since the PMD has already
+ * attempted at least one CQ dequeue.
+ */
+ dlb2_umonitor(&cq_base[qm_port->cq_idx]);
+
+ /* Avoid race condition. Check if still empty */
+ if (dlb2_cq_is_empty(qm_port)) {
+ dlb2_umwait(RTE_LIBRTE_PMD_DLB2_UMWAIT_CTL_STATE,
+ timeout + start_ticks);
+ DLB2_INC_STAT(
+ ev_port->stats.traffic.rx_umonitor_umwait, 1);
+ }
+ } else {
+ uint64_t poll_interval = RTE_LIBRTE_PMD_DLB2_POLL_INTERVAL;
+ uint64_t curr_ticks = rte_get_timer_cycles();
+ uint64_t init_ticks = curr_ticks;
+
+ while ((curr_ticks - start_ticks < timeout) &&
+ (curr_ticks - init_ticks < poll_interval))
+ curr_ticks = rte_get_timer_cycles();
+ }
+
+ return 0;
+}
+
+static inline int
+dlb2_process_dequeue_qes(struct dlb2_eventdev_port *ev_port,
+ struct dlb2_port *qm_port,
+ struct rte_event *events,
+ struct dlb2_dequeue_qe *qes,
+ int cnt)
+{
+ uint8_t *qid_mappings = qm_port->qid_mappings;
+ int i, num, evq_id;
+
+ RTE_SET_USED(ev_port); /* avoids unused variable error if stats off */
+
+ for (i = 0, num = 0; i < cnt; i++) {
+ struct dlb2_dequeue_qe *qe = &qes[i];
+ int sched_type_map[DLB2_NUM_HW_SCHED_TYPES] = {
+ [DLB2_SCHED_ATOMIC] = RTE_SCHED_TYPE_ATOMIC,
+ [DLB2_SCHED_UNORDERED] = RTE_SCHED_TYPE_PARALLEL,
+ [DLB2_SCHED_ORDERED] = RTE_SCHED_TYPE_ORDERED,
+ [DLB2_SCHED_DIRECTED] = RTE_SCHED_TYPE_ATOMIC,
+ };
+
+ /* Fill in event information.
+ * Note that flow_id must be embedded in the data by
+ * the app, such as the mbuf RSS hash field if the data
+ * buffer is a mbuf.
+ */
+ if (unlikely(qe->error)) {
+ DLB2_LOG_ERR("QE error bit ON\n");
+ DLB2_INC_STAT(ev_port->stats.traffic.rx_drop, 1);
+ dlb2_consume_qe_immediate(qm_port, 1);
+ continue; /* Ignore */
+ }
+
+ events[num].u64 = qe->data;
+ events[num].flow_id = qe->flow_id;
+ events[num].priority = DLB2_TO_EV_PRIO((uint8_t)qe->priority);
+ events[num].event_type = qe->u.event_type.major;
+ events[num].sub_event_type = qe->u.event_type.sub;
+ events[num].sched_type = sched_type_map[qe->sched_type];
+ events[num].impl_opaque = qe->qid_depth;
+
+ /* qid not preserved for directed queues */
+ if (qm_port->is_directed)
+ evq_id = ev_port->link[0].queue_id;
+ else
+ evq_id = qid_mappings[qe->qid];
+
+ events[num].queue_id = evq_id;
+ DLB2_INC_STAT(
+ ev_port->stats.queue[evq_id].qid_depth[qe->qid_depth],
+ 1);
+ DLB2_INC_STAT(ev_port->stats.rx_sched_cnt[qe->sched_type], 1);
+ DLB2_INC_STAT(ev_port->stats.traffic.rx_ok, 1);
+ num++;
+ }
+
+ return num;
+}
+
+static inline int
+dlb2_process_dequeue_four_qes(struct dlb2_eventdev_port *ev_port,
+ struct dlb2_port *qm_port,
+ struct rte_event *events,
+ struct dlb2_dequeue_qe *qes)
+{
+ int sched_type_map[] = {
+ [DLB2_SCHED_ATOMIC] = RTE_SCHED_TYPE_ATOMIC,
+ [DLB2_SCHED_UNORDERED] = RTE_SCHED_TYPE_PARALLEL,
+ [DLB2_SCHED_ORDERED] = RTE_SCHED_TYPE_ORDERED,
+ [DLB2_SCHED_DIRECTED] = RTE_SCHED_TYPE_ATOMIC,
+ };
+ const int num_events = DLB2_NUM_QES_PER_CACHE_LINE;
+ uint8_t *qid_mappings = qm_port->qid_mappings;
+ __m128i sse_evt[2];
+
+ RTE_SET_USED(ev_port); /* avoids unused variable error, if stats off */
+
+ /* In the unlikely case that any of the QE error bits are set, process
+ * them one at a time.
+ */
+ if (unlikely(qes[0].error || qes[1].error ||
+ qes[2].error || qes[3].error))
+ return dlb2_process_dequeue_qes(ev_port, qm_port, events,
+ qes, num_events);
+
+ events[0].u64 = qes[0].data;
+ events[1].u64 = qes[1].data;
+ events[2].u64 = qes[2].data;
+ events[3].u64 = qes[3].data;
+
+ /* Construct the metadata portion of two struct rte_events
+ * in one 128b SSE register. Event metadata is constructed in the SSE
+ * registers like so:
+ * sse_evt[0][63:0]: event[0]'s metadata
+ * sse_evt[0][127:64]: event[1]'s metadata
+ * sse_evt[1][63:0]: event[2]'s metadata
+ * sse_evt[1][127:64]: event[3]'s metadata
+ */
+ sse_evt[0] = _mm_setzero_si128();
+ sse_evt[1] = _mm_setzero_si128();
+
+ /* Convert the hardware queue ID to an event queue ID and store it in
+ * the metadata:
+ * sse_evt[0][47:40] = qid_mappings[qes[0].qid]
+ * sse_evt[0][111:104] = qid_mappings[qes[1].qid]
+ * sse_evt[1][47:40] = qid_mappings[qes[2].qid]
+ * sse_evt[1][111:104] = qid_mappings[qes[3].qid]
+ */
+#define RTE_EVENT_QUEUE_ID_BYTE 5
+ sse_evt[0] = _mm_insert_epi8(sse_evt[0],
+ qid_mappings[qes[0].qid],
+ RTE_EVENT_QUEUE_ID_BYTE);
+ sse_evt[0] = _mm_insert_epi8(sse_evt[0],
+ qid_mappings[qes[1].qid],
+ RTE_EVENT_QUEUE_ID_BYTE + 8);
+ sse_evt[1] = _mm_insert_epi8(sse_evt[1],
+ qid_mappings[qes[2].qid],
+ RTE_EVENT_QUEUE_ID_BYTE);
+ sse_evt[1] = _mm_insert_epi8(sse_evt[1],
+ qid_mappings[qes[3].qid],
+ RTE_EVENT_QUEUE_ID_BYTE + 8);
+
+ /* Convert the hardware priority to an event priority and store it in
+ * the metadata, while also returning the queue depth status
+ * value captured by the hardware, storing it in impl_opaque, which can
+ * be read by the application but not modified
+ * sse_evt[0][55:48] = DLB2_TO_EV_PRIO(qes[0].priority)
+ * sse_evt[0][63:56] = qes[0].qid_depth
+ * sse_evt[0][119:112] = DLB2_TO_EV_PRIO(qes[1].priority)
+ * sse_evt[0][127:120] = qes[1].qid_depth
+ * sse_evt[1][55:48] = DLB2_TO_EV_PRIO(qes[2].priority)
+ * sse_evt[1][63:56] = qes[2].qid_depth
+ * sse_evt[1][119:112] = DLB2_TO_EV_PRIO(qes[3].priority)
+ * sse_evt[1][127:120] = qes[3].qid_depth
+ */
+#define RTE_EVENT_PRIO_IMPL_OPAQUE_WORD 3
+#define RTE_BYTE_SHIFT 8
+ sse_evt[0] =
+ _mm_insert_epi16(sse_evt[0],
+ DLB2_TO_EV_PRIO((uint8_t)qes[0].priority) |
+ (qes[0].qid_depth << RTE_BYTE_SHIFT),
+ RTE_EVENT_PRIO_IMPL_OPAQUE_WORD);
+ sse_evt[0] =
+ _mm_insert_epi16(sse_evt[0],
+ DLB2_TO_EV_PRIO((uint8_t)qes[1].priority) |
+ (qes[1].qid_depth << RTE_BYTE_SHIFT),
+ RTE_EVENT_PRIO_IMPL_OPAQUE_WORD + 4);
+ sse_evt[1] =
+ _mm_insert_epi16(sse_evt[1],
+ DLB2_TO_EV_PRIO((uint8_t)qes[2].priority) |
+ (qes[2].qid_depth << RTE_BYTE_SHIFT),
+ RTE_EVENT_PRIO_IMPL_OPAQUE_WORD);
+ sse_evt[1] =
+ _mm_insert_epi16(sse_evt[1],
+ DLB2_TO_EV_PRIO((uint8_t)qes[3].priority) |
+ (qes[3].qid_depth << RTE_BYTE_SHIFT),
+ RTE_EVENT_PRIO_IMPL_OPAQUE_WORD + 4);
+
+ /* Write the event type, sub event type, and flow_id to the event
+ * metadata.
+ * sse_evt[0][31:0] = qes[0].flow_id |
+ * qes[0].u.event_type.major << 28 |
+ * qes[0].u.event_type.sub << 20;
+ * sse_evt[0][95:64] = qes[1].flow_id |
+ * qes[1].u.event_type.major << 28 |
+ * qes[1].u.event_type.sub << 20;
+ * sse_evt[1][31:0] = qes[2].flow_id |
+ * qes[2].u.event_type.major << 28 |
+ * qes[2].u.event_type.sub << 20;
+ * sse_evt[1][95:64] = qes[3].flow_id |
+ * qes[3].u.event_type.major << 28 |
+ * qes[3].u.event_type.sub << 20;
+ */
+#define RTE_EVENT_EV_TYPE_DW 0
+#define RTE_EVENT_EV_TYPE_SHIFT 28
+#define RTE_EVENT_SUB_EV_TYPE_SHIFT 20
+ sse_evt[0] = _mm_insert_epi32(sse_evt[0],
+ qes[0].flow_id |
+ qes[0].u.event_type.major << RTE_EVENT_EV_TYPE_SHIFT |
+ qes[0].u.event_type.sub << RTE_EVENT_SUB_EV_TYPE_SHIFT,
+ RTE_EVENT_EV_TYPE_DW);
+ sse_evt[0] = _mm_insert_epi32(sse_evt[0],
+ qes[1].flow_id |
+ qes[1].u.event_type.major << RTE_EVENT_EV_TYPE_SHIFT |
+ qes[1].u.event_type.sub << RTE_EVENT_SUB_EV_TYPE_SHIFT,
+ RTE_EVENT_EV_TYPE_DW + 2);
+ sse_evt[1] = _mm_insert_epi32(sse_evt[1],
+ qes[2].flow_id |
+ qes[2].u.event_type.major << RTE_EVENT_EV_TYPE_SHIFT |
+ qes[2].u.event_type.sub << RTE_EVENT_SUB_EV_TYPE_SHIFT,
+ RTE_EVENT_EV_TYPE_DW);
+ sse_evt[1] = _mm_insert_epi32(sse_evt[1],
+ qes[3].flow_id |
+ qes[3].u.event_type.major << RTE_EVENT_EV_TYPE_SHIFT |
+ qes[3].u.event_type.sub << RTE_EVENT_SUB_EV_TYPE_SHIFT,
+ RTE_EVENT_EV_TYPE_DW + 2);
+
+ /* Write the sched type to the event metadata. 'op' and 'rsvd' are not
+ * set:
+ * sse_evt[0][39:32] = sched_type_map[qes[0].sched_type] << 6
+ * sse_evt[0][103:96] = sched_type_map[qes[1].sched_type] << 6
+ * sse_evt[1][39:32] = sched_type_map[qes[2].sched_type] << 6
+ * sse_evt[1][103:96] = sched_type_map[qes[3].sched_type] << 6
+ */
+#define RTE_EVENT_SCHED_TYPE_BYTE 4
+#define RTE_EVENT_SCHED_TYPE_SHIFT 6
+ sse_evt[0] = _mm_insert_epi8(sse_evt[0],
+ sched_type_map[qes[0].sched_type] << RTE_EVENT_SCHED_TYPE_SHIFT,
+ RTE_EVENT_SCHED_TYPE_BYTE);
+ sse_evt[0] = _mm_insert_epi8(sse_evt[0],
+ sched_type_map[qes[1].sched_type] << RTE_EVENT_SCHED_TYPE_SHIFT,
+ RTE_EVENT_SCHED_TYPE_BYTE + 8);
+ sse_evt[1] = _mm_insert_epi8(sse_evt[1],
+ sched_type_map[qes[2].sched_type] << RTE_EVENT_SCHED_TYPE_SHIFT,
+ RTE_EVENT_SCHED_TYPE_BYTE);
+ sse_evt[1] = _mm_insert_epi8(sse_evt[1],
+ sched_type_map[qes[3].sched_type] << RTE_EVENT_SCHED_TYPE_SHIFT,
+ RTE_EVENT_SCHED_TYPE_BYTE + 8);
+
+ /* Store the metadata to the event (use the double-precision
+ * _mm_storeh_pd because there is no integer function for storing the
+ * upper 64b):
+ * events[0].event = sse_evt[0][63:0]
+ * events[1].event = sse_evt[0][127:64]
+ * events[2].event = sse_evt[1][63:0]
+ * events[3].event = sse_evt[1][127:64]
+ */
+ _mm_storel_epi64((__m128i *)&events[0].event, sse_evt[0]);
+ _mm_storeh_pd((double *)&events[1].event, (__m128d) sse_evt[0]);
+ _mm_storel_epi64((__m128i *)&events[2].event, sse_evt[1]);
+ _mm_storeh_pd((double *)&events[3].event, (__m128d) sse_evt[1]);
+
+ DLB2_INC_STAT(ev_port->stats.rx_sched_cnt[qes[0].sched_type], 1);
+ DLB2_INC_STAT(ev_port->stats.rx_sched_cnt[qes[1].sched_type], 1);
+ DLB2_INC_STAT(ev_port->stats.rx_sched_cnt[qes[2].sched_type], 1);
+ DLB2_INC_STAT(ev_port->stats.rx_sched_cnt[qes[3].sched_type], 1);
+
+ DLB2_INC_STAT(
+ ev_port->stats.queue[events[0].queue_id].
+ qid_depth[qes[0].qid_depth],
+ 1);
+ DLB2_INC_STAT(
+ ev_port->stats.queue[events[1].queue_id].
+ qid_depth[qes[1].qid_depth],
+ 1);
+ DLB2_INC_STAT(
+ ev_port->stats.queue[events[2].queue_id].
+ qid_depth[qes[2].qid_depth],
+ 1);
+ DLB2_INC_STAT(
+ ev_port->stats.queue[events[3].queue_id].
+ qid_depth[qes[3].qid_depth],
+ 1);
+
+ DLB2_INC_STAT(ev_port->stats.traffic.rx_ok, num_events);
+
+ return num_events;
+}
+
+static __rte_always_inline int
+dlb2_recv_qe_sparse(struct dlb2_port *qm_port, struct dlb2_dequeue_qe *qe)
+{
+ volatile struct dlb2_dequeue_qe *cq_addr;
+ uint8_t xor_mask[2] = {0x0F, 0x00};
+ const uint8_t and_mask = 0x0F;
+ __m128i *qes = (__m128i *)qe;
+ uint8_t gen_bits, gen_bit;
+ uintptr_t addr[4];
+ uint16_t idx;
+
+ cq_addr = dlb2_port[qm_port->id][PORT_TYPE(qm_port)].cq_base;
+
+ idx = qm_port->cq_idx;
+
+ /* Load the next 4 QEs */
+ addr[0] = (uintptr_t)&cq_addr[idx];
+ addr[1] = (uintptr_t)&cq_addr[(idx + 4) & qm_port->cq_depth_mask];
+ addr[2] = (uintptr_t)&cq_addr[(idx + 8) & qm_port->cq_depth_mask];
+ addr[3] = (uintptr_t)&cq_addr[(idx + 12) & qm_port->cq_depth_mask];
+
+ /* Prefetch next batch of QEs (all CQs occupy minimum 8 cache lines) */
+ rte_prefetch0(&cq_addr[(idx + 16) & qm_port->cq_depth_mask]);
+ rte_prefetch0(&cq_addr[(idx + 20) & qm_port->cq_depth_mask]);
+ rte_prefetch0(&cq_addr[(idx + 24) & qm_port->cq_depth_mask]);
+ rte_prefetch0(&cq_addr[(idx + 28) & qm_port->cq_depth_mask]);
+
+ /* Correct the xor_mask for wrap-around QEs */
+ gen_bit = qm_port->gen_bit;
+ xor_mask[gen_bit] ^= !!((idx + 4) > qm_port->cq_depth_mask) << 1;
+ xor_mask[gen_bit] ^= !!((idx + 8) > qm_port->cq_depth_mask) << 2;
+ xor_mask[gen_bit] ^= !!((idx + 12) > qm_port->cq_depth_mask) << 3;
+
+ /* Read the cache lines backwards to ensure that if QE[N] (N > 0) is
+ * valid, then QEs[0:N-1] are too.
+ */
+ qes[3] = _mm_load_si128((__m128i *)(void *)addr[3]);
+ rte_compiler_barrier();
+ qes[2] = _mm_load_si128((__m128i *)(void *)addr[2]);
+ rte_compiler_barrier();
+ qes[1] = _mm_load_si128((__m128i *)(void *)addr[1]);
+ rte_compiler_barrier();
+ qes[0] = _mm_load_si128((__m128i *)(void *)addr[0]);
+
+ /* Extract and combine the gen bits */
+ gen_bits = ((_mm_extract_epi8(qes[0], 15) & 0x1) << 0) |
+ ((_mm_extract_epi8(qes[1], 15) & 0x1) << 1) |
+ ((_mm_extract_epi8(qes[2], 15) & 0x1) << 2) |
+ ((_mm_extract_epi8(qes[3], 15) & 0x1) << 3);
+
+ /* XOR the combined bits such that a 1 represents a valid QE */
+ gen_bits ^= xor_mask[gen_bit];
+
+ /* Mask off gen bits we don't care about */
+ gen_bits &= and_mask;
+
+ return __builtin_popcount(gen_bits);
+}
+
+static inline void
+dlb2_inc_cq_idx(struct dlb2_port *qm_port, int cnt)
+{
+ uint16_t idx = qm_port->cq_idx_unmasked + cnt;
+
+ qm_port->cq_idx_unmasked = idx;
+ qm_port->cq_idx = idx & qm_port->cq_depth_mask;
+ qm_port->gen_bit = (~(idx >> qm_port->gen_bit_shift)) & 0x1;
+}
+
+static int
+dlb2_event_release(struct dlb2_eventdev *dlb2,
+ uint8_t port_id,
+ int n)
+{
+ struct process_local_port_data *port_data;
+ struct dlb2_eventdev_port *ev_port;
+ struct dlb2_port *qm_port;
+ int i, cnt;
+
+ if (port_id > dlb2->num_ports) {
+ DLB2_LOG_ERR("Invalid port id %d in dlb2-event_release\n",
+ port_id);
+ rte_errno = -EINVAL;
+ return rte_errno;
+ }
+
+ ev_port = &dlb2->ev_ports[port_id];
+ qm_port = &ev_port->qm_port;
+ port_data = &dlb2_port[qm_port->id][PORT_TYPE(qm_port)];
+
+ cnt = 0;
+
+ if (qm_port->is_directed) {
+ cnt = n;
+ goto sw_credit_update;
+ }
+
+ for (i = 0; i < n; i += DLB2_NUM_QES_PER_CACHE_LINE) {
+ int j;
+
+ /* Zero-out QEs */
+ qm_port->qe4[0].cmd_byte = 0;
+ qm_port->qe4[1].cmd_byte = 0;
+ qm_port->qe4[2].cmd_byte = 0;
+ qm_port->qe4[3].cmd_byte = 0;
+
+ for (j = 0; j < DLB2_NUM_QES_PER_CACHE_LINE && (i + j) < n; j++)
+ qm_port->qe4[j].cmd_byte = DLB2_COMP_CMD_BYTE;
+
+ qm_port->issued_releases += j;
+
+ if (j == 0)
+ break;
+
+ if (qm_port->token_pop_mode == DELAYED_POP && j < 4 &&
+ qm_port->issued_releases >= qm_port->token_pop_thresh - 1) {
+ dlb2_construct_token_pop_qe(qm_port, j);
+
+ /* Reset the releases counter for the next QE batch */
+ qm_port->issued_releases -= qm_port->token_pop_thresh;
+ }
+
+ dlb2_hw_do_enqueue(qm_port, i == 0, port_data);
+
+ cnt += j;
+ }
+
+ if (qm_port->token_pop_mode == DELAYED_POP &&
+ qm_port->issued_releases >= qm_port->token_pop_thresh - 1) {
+ dlb2_consume_qe_immediate(qm_port, qm_port->owed_tokens);
+ qm_port->issued_releases -= qm_port->token_pop_thresh;
+ }
+
+sw_credit_update:
+ /* each release returns one credit */
+ if (!ev_port->outstanding_releases) {
+ DLB2_LOG_ERR("Unrecoverable application error. Outstanding releases underflowed.\n");
+ rte_errno = -ENOTRECOVERABLE;
+ return rte_errno;
+ }
+
+ ev_port->outstanding_releases -= cnt;
+ ev_port->inflight_credits += cnt;
+
+ /* Replenish s/w credits if enough releases are performed */
+ dlb2_replenish_sw_credits(dlb2, ev_port);
+ return 0;
+}
+
+static inline int16_t
+dlb2_hw_dequeue_sparse(struct dlb2_eventdev *dlb2,
+ struct dlb2_eventdev_port *ev_port,
+ struct rte_event *events,
+ uint16_t max_num,
+ uint64_t dequeue_timeout_ticks)
+{
+ uint64_t timeout;
+ uint64_t start_ticks = 0ULL;
+ struct dlb2_port *qm_port;
+ int num = 0;
+
+ qm_port = &ev_port->qm_port;
+
+ /* We have a special implementation for waiting. Wait can be:
+ * 1) no waiting at all
+ * 2) busy poll only
+ * 3) wait for interrupt. If wakeup and poll time
+ * has expired, then return to caller
+ * 4) umonitor/umwait repeatedly up to poll time
+ */
+
+ /* If configured for per dequeue wait, then use wait value provided
+ * to this API. Otherwise we must use the global
+ * value from eventdev config time.
+ */
+ if (!dlb2->global_dequeue_wait)
+ timeout = dequeue_timeout_ticks;
+ else
+ timeout = dlb2->global_dequeue_wait_ticks;
+
+ start_ticks = rte_get_timer_cycles();
+
+ while (num < max_num) {
+ struct dlb2_dequeue_qe qes[DLB2_NUM_QES_PER_CACHE_LINE];
+ int num_avail;
+
+ /* Copy up to 4 QEs from the current cache line into qes */
+ num_avail = dlb2_recv_qe_sparse(qm_port, qes);
+
+ /* But don't process more than the user requested */
+ num_avail = RTE_MIN(num_avail, max_num - num);
+
+ dlb2_inc_cq_idx(qm_port, num_avail << 2);
+
+ if (num_avail == DLB2_NUM_QES_PER_CACHE_LINE)
+ num += dlb2_process_dequeue_four_qes(ev_port,
+ qm_port,
+ &events[num],
+ &qes[0]);
+ else if (num_avail)
+ num += dlb2_process_dequeue_qes(ev_port,
+ qm_port,
+ &events[num],
+ &qes[0],
+ num_avail);
+ else if ((timeout == 0) || (num > 0))
+ /* Not waiting in any form, or 1+ events received? */
+ break;
+ else if (dlb2_dequeue_wait(dlb2, ev_port, qm_port,
+ timeout, start_ticks))
+ break;
+ }
+
+ qm_port->owed_tokens += num;
+
+ if (num) {
+ if (qm_port->token_pop_mode == AUTO_POP)
+ dlb2_consume_qe_immediate(qm_port, num);
+
+ ev_port->outstanding_releases += num;
+
+ dlb2_port_credits_inc(qm_port, num);
+ }
+
+ return num;
+}
+
+static __rte_always_inline int
+dlb2_recv_qe(struct dlb2_port *qm_port, struct dlb2_dequeue_qe *qe,
+ uint8_t *offset)
+{
+ uint8_t xor_mask[2][4] = { {0x0F, 0x0E, 0x0C, 0x08},
+ {0x00, 0x01, 0x03, 0x07} };
+ uint8_t and_mask[4] = {0x0F, 0x0E, 0x0C, 0x08};
+ volatile struct dlb2_dequeue_qe *cq_addr;
+ __m128i *qes = (__m128i *)qe;
+ uint64_t *cache_line_base;
+ uint8_t gen_bits;
+
+ cq_addr = dlb2_port[qm_port->id][PORT_TYPE(qm_port)].cq_base;
+ cq_addr = &cq_addr[qm_port->cq_idx];
+
+ cache_line_base = (void *)(((uintptr_t)cq_addr) & ~0x3F);
+ *offset = ((uintptr_t)cq_addr & 0x30) >> 4;
+
+ /* Load the next CQ cache line from memory. Pack these reads as tight
+ * as possible to reduce the chance that DLB invalidates the line while
+ * the CPU is reading it. Read the cache line backwards to ensure that
+ * if QE[N] (N > 0) is valid, then QEs[0:N-1] are too.
+ *
+ * (Valid QEs start at &qe[offset])
+ */
+ qes[3] = _mm_load_si128((__m128i *)&cache_line_base[6]);
+ qes[2] = _mm_load_si128((__m128i *)&cache_line_base[4]);
+ qes[1] = _mm_load_si128((__m128i *)&cache_line_base[2]);
+ qes[0] = _mm_load_si128((__m128i *)&cache_line_base[0]);
+
+ /* Evict the cache line ASAP */
+ dlb2_cldemote(cache_line_base);
+
+ /* Extract and combine the gen bits */
+ gen_bits = ((_mm_extract_epi8(qes[0], 15) & 0x1) << 0) |
+ ((_mm_extract_epi8(qes[1], 15) & 0x1) << 1) |
+ ((_mm_extract_epi8(qes[2], 15) & 0x1) << 2) |
+ ((_mm_extract_epi8(qes[3], 15) & 0x1) << 3);
+
+ /* XOR the combined bits such that a 1 represents a valid QE */
+ gen_bits ^= xor_mask[qm_port->gen_bit][*offset];
+
+ /* Mask off gen bits we don't care about */
+ gen_bits &= and_mask[*offset];
+
+ return __builtin_popcount(gen_bits);
+}
+
+static inline int16_t
+dlb2_hw_dequeue(struct dlb2_eventdev *dlb2,
+ struct dlb2_eventdev_port *ev_port,
+ struct rte_event *events,
+ uint16_t max_num,
+ uint64_t dequeue_timeout_ticks)
+{
+ uint64_t timeout;
+ uint64_t start_ticks = 0ULL;
+ struct dlb2_port *qm_port;
+ int num = 0;
+
+ qm_port = &ev_port->qm_port;
+
+ /* We have a special implementation for waiting. Wait can be:
+ * 1) no waiting at all
+ * 2) busy poll only
+ * 3) wait for interrupt. If wakeup and poll time
+ * has expired, then return to caller
+ * 4) umonitor/umwait repeatedly up to poll time
+ */
+
+ /* If configured for per dequeue wait, then use wait value provided
+ * to this API. Otherwise we must use the global
+ * value from eventdev config time.
+ */
+ if (!dlb2->global_dequeue_wait)
+ timeout = dequeue_timeout_ticks;
+ else
+ timeout = dlb2->global_dequeue_wait_ticks;
+
+ start_ticks = rte_get_timer_cycles();
+
+ while (num < max_num) {
+ struct dlb2_dequeue_qe qes[DLB2_NUM_QES_PER_CACHE_LINE];
+ uint8_t offset;
+ int num_avail;
+
+ /* Copy up to 4 QEs from the current cache line into qes */
+ num_avail = dlb2_recv_qe(qm_port, qes, &offset);
+
+ /* But don't process more than the user requested */
+ num_avail = RTE_MIN(num_avail, max_num - num);
+
+ dlb2_inc_cq_idx(qm_port, num_avail);
+
+ if (num_avail == DLB2_NUM_QES_PER_CACHE_LINE)
+ num += dlb2_process_dequeue_four_qes(ev_port,
+ qm_port,
+ &events[num],
+ &qes[offset]);
+ else if (num_avail)
+ num += dlb2_process_dequeue_qes(ev_port,
+ qm_port,
+ &events[num],
+ &qes[offset],
+ num_avail);
+ else if ((timeout == 0) || (num > 0))
+ /* Not waiting in any form, or 1+ events received? */
+ break;
+ else if (dlb2_dequeue_wait(dlb2, ev_port, qm_port,
+ timeout, start_ticks))
+ break;
+ }
+
+ qm_port->owed_tokens += num;
+
+ if (num) {
+ if (qm_port->token_pop_mode == AUTO_POP)
+ dlb2_consume_qe_immediate(qm_port, num);
+
+ ev_port->outstanding_releases += num;
+
+ dlb2_port_credits_inc(qm_port, num);
+ }
+
+ return num;
+}
+
+static uint16_t
+dlb2_event_dequeue_burst(void *event_port, struct rte_event *ev, uint16_t num,
+ uint64_t wait)
+{
+ struct dlb2_eventdev_port *ev_port = event_port;
+ struct dlb2_port *qm_port = &ev_port->qm_port;
+ struct dlb2_eventdev *dlb2 = ev_port->dlb2;
+ uint16_t cnt;
+
+ RTE_ASSERT(ev_port->setup_done);
+ RTE_ASSERT(ev != NULL);
+
+ if (ev_port->implicit_release && ev_port->outstanding_releases > 0) {
+ uint16_t out_rels = ev_port->outstanding_releases;
+
+ if (dlb2_event_release(dlb2, ev_port->id, out_rels))
+ return 0; /* rte_errno is set */
+
+ DLB2_INC_STAT(ev_port->stats.tx_implicit_rel, out_rels);
+ }
+
+ if (qm_port->token_pop_mode == DEFERRED_POP && qm_port->owed_tokens)
+ dlb2_consume_qe_immediate(qm_port, qm_port->owed_tokens);
+
+ cnt = dlb2_hw_dequeue(dlb2, ev_port, ev, num, wait);
+
+ DLB2_INC_STAT(ev_port->stats.traffic.total_polls, 1);
+ DLB2_INC_STAT(ev_port->stats.traffic.zero_polls, ((cnt == 0) ? 1 : 0));
+
+ return cnt;
+}
+
+static uint16_t
+dlb2_event_dequeue(void *event_port, struct rte_event *ev, uint64_t wait)
+{
+ return dlb2_event_dequeue_burst(event_port, ev, 1, wait);
+}
+
+static uint16_t
+dlb2_event_dequeue_burst_sparse(void *event_port, struct rte_event *ev,
+ uint16_t num, uint64_t wait)
+{
+ struct dlb2_eventdev_port *ev_port = event_port;
+ struct dlb2_port *qm_port = &ev_port->qm_port;
+ struct dlb2_eventdev *dlb2 = ev_port->dlb2;
+ uint16_t cnt;
+
+ RTE_ASSERT(ev_port->setup_done);
+ RTE_ASSERT(ev != NULL);
+
+ if (ev_port->implicit_release && ev_port->outstanding_releases > 0) {
+ uint16_t out_rels = ev_port->outstanding_releases;
+
+ if (dlb2_event_release(dlb2, ev_port->id, out_rels))
+ return 0; /* rte_errno is set */
+
+ DLB2_INC_STAT(ev_port->stats.tx_implicit_rel, out_rels);
+ }
+
+ if (qm_port->token_pop_mode == DEFERRED_POP && qm_port->owed_tokens)
+ dlb2_consume_qe_immediate(qm_port, qm_port->owed_tokens);
+
+ cnt = dlb2_hw_dequeue_sparse(dlb2, ev_port, ev, num, wait);
+
+ DLB2_INC_STAT(ev_port->stats.traffic.total_polls, 1);
+ DLB2_INC_STAT(ev_port->stats.traffic.zero_polls, ((cnt == 0) ? 1 : 0));
+ return cnt;
+}
+
+static uint16_t
+dlb2_event_dequeue_sparse(void *event_port, struct rte_event *ev,
+ uint64_t wait)
+{
+ return dlb2_event_dequeue_burst_sparse(event_port, ev, 1, wait);
+}
+
static void
dlb2_entry_points_init(struct rte_eventdev *dev)
{
+ struct dlb2_eventdev *dlb2;
+
/* Expose PMD's eventdev interface */
static struct rte_eventdev_ops dlb2_eventdev_entry_ops = {
.dev_infos_get = dlb2_eventdev_info_get,
@@ -2781,6 +3558,15 @@ dlb2_entry_points_init(struct rte_eventdev *dev)
dev->enqueue_burst = dlb2_event_enqueue_burst;
dev->enqueue_new_burst = dlb2_event_enqueue_new_burst;
dev->enqueue_forward_burst = dlb2_event_enqueue_forward_burst;
+
+ dlb2 = dev->data->dev_private;
+ if (dlb2->poll_mode == DLB2_CQ_POLL_MODE_SPARSE) {
+ dev->dequeue = dlb2_event_dequeue_sparse;
+ dev->dequeue_burst = dlb2_event_dequeue_burst_sparse;
+ } else {
+ dev->dequeue = dlb2_event_dequeue;
+ dev->dequeue_burst = dlb2_event_dequeue_burst;
+ }
}
int
--
2.6.4
next prev parent reply other threads:[~2020-09-11 20:32 UTC|newest]
Thread overview: 366+ messages / expand[flat|nested] mbox.gz Atom feed top
2020-09-11 20:26 [dpdk-dev] [PATCH 00/22] Add DLB2 PMD Timothy McDaniel
2020-09-11 20:26 ` [dpdk-dev] [PATCH 01/22] event/dlb2: add meson build infrastructure Timothy McDaniel
2020-10-06 15:58 ` Eads, Gage
2020-10-17 18:20 ` [dpdk-dev] [PATCH v2 00/22] Add DLB2 PMD Timothy McDaniel
2020-10-17 18:20 ` [dpdk-dev] [PATCH v2 01/22] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-18 8:48 ` Jerin Jacob
2020-10-19 8:33 ` Bruce Richardson
2020-10-20 15:17 ` McDaniel, Timothy
2020-10-20 15:20 ` Thomas Monjalon
2020-10-20 15:33 ` McDaniel, Timothy
2020-10-20 15:38 ` Bruce Richardson
2020-10-20 15:34 ` Bruce Richardson
2020-10-20 15:43 ` McDaniel, Timothy
2020-10-21 16:33 ` McDaniel, Timothy
2020-10-20 14:07 ` McDaniel, Timothy
2020-10-19 9:59 ` Kinsella, Ray
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 00/23] Add DLB2 PMD Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-30 9:43 ` [dpdk-dev] [PATCH v5 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-30 10:01 ` [dpdk-dev] [PATCH v5 00/23] Add DLB2 PMD Thomas Monjalon
2020-10-30 10:16 ` McDaniel, Timothy
2020-10-30 10:32 ` Jerin Jacob
2020-10-30 10:43 ` Thomas Monjalon
2020-10-30 11:58 ` McDaniel, Timothy
2020-10-30 13:15 ` Thomas Monjalon
2020-10-30 15:35 ` McDaniel, Timothy
2020-10-30 15:47 ` Thomas Monjalon
2020-10-30 16:02 ` McDaniel, Timothy
2020-10-30 16:42 ` Thomas Monjalon
2020-10-30 14:21 ` Jerin Jacob
2020-10-30 15:25 ` McDaniel, Timothy
2020-10-30 15:31 ` Jerin Jacob
2020-10-30 16:08 ` Van Haaren, Harry
2020-10-30 16:13 ` McDaniel, Timothy
2020-10-30 15:33 ` David Marchand
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 " Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-30 19:51 ` Eads, Gage
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-30 19:51 ` Eads, Gage
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-30 19:50 ` Eads, Gage
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-30 18:50 ` Eads, Gage
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-30 18:28 ` [dpdk-dev] [PATCH v6 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-30 18:29 ` [dpdk-dev] [PATCH v6 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 00/23] Add DLB2 PMD Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-30 23:51 ` [dpdk-dev] [PATCH v7 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 00/23] Add DLB2 PMD Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-31 9:52 ` Jerin Jacob
2020-10-31 17:13 ` McDaniel, Timothy
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-31 10:51 ` David Marchand
2020-10-31 16:37 ` McDaniel, Timothy
2020-10-31 19:19 ` McDaniel, Timothy
2020-10-31 21:38 ` David Marchand
2020-10-31 21:43 ` McDaniel, Timothy
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-31 2:01 ` [dpdk-dev] [PATCH v8 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-31 17:25 ` [dpdk-dev] [PATCH v9 00/23] Add DLB2 PMD Timothy McDaniel
2020-10-31 17:25 ` [dpdk-dev] [PATCH v9 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-31 17:26 ` [dpdk-dev] [PATCH v9 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 00/23] Add DLB2 PMD Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 09/23] event/dlb2: add xstats Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 13/23] event/dlb2: add port setup Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 14/23] event/dlb2: add port link Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-11-01 20:00 ` [dpdk-dev] [PATCH v10 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-11-01 20:01 ` [dpdk-dev] [PATCH v10 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-11-01 20:01 ` [dpdk-dev] [PATCH v10 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 00/23] Add DLB2 PMD Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 09/23] event/dlb2: add xstats Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 13/23] event/dlb2: add port setup Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 14/23] event/dlb2: add port link Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-11-01 23:37 ` [dpdk-dev] [PATCH v11 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-11-01 23:38 ` [dpdk-dev] [PATCH v11 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-11-01 23:38 ` [dpdk-dev] [PATCH v11 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-11-01 23:38 ` [dpdk-dev] [PATCH v11 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-11-01 23:38 ` [dpdk-dev] [PATCH v11 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-11-02 8:49 ` [dpdk-dev] [PATCH v11 00/23] Add DLB2 PMD Jerin Jacob
2020-10-17 18:20 ` [dpdk-dev] [PATCH v2 02/22] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-18 8:57 ` Jerin Jacob
2020-10-20 14:08 ` McDaniel, Timothy
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 03/22] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-20 14:01 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 04/22] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 05/22] event/dlb2: add inline functions Timothy McDaniel
2020-10-18 8:59 ` Jerin Jacob
2020-10-20 14:08 ` McDaniel, Timothy
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 06/22] event/dlb2: add probe Timothy McDaniel
2020-10-18 8:39 ` Jerin Jacob
2020-10-20 14:04 ` McDaniel, Timothy
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 07/22] event/dlb2: add xstats Timothy McDaniel
2020-10-20 14:01 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 08/22] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-20 14:01 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 09/22] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-20 14:02 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 10/22] event/dlb2: add queue setup Timothy McDaniel
2020-10-20 14:01 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 11/22] event/dlb2: add port setup Timothy McDaniel
2020-10-20 14:02 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 12/22] event/dlb2: add port link Timothy McDaniel
2020-10-20 14:02 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 13/22] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 14/22] event/dlb2: add eventdev start Timothy McDaniel
2020-10-20 14:04 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 15/22] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 16/22] event/dlb2: add dequeue " Timothy McDaniel
2020-10-20 14:04 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 17/22] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-20 14:04 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 18/22] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-18 9:13 ` Jerin Jacob
2020-10-20 14:12 ` McDaniel, Timothy
2020-10-19 10:01 ` Kinsella, Ray
2020-10-20 14:05 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 19/22] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 20/22] event/dlb2: add queue and port release Timothy McDaniel
2020-10-20 14:04 ` Eads, Gage
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 21/22] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-17 18:21 ` [dpdk-dev] [PATCH v2 22/22] doc: add new DLB2 eventdev driver to relnotes Timothy McDaniel
2020-10-18 9:22 ` Jerin Jacob
2020-10-20 14:13 ` McDaniel, Timothy
2020-10-24 13:06 ` [dpdk-dev] [PATCH v2 00/22] Add DLB2 PMD Thomas Monjalon
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 00/23] " Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-24 12:58 ` Jerin Jacob
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-23 18:30 ` [dpdk-dev] [PATCH v3 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-29 15:24 ` [dpdk-dev] [PATCH v4 00/23] Add DLB2 PMD Timothy McDaniel
2020-10-29 15:24 ` [dpdk-dev] [PATCH v4 01/23] event/dlb2: add documentation and meson build infrastructure Timothy McDaniel
2020-10-29 15:24 ` [dpdk-dev] [PATCH v4 02/23] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-29 15:24 ` [dpdk-dev] [PATCH v4 03/23] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-29 15:29 ` Stephen Hemminger
2020-10-29 16:07 ` McDaniel, Timothy
2020-10-29 15:30 ` Stephen Hemminger
2020-10-29 16:10 ` McDaniel, Timothy
2020-10-29 15:24 ` [dpdk-dev] [PATCH v4 04/23] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-29 15:24 ` [dpdk-dev] [PATCH v4 05/23] event/dlb2: add inline functions Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 06/23] event/dlb2: add eventdev probe Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 07/23] event/dlb2: add flexible interface Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 08/23] event/dlb2: add probe-time hardware init Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 09/23] event/dlb2: add xstats Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 10/23] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 11/23] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 12/23] event/dlb2: add queue setup Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 13/23] event/dlb2: add port setup Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 14/23] event/dlb2: add port link Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 15/23] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 16/23] event/dlb2: add eventdev start Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 17/23] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 18/23] event/dlb2: add dequeue " Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 19/23] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 20/23] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 21/23] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 22/23] event/dlb2: add queue and port release Timothy McDaniel
2020-10-29 15:25 ` [dpdk-dev] [PATCH v4 23/23] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-09-11 20:26 ` [dpdk-dev] [PATCH 02/22] event/dlb2: add dynamic logging Timothy McDaniel
2020-10-06 16:52 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 03/22] event/dlb2: add private data structures and constants Timothy McDaniel
2020-10-06 16:52 ` Eads, Gage
2020-10-07 16:14 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 04/22] event/dlb2: add definitions shared with LKM or shared code Timothy McDaniel
2020-10-06 19:26 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 05/22] event/dlb2: add inline functions Timothy McDaniel
2020-10-06 21:33 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 06/22] event/dlb2: add probe Timothy McDaniel
2020-10-07 16:56 ` Eads, Gage
2020-10-18 9:05 ` Jerin Jacob
2020-10-20 14:11 ` McDaniel, Timothy
2020-09-11 20:26 ` [dpdk-dev] [PATCH 07/22] event/dlb2: add xstats Timothy McDaniel
2020-09-17 20:58 ` Chen, Mike Ximing
2020-09-17 21:26 ` McDaniel, Timothy
2020-09-18 0:37 ` Chen, Mike Ximing
2020-09-18 8:39 ` Bruce Richardson
2020-10-07 18:47 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 08/22] event/dlb2: add infos get and configure Timothy McDaniel
2020-10-07 19:14 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 09/22] event/dlb2: add queue and port default conf Timothy McDaniel
2020-10-07 19:15 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 10/22] event/dlb2: add queue setup Timothy McDaniel
2020-10-07 19:26 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 11/22] event/dlb2: add port setup Timothy McDaniel
2020-10-07 20:34 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 12/22] event/dlb2: add port link Timothy McDaniel
2020-10-07 20:40 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 13/22] event/dlb2: add port unlink and port unlinks in progress Timothy McDaniel
2020-10-07 20:44 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 14/22] event/dlb2: add eventdev start Timothy McDaniel
2020-10-07 20:51 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 15/22] event/dlb2: add enqueue and its burst variants Timothy McDaniel
2020-10-07 21:02 ` Eads, Gage
2020-09-11 20:26 ` Timothy McDaniel [this message]
2020-10-07 21:18 ` [dpdk-dev] [PATCH 16/22] event/dlb2: add dequeue " Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 17/22] event/dlb2: add eventdev stop and close Timothy McDaniel
2020-10-07 21:21 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 18/22] event/dlb2: add PMD's token pop public interface Timothy McDaniel
2020-10-07 21:24 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 19/22] event/dlb2: add PMD self-tests Timothy McDaniel
2020-10-07 21:33 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 20/22] event/dlb2: add queue and port release Timothy McDaniel
2020-10-07 21:55 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 21/22] event/dlb2: add timeout ticks entry point Timothy McDaniel
2020-10-07 21:58 ` Eads, Gage
2020-09-11 20:26 ` [dpdk-dev] [PATCH 22/22] doc: add new DLB2 eventdev driver to relnotes Timothy McDaniel
2020-10-07 22:04 ` Eads, Gage
2020-09-21 17:11 ` [dpdk-dev] [PATCH 00/22] Add DLB2 PMD Jerin Jacob
2020-09-21 17:15 ` McDaniel, Timothy
2020-09-29 18:41 ` Jerin Jacob
2020-09-29 18:46 ` McDaniel, Timothy
2020-09-30 16:10 ` McDaniel, Timothy
2020-09-29 18:46 ` Jerin Jacob
2020-09-30 16:14 ` McDaniel, Timothy
Reply instructions:
You may reply publicly to this message via plain-text email
using any one of the following methods:
* Save the following mbox file, import it into your mail client,
and reply-to-all from there: mbox
Avoid top-posting and favor interleaved quoting:
https://en.wikipedia.org/wiki/Posting_style#Interleaved_style
* Reply using the --to, --cc, and --in-reply-to
switches of git-send-email(1):
git send-email \
--in-reply-to=1599855987-25976-17-git-send-email-timothy.mcdaniel@intel.com \
--to=timothy.mcdaniel@intel.com \
--cc=dev@dpdk.org \
--cc=erik.g.carrillo@intel.com \
--cc=gage.eads@intel.com \
--cc=harry.van.haaren@intel.com \
--cc=jerinj@marvell.com \
/path/to/YOUR_REPLY
https://kernel.org/pub/software/scm/git/docs/git-send-email.html
* If your mail client supports setting the In-Reply-To header
via mailto: links, try the mailto: link
Be sure your reply has a Subject: header at the top and a blank line
before the message body.
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).