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, thomas@monjalon.net Subject: [dpdk-dev] [PATCH v10 17/23] event/dlb: add enqueue and its burst variants Date: Fri, 30 Oct 2020 13:27:32 -0500 Message-ID: <1604082458-15368-18-git-send-email-timothy.mcdaniel@intel.com> (raw) In-Reply-To: <1604082458-15368-1-git-send-email-timothy.mcdaniel@intel.com> Add support for enqueue and its variants. Signed-off-by: Timothy McDaniel <timothy.mcdaniel@intel.com> Reviewed-by: Gage Eads <gage.eads@intel.com> --- doc/guides/eventdevs/dlb.rst | 163 ++++++++++- drivers/event/dlb/dlb.c | 682 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 844 insertions(+), 1 deletion(-) diff --git a/doc/guides/eventdevs/dlb.rst b/doc/guides/eventdevs/dlb.rst index f106a07..ae126c4 100644 --- a/doc/guides/eventdevs/dlb.rst +++ b/doc/guides/eventdevs/dlb.rst @@ -118,7 +118,7 @@ the DLB does not limit the number of flows a queue can track. In the DLB, all load-balanced queues can use the full 16-bit flow ID range. Load-balanced and Directed Ports -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ DLB ports come in two flavors: load-balanced and directed. The eventdev API does not have the same concept, but it has a similar one: ports and queues that @@ -157,3 +157,164 @@ type (atomic, ordered, or parallel) is not preserved, and an event's sched_type will be set to ``RTE_SCHED_TYPE_ATOMIC`` when it is dequeued from a directed port. +Flow ID +~~~~~~~ + +The flow ID field is not preserved in the event when it is scheduled in the +DLB, because the DLB hardware control word format does not have sufficient +space to preserve every event field. As a result, the flow ID specified with +the enqueued event will not be in the dequeued event. If this field is +required, the application should pass it through an out-of-band path (for +example in the mbuf's udata64 field, if the event points to an mbuf) or +reconstruct the flow ID after receiving the event. + +Also, the DLB hardware control word supports a 16-bit flow ID. Since struct +rte_event's flow_id field is 20 bits, the DLB PMD drops the most significant +four bits from the event's flow ID. + +Hardware Credits +~~~~~~~~~~~~~~~~ + +DLB uses a hardware credit scheme to prevent software from overflowing hardware +event storage, with each unit of storage represented by a credit. A port spends +a credit to enqueue an event, and hardware refills the ports with credits as the +events are scheduled to ports. Refills come from credit pools, and each port is +a member of a load-balanced credit pool and a directed credit pool. The +load-balanced credits are used to enqueue to load-balanced queues, and directed +credits are used for directed queues. + +A DLB eventdev contains one load-balanced and one directed credit pool. These +pools' sizes are controlled by the nb_events_limit field in struct +rte_event_dev_config. The load-balanced pool is sized to contain +nb_events_limit credits, and the directed pool is sized to contain +nb_events_limit/4 credits. The directed pool size can be overridden with the +num_dir_credits vdev argument, like so: + + .. code-block:: console + + --vdev=dlb1_event,num_dir_credits=<value> + +This can be used if the default allocation is too low or too high for the +specific application needs. The PMD also supports a vdev arg that limits the +max_num_events reported by rte_event_dev_info_get(): + + .. code-block:: console + + --vdev=dlb1_event,max_num_events=<value> + +By default, max_num_events is reported as the total available load-balanced +credits. If multiple DLB-based applications are being used, it may be desirable +to control how many load-balanced credits each application uses, particularly +when application(s) are written to configure nb_events_limit equal to the +reported max_num_events. + +Each port is a member of both credit pools. A port's credit allocation is +defined by its low watermark, high watermark, and refill quanta. These three +parameters are calculated by the dlb PMD like so: + +- The load-balanced high watermark is set to the port's enqueue_depth. + The directed high watermark is set to the minimum of the enqueue_depth and + the directed pool size divided by the total number of ports. +- The refill quanta is set to half the high watermark. +- The low watermark is set to the minimum of 16 and the refill quanta. + +When the eventdev is started, each port is pre-allocated a high watermark's +worth of credits. For example, if an eventdev contains four ports with enqueue +depths of 32 and a load-balanced credit pool size of 4096, each port will start +with 32 load-balanced credits, and there will be 3968 credits available to +replenish the ports. Thus, a single port is not capable of enqueueing up to the +nb_events_limit (without any events being dequeued), since the other ports are +retaining their initial credit allocation; in short, all ports must enqueue in +order to reach the limit. + +If a port attempts to enqueue and has no credits available, the enqueue +operation will fail and the application must retry the enqueue. Credits are +replenished asynchronously by the DLB hardware. + +Software Credits +~~~~~~~~~~~~~~~~ + +The DLB is a "closed system" event dev, and the DLB PMD layers a software +credit scheme on top of the hardware credit scheme in order to comply with +the per-port backpressure described in the eventdev API. + +The DLB's hardware scheme is local to a queue/pipeline stage: a port spends a +credit when it enqueues to a queue, and credits are later replenished after the +events are dequeued and released. + +In the software credit scheme, a credit is consumed when a new (.op = +RTE_EVENT_OP_NEW) event is injected into the system, and the credit is +replenished when the event is released from the system (either explicitly with +RTE_EVENT_OP_RELEASE or implicitly in dequeue_burst()). + +In this model, an event is "in the system" from its first enqueue into eventdev +until it is last dequeued. If the event goes through multiple event queues, it +is still considered "in the system" while a worker thread is processing it. + +A port will fail to enqueue if the number of events in the system exceeds its +``new_event_threshold`` (specified at port setup time). A port will also fail +to enqueue if it lacks enough hardware credits to enqueue; load-balanced +credits are used to enqueue to a load-balanced queue, and directed credits are +used to enqueue to a directed queue. + +The out-of-credit situations are typically transient, and an eventdev +application using the DLB ought to retry its enqueues if they fail. +If enqueue fails, DLB PMD sets rte_errno as follows: + +- -ENOSPC: Credit exhaustion (either hardware or software) +- -EINVAL: Invalid argument, such as port ID, queue ID, or sched_type. + +Depending on the pipeline the application has constructed, it's possible to +enter a credit deadlock scenario wherein the worker thread lacks the credit +to enqueue an event, and it must dequeue an event before it can recover the +credit. If the worker thread retries its enqueue indefinitely, it will not +make forward progress. Such deadlock is possible if the application has event +"loops", in which an event in dequeued from queue A and later enqueued back to +queue A. + +Due to this, workers should stop retrying after a time, release the events it +is attempting to enqueue, and dequeue more events. It is important that the +worker release the events and don't simply set them aside to retry the enqueue +again later, because the port has limited history list size (by default, twice +the port's dequeue_depth). + +Priority +~~~~~~~~ + +The DLB supports event priority and per-port queue service priority, as +described in the eventdev header file. The DLB does not support 'global' event +queue priority established at queue creation time. + +DLB supports 8 event and queue service priority levels. For both priority +types, the PMD uses the upper three bits of the priority field to determine the +DLB priority, discarding the 5 least significant bits. The 5 least significant +event priority bits are not preserved when an event is enqueued. + +Atomic Inflights Allocation +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +In the last stage prior to scheduling an atomic event to a CQ, DLB holds the +inflight event in a temporary buffer that is divided among load-balanced +queues. If a queue's atomic buffer storage fills up, this can result in +head-of-line-blocking. For example: + +- An LDB queue allocated N atomic buffer entries +- All N entries are filled with events from flow X, which is pinned to CQ 0. + +Until CQ 0 releases 1+ events, no other atomic flows for that LDB queue can be +scheduled. The likelihood of this case depends on the eventdev configuration, +traffic behavior, event processing latency, potential for a worker to be +interrupted or otherwise delayed, etc. + +By default, the PMD allocates 16 buffer entries for each load-balanced queue, +which provides an even division across all 128 queues but potentially wastes +buffer space (e.g. if not all queues are used, or aren't used for atomic +scheduling). + +The PMD provides a dev arg to override the default per-queue allocation. To +increase a vdev's per-queue atomic-inflight allocation to (for example) 64: + + .. code-block:: console + + --vdev=dlb1_event,atm_inflights=64 + diff --git a/drivers/event/dlb/dlb.c b/drivers/event/dlb/dlb.c index 780ff7d..4d65a7f 100644 --- a/drivers/event/dlb/dlb.c +++ b/drivers/event/dlb/dlb.c @@ -71,6 +71,25 @@ static struct rte_event_dev_info evdev_dlb_default_info = { struct process_local_port_data dlb_port[DLB_MAX_NUM_PORTS][NUM_DLB_PORT_TYPES]; +static inline uint16_t +dlb_event_enqueue_delayed(void *event_port, + const struct rte_event events[]); + +static inline uint16_t +dlb_event_enqueue_burst_delayed(void *event_port, + const struct rte_event events[], + uint16_t num); + +static inline uint16_t +dlb_event_enqueue_new_burst_delayed(void *event_port, + const struct rte_event events[], + uint16_t num); + +static inline uint16_t +dlb_event_enqueue_forward_burst_delayed(void *event_port, + const struct rte_event events[], + uint16_t num); + uint32_t dlb_get_queue_depth(struct dlb_eventdev *dlb, struct dlb_eventdev_queue *queue) @@ -2135,6 +2154,664 @@ dlb_eventdev_start(struct rte_eventdev *dev) return 0; } +static inline int +dlb_check_enqueue_sw_credits(struct dlb_eventdev *dlb, + struct dlb_eventdev_port *ev_port) +{ + uint32_t sw_inflights = __atomic_load_n(&dlb->inflights, + __ATOMIC_SEQ_CST); + const int num = 1; + + if (unlikely(ev_port->inflight_max < sw_inflights)) { + DLB_INC_STAT(ev_port->stats.traffic.tx_nospc_inflight_max, 1); + rte_errno = -ENOSPC; + return 1; + } + + if (ev_port->inflight_credits < num) { + /* check if event enqueue brings ev_port over max threshold */ + uint32_t credit_update_quanta = ev_port->credit_update_quanta; + + if (sw_inflights + credit_update_quanta > + dlb->new_event_limit) { + DLB_INC_STAT( + ev_port->stats.traffic.tx_nospc_new_event_limit, + 1); + rte_errno = -ENOSPC; + return 1; + } + + __atomic_fetch_add(&dlb->inflights, credit_update_quanta, + __ATOMIC_SEQ_CST); + ev_port->inflight_credits += (credit_update_quanta); + + if (ev_port->inflight_credits < num) { + DLB_INC_STAT( + ev_port->stats.traffic.tx_nospc_inflight_credits, + 1); + rte_errno = -ENOSPC; + return 1; + } + } + + return 0; +} + +static inline void +dlb_replenish_sw_credits(struct dlb_eventdev *dlb, + struct dlb_eventdev_port *ev_port) +{ + uint16_t quanta = ev_port->credit_update_quanta; + + if (ev_port->inflight_credits >= quanta * 2) { + /* Replenish credits, saving one quanta for enqueues */ + uint16_t val = ev_port->inflight_credits - quanta; + + __atomic_fetch_sub(&dlb->inflights, val, __ATOMIC_SEQ_CST); + ev_port->inflight_credits -= val; + } +} + +static __rte_always_inline uint16_t +dlb_read_pc(struct process_local_port_data *port_data, bool ldb) +{ + volatile uint16_t *popcount; + + if (ldb) + popcount = port_data->ldb_popcount; + else + popcount = port_data->dir_popcount; + + return *popcount; +} + +static inline int +dlb_check_enqueue_hw_ldb_credits(struct dlb_port *qm_port, + struct process_local_port_data *port_data) +{ + if (unlikely(qm_port->cached_ldb_credits == 0)) { + uint16_t pc; + + pc = dlb_read_pc(port_data, true); + + qm_port->cached_ldb_credits = pc - + qm_port->ldb_pushcount_at_credit_expiry; + if (unlikely(qm_port->cached_ldb_credits == 0)) { + DLB_INC_STAT( + qm_port->ev_port->stats.traffic.tx_nospc_ldb_hw_credits, + 1); + + DLB_LOG_DBG("ldb credits exhausted\n"); + return 1; + } + qm_port->ldb_pushcount_at_credit_expiry += + qm_port->cached_ldb_credits; + } + + return 0; +} + +static inline int +dlb_check_enqueue_hw_dir_credits(struct dlb_port *qm_port, + struct process_local_port_data *port_data) +{ + if (unlikely(qm_port->cached_dir_credits == 0)) { + uint16_t pc; + + pc = dlb_read_pc(port_data, false); + + qm_port->cached_dir_credits = pc - + qm_port->dir_pushcount_at_credit_expiry; + + if (unlikely(qm_port->cached_dir_credits == 0)) { + DLB_INC_STAT( + qm_port->ev_port->stats.traffic.tx_nospc_dir_hw_credits, + 1); + + DLB_LOG_DBG("dir credits exhausted\n"); + return 1; + } + qm_port->dir_pushcount_at_credit_expiry += + qm_port->cached_dir_credits; + } + + return 0; +} + +static inline int +dlb_event_enqueue_prep(struct dlb_eventdev_port *ev_port, + struct dlb_port *qm_port, + const struct rte_event ev[], + struct process_local_port_data *port_data, + uint8_t *sched_type, + uint8_t *queue_id) +{ + struct dlb_eventdev *dlb = ev_port->dlb; + struct dlb_eventdev_queue *ev_queue; + uint16_t *cached_credits = NULL; + struct dlb_queue *qm_queue; + + ev_queue = &dlb->ev_queues[ev->queue_id]; + qm_queue = &ev_queue->qm_queue; + *queue_id = qm_queue->id; + + /* Ignore sched_type and hardware credits on release events */ + if (ev->op == RTE_EVENT_OP_RELEASE) + goto op_check; + + if (!qm_queue->is_directed) { + /* Load balanced destination queue */ + + if (dlb_check_enqueue_hw_ldb_credits(qm_port, port_data)) { + rte_errno = -ENOSPC; + return 1; + } + cached_credits = &qm_port->cached_ldb_credits; + + switch (ev->sched_type) { + case RTE_SCHED_TYPE_ORDERED: + DLB_LOG_DBG("dlb: put_qe: RTE_SCHED_TYPE_ORDERED\n"); + if (qm_queue->sched_type != RTE_SCHED_TYPE_ORDERED) { + DLB_LOG_ERR("dlb: tried to send ordered event to unordered queue %d\n", + *queue_id); + rte_errno = -EINVAL; + return 1; + } + *sched_type = DLB_SCHED_ORDERED; + break; + case RTE_SCHED_TYPE_ATOMIC: + DLB_LOG_DBG("dlb: put_qe: RTE_SCHED_TYPE_ATOMIC\n"); + *sched_type = DLB_SCHED_ATOMIC; + break; + case RTE_SCHED_TYPE_PARALLEL: + DLB_LOG_DBG("dlb: put_qe: RTE_SCHED_TYPE_PARALLEL\n"); + if (qm_queue->sched_type == RTE_SCHED_TYPE_ORDERED) + *sched_type = DLB_SCHED_ORDERED; + else + *sched_type = DLB_SCHED_UNORDERED; + break; + default: + DLB_LOG_ERR("Unsupported LDB sched type in put_qe\n"); + DLB_INC_STAT(ev_port->stats.tx_invalid, 1); + rte_errno = -EINVAL; + return 1; + } + } else { + /* Directed destination queue */ + + if (dlb_check_enqueue_hw_dir_credits(qm_port, port_data)) { + rte_errno = -ENOSPC; + return 1; + } + cached_credits = &qm_port->cached_dir_credits; + + DLB_LOG_DBG("dlb: put_qe: RTE_SCHED_TYPE_DIRECTED\n"); + + *sched_type = DLB_SCHED_DIRECTED; + } + +op_check: + switch (ev->op) { + case RTE_EVENT_OP_NEW: + /* Check that a sw credit is available */ + if (dlb_check_enqueue_sw_credits(dlb, ev_port)) { + rte_errno = -ENOSPC; + return 1; + } + ev_port->inflight_credits--; + (*cached_credits)--; + break; + case RTE_EVENT_OP_FORWARD: + /* Check for outstanding_releases underflow. If this occurs, + * the application is not using the EVENT_OPs correctly; for + * example, forwarding or releasing events that were not + * dequeued. + */ + RTE_ASSERT(ev_port->outstanding_releases > 0); + ev_port->outstanding_releases--; + qm_port->issued_releases++; + (*cached_credits)--; + break; + case RTE_EVENT_OP_RELEASE: + ev_port->inflight_credits++; + /* Check for outstanding_releases underflow. If this occurs, + * the application is not using the EVENT_OPs correctly; for + * example, forwarding or releasing events that were not + * dequeued. + */ + RTE_ASSERT(ev_port->outstanding_releases > 0); + ev_port->outstanding_releases--; + qm_port->issued_releases++; + /* Replenish s/w credits if enough are cached */ + dlb_replenish_sw_credits(dlb, ev_port); + break; + } + + DLB_INC_STAT(ev_port->stats.tx_op_cnt[ev->op], 1); + DLB_INC_STAT(ev_port->stats.traffic.tx_ok, 1); + +#ifndef RTE_LIBRTE_PMD_DLB_QUELL_STATS + if (ev->op != RTE_EVENT_OP_RELEASE) { + DLB_INC_STAT(ev_port->stats.enq_ok[ev->queue_id], 1); + DLB_INC_STAT(ev_port->stats.tx_sched_cnt[*sched_type], 1); + } +#endif + + return 0; +} + +static uint8_t cmd_byte_map[NUM_DLB_PORT_TYPES][DLB_NUM_HW_SCHED_TYPES] = { + { + /* Load-balanced cmd bytes */ + [RTE_EVENT_OP_NEW] = DLB_NEW_CMD_BYTE, + [RTE_EVENT_OP_FORWARD] = DLB_FWD_CMD_BYTE, + [RTE_EVENT_OP_RELEASE] = DLB_COMP_CMD_BYTE, + }, + { + /* Directed cmd bytes */ + [RTE_EVENT_OP_NEW] = DLB_NEW_CMD_BYTE, + [RTE_EVENT_OP_FORWARD] = DLB_NEW_CMD_BYTE, + [RTE_EVENT_OP_RELEASE] = DLB_NOOP_CMD_BYTE, + }, +}; + +static inline void +dlb_event_build_hcws(struct dlb_port *qm_port, + const struct rte_event ev[], + int num, + uint8_t *sched_type, + uint8_t *queue_id) +{ + struct dlb_enqueue_qe *qe; + uint16_t sched_word[4]; + __m128i sse_qe[2]; + int i; + + qe = qm_port->qe4; + + sse_qe[0] = _mm_setzero_si128(); + sse_qe[1] = _mm_setzero_si128(); + + switch (num) { + case 4: + /* Construct the metadata portion of two HCWs in one 128b SSE + * register. HCW metadata is constructed in the SSE registers + * like so: + * sse_qe[0][63:0]: qe[0]'s metadata + * sse_qe[0][127:64]: qe[1]'s metadata + * sse_qe[1][63:0]: qe[2]'s metadata + * sse_qe[1][127:64]: qe[3]'s metadata + */ + + /* Convert the event operation into a command byte and store it + * in the metadata: + * sse_qe[0][63:56] = cmd_byte_map[is_directed][ev[0].op] + * sse_qe[0][127:120] = cmd_byte_map[is_directed][ev[1].op] + * sse_qe[1][63:56] = cmd_byte_map[is_directed][ev[2].op] + * sse_qe[1][127:120] = cmd_byte_map[is_directed][ev[3].op] + */ +#define DLB_QE_CMD_BYTE 7 + sse_qe[0] = _mm_insert_epi8(sse_qe[0], + cmd_byte_map[qm_port->is_directed][ev[0].op], + DLB_QE_CMD_BYTE); + sse_qe[0] = _mm_insert_epi8(sse_qe[0], + cmd_byte_map[qm_port->is_directed][ev[1].op], + DLB_QE_CMD_BYTE + 8); + sse_qe[1] = _mm_insert_epi8(sse_qe[1], + cmd_byte_map[qm_port->is_directed][ev[2].op], + DLB_QE_CMD_BYTE); + sse_qe[1] = _mm_insert_epi8(sse_qe[1], + cmd_byte_map[qm_port->is_directed][ev[3].op], + DLB_QE_CMD_BYTE + 8); + + /* Store priority, scheduling type, and queue ID in the sched + * word array because these values are re-used when the + * destination is a directed queue. + */ + sched_word[0] = EV_TO_DLB_PRIO(ev[0].priority) << 10 | + sched_type[0] << 8 | + queue_id[0]; + sched_word[1] = EV_TO_DLB_PRIO(ev[1].priority) << 10 | + sched_type[1] << 8 | + queue_id[1]; + sched_word[2] = EV_TO_DLB_PRIO(ev[2].priority) << 10 | + sched_type[2] << 8 | + queue_id[2]; + sched_word[3] = EV_TO_DLB_PRIO(ev[3].priority) << 10 | + sched_type[3] << 8 | + queue_id[3]; + + /* Store the event priority, scheduling type, and queue ID in + * the metadata: + * sse_qe[0][31:16] = sched_word[0] + * sse_qe[0][95:80] = sched_word[1] + * sse_qe[1][31:16] = sched_word[2] + * sse_qe[1][95:80] = sched_word[3] + */ +#define DLB_QE_QID_SCHED_WORD 1 + sse_qe[0] = _mm_insert_epi16(sse_qe[0], + sched_word[0], + DLB_QE_QID_SCHED_WORD); + sse_qe[0] = _mm_insert_epi16(sse_qe[0], + sched_word[1], + DLB_QE_QID_SCHED_WORD + 4); + sse_qe[1] = _mm_insert_epi16(sse_qe[1], + sched_word[2], + DLB_QE_QID_SCHED_WORD); + sse_qe[1] = _mm_insert_epi16(sse_qe[1], + sched_word[3], + DLB_QE_QID_SCHED_WORD + 4); + + /* If the destination is a load-balanced queue, store the lock + * ID. If it is a directed queue, DLB places this field in + * bytes 10-11 of the received QE, so we format it accordingly: + * sse_qe[0][47:32] = dir queue ? sched_word[0] : flow_id[0] + * sse_qe[0][111:96] = dir queue ? sched_word[1] : flow_id[1] + * sse_qe[1][47:32] = dir queue ? sched_word[2] : flow_id[2] + * sse_qe[1][111:96] = dir queue ? sched_word[3] : flow_id[3] + */ +#define DLB_QE_LOCK_ID_WORD 2 + sse_qe[0] = _mm_insert_epi16(sse_qe[0], + (sched_type[0] == DLB_SCHED_DIRECTED) ? + sched_word[0] : ev[0].flow_id, + DLB_QE_LOCK_ID_WORD); + sse_qe[0] = _mm_insert_epi16(sse_qe[0], + (sched_type[1] == DLB_SCHED_DIRECTED) ? + sched_word[1] : ev[1].flow_id, + DLB_QE_LOCK_ID_WORD + 4); + sse_qe[1] = _mm_insert_epi16(sse_qe[1], + (sched_type[2] == DLB_SCHED_DIRECTED) ? + sched_word[2] : ev[2].flow_id, + DLB_QE_LOCK_ID_WORD); + sse_qe[1] = _mm_insert_epi16(sse_qe[1], + (sched_type[3] == DLB_SCHED_DIRECTED) ? + sched_word[3] : ev[3].flow_id, + DLB_QE_LOCK_ID_WORD + 4); + + /* Store the event type and sub event type in the metadata: + * sse_qe[0][15:0] = flow_id[0] + * sse_qe[0][79:64] = flow_id[1] + * sse_qe[1][15:0] = flow_id[2] + * sse_qe[1][79:64] = flow_id[3] + */ +#define DLB_QE_EV_TYPE_WORD 0 + sse_qe[0] = _mm_insert_epi16(sse_qe[0], + ev[0].sub_event_type << 8 | + ev[0].event_type, + DLB_QE_EV_TYPE_WORD); + sse_qe[0] = _mm_insert_epi16(sse_qe[0], + ev[1].sub_event_type << 8 | + ev[1].event_type, + DLB_QE_EV_TYPE_WORD + 4); + sse_qe[1] = _mm_insert_epi16(sse_qe[1], + ev[2].sub_event_type << 8 | + ev[2].event_type, + DLB_QE_EV_TYPE_WORD); + sse_qe[1] = _mm_insert_epi16(sse_qe[1], + ev[3].sub_event_type << 8 | + ev[3].event_type, + DLB_QE_EV_TYPE_WORD + 4); + + /* Store the metadata to memory (use the double-precision + * _mm_storeh_pd because there is no integer function for + * storing the upper 64b): + * qe[0] metadata = sse_qe[0][63:0] + * qe[1] metadata = sse_qe[0][127:64] + * qe[2] metadata = sse_qe[1][63:0] + * qe[3] metadata = sse_qe[1][127:64] + */ + _mm_storel_epi64((__m128i *)&qe[0].u.opaque_data, sse_qe[0]); + _mm_storeh_pd((double *)&qe[1].u.opaque_data, + (__m128d) sse_qe[0]); + _mm_storel_epi64((__m128i *)&qe[2].u.opaque_data, sse_qe[1]); + _mm_storeh_pd((double *)&qe[3].u.opaque_data, + (__m128d) sse_qe[1]); + + qe[0].data = ev[0].u64; + qe[1].data = ev[1].u64; + qe[2].data = ev[2].u64; + qe[3].data = ev[3].u64; + + break; + case 3: + case 2: + case 1: + for (i = 0; i < num; i++) { + qe[i].cmd_byte = + cmd_byte_map[qm_port->is_directed][ev[i].op]; + qe[i].sched_type = sched_type[i]; + qe[i].data = ev[i].u64; + qe[i].qid = queue_id[i]; + qe[i].priority = EV_TO_DLB_PRIO(ev[i].priority); + qe[i].lock_id = ev[i].flow_id; + if (sched_type[i] == DLB_SCHED_DIRECTED) { + struct dlb_msg_info *info = + (struct dlb_msg_info *)&qe[i].lock_id; + + info->qid = queue_id[i]; + info->sched_type = DLB_SCHED_DIRECTED; + info->priority = qe[i].priority; + } + qe[i].u.event_type.major = ev[i].event_type; + qe[i].u.event_type.sub = ev[i].sub_event_type; + } + break; + case 0: + break; + } +} + +static inline void +dlb_construct_token_pop_qe(struct dlb_port *qm_port, int idx) +{ + struct dlb_cq_pop_qe *qe = (void *)qm_port->qe4; + int num = qm_port->owed_tokens; + + if (qm_port->use_rsvd_token_scheme) { + /* Check if there's a deficit of reserved tokens, and return + * early if there are no (unreserved) tokens to consume. + */ + if (num <= qm_port->cq_rsvd_token_deficit) { + qm_port->cq_rsvd_token_deficit -= num; + qm_port->owed_tokens = 0; + return; + } + num -= qm_port->cq_rsvd_token_deficit; + qm_port->cq_rsvd_token_deficit = 0; + } + + qe[idx].cmd_byte = DLB_POP_CMD_BYTE; + qe[idx].tokens = num - 1; + + qm_port->owed_tokens = 0; +} + +static __rte_always_inline void +dlb_pp_write(struct dlb_enqueue_qe *qe4, + struct process_local_port_data *port_data) +{ + dlb_movdir64b(port_data->pp_addr, qe4); +} + +static inline void +dlb_hw_do_enqueue(struct dlb_port *qm_port, + bool do_sfence, + struct process_local_port_data *port_data) +{ + DLB_LOG_DBG("dlb: Flushing QE(s) to DLB\n"); + + /* Since MOVDIR64B is weakly-ordered, use an SFENCE to ensure that + * application writes complete before enqueueing the release HCW. + */ + if (do_sfence) + rte_wmb(); + + dlb_pp_write(qm_port->qe4, port_data); +} + +static inline int +dlb_consume_qe_immediate(struct dlb_port *qm_port, int num) +{ + struct process_local_port_data *port_data; + struct dlb_cq_pop_qe *qe; + + RTE_ASSERT(qm_port->config_state == DLB_CONFIGURED); + + if (qm_port->use_rsvd_token_scheme) { + /* Check if there's a deficit of reserved tokens, and return + * early if there are no (unreserved) tokens to consume. + */ + if (num <= qm_port->cq_rsvd_token_deficit) { + qm_port->cq_rsvd_token_deficit -= num; + qm_port->owed_tokens = 0; + return 0; + } + num -= qm_port->cq_rsvd_token_deficit; + qm_port->cq_rsvd_token_deficit = 0; + } + + qe = qm_port->consume_qe; + + qe->tokens = num - 1; + qe->int_arm = 0; + + /* No store fence needed since no pointer is being sent, and CQ token + * pops can be safely reordered with other HCWs. + */ + port_data = &dlb_port[qm_port->id][PORT_TYPE(qm_port)]; + + dlb_movntdq_single(port_data->pp_addr, qe); + + DLB_LOG_DBG("dlb: consume immediate - %d QEs\n", num); + + qm_port->owed_tokens = 0; + + return 0; +} + +static inline uint16_t +__dlb_event_enqueue_burst(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + struct dlb_eventdev_port *ev_port = event_port; + struct dlb_port *qm_port = &ev_port->qm_port; + struct process_local_port_data *port_data; + int i; + + RTE_ASSERT(ev_port->enq_configured); + RTE_ASSERT(events != NULL); + + rte_errno = 0; + i = 0; + + port_data = &dlb_port[qm_port->id][PORT_TYPE(qm_port)]; + + while (i < num) { + uint8_t sched_types[DLB_NUM_QES_PER_CACHE_LINE]; + uint8_t queue_ids[DLB_NUM_QES_PER_CACHE_LINE]; + int pop_offs = 0; + int j = 0; + + memset(qm_port->qe4, + 0, + DLB_NUM_QES_PER_CACHE_LINE * + sizeof(struct dlb_enqueue_qe)); + + for (; j < DLB_NUM_QES_PER_CACHE_LINE && (i + j) < num; j++) { + const struct rte_event *ev = &events[i + j]; + + if (dlb_event_enqueue_prep(ev_port, qm_port, ev, + port_data, &sched_types[j], + &queue_ids[j])) + break; + } + + if (j == 0) + break; + + dlb_event_build_hcws(qm_port, &events[i], j - pop_offs, + sched_types, queue_ids); + + dlb_hw_do_enqueue(qm_port, i == 0, port_data); + + /* Don't include the token pop QE in the enqueue count */ + i += j - pop_offs; + + /* Don't interpret j < DLB_NUM_... as out-of-credits if + * pop_offs != 0 + */ + if (j < DLB_NUM_QES_PER_CACHE_LINE && pop_offs == 0) + break; + } + + RTE_ASSERT(!((i == 0 && rte_errno != -ENOSPC))); + + return i; +} + +static inline uint16_t +dlb_event_enqueue_burst(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + return __dlb_event_enqueue_burst(event_port, events, num); +} + +static inline uint16_t +dlb_event_enqueue_burst_delayed(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + return __dlb_event_enqueue_burst(event_port, events, num); +} + +static inline uint16_t +dlb_event_enqueue(void *event_port, + const struct rte_event events[]) +{ + return __dlb_event_enqueue_burst(event_port, events, 1); +} + +static inline uint16_t +dlb_event_enqueue_delayed(void *event_port, + const struct rte_event events[]) +{ + return __dlb_event_enqueue_burst(event_port, events, 1); +} + +static uint16_t +dlb_event_enqueue_new_burst(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + return __dlb_event_enqueue_burst(event_port, events, num); +} + +static uint16_t +dlb_event_enqueue_new_burst_delayed(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + return __dlb_event_enqueue_burst(event_port, events, num); +} + +static uint16_t +dlb_event_enqueue_forward_burst(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + return __dlb_event_enqueue_burst(event_port, events, num); +} + +static uint16_t +dlb_event_enqueue_forward_burst_delayed(void *event_port, + const struct rte_event events[], + uint16_t num) +{ + return __dlb_event_enqueue_burst(event_port, events, num); +} + void dlb_entry_points_init(struct rte_eventdev *dev) { @@ -2159,6 +2836,11 @@ dlb_entry_points_init(struct rte_eventdev *dev) /* Expose PMD's eventdev interface */ dev->dev_ops = &dlb_eventdev_entry_ops; + + dev->enqueue = dlb_event_enqueue; + dev->enqueue_burst = dlb_event_enqueue_burst; + dev->enqueue_new_burst = dlb_event_enqueue_new_burst; + dev->enqueue_forward_burst = dlb_event_enqueue_forward_burst; } int -- 2.6.4
next prev parent reply other threads:[~2020-10-30 18:32 UTC|newest] Thread overview: 312+ messages / expand[flat|nested] mbox.gz Atom feed top 2020-06-12 21:24 [dpdk-dev] [PATCH 00/27] V1 event/dlb add Intel DLB PMD McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 01/27] eventdev: dlb upstream prerequisites McDaniel, Timothy 2020-06-13 3:59 ` Jerin Jacob 2020-06-13 10:43 ` Mattias Rönnblom 2020-06-18 15:51 ` McDaniel, Timothy 2020-06-18 15:44 ` McDaniel, Timothy 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 00/23] Add DLB PMD Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 14/23] event/dlb: add port link Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-29 14:57 ` [dpdk-dev] [PATCH v7 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-29 16:22 ` Thomas Monjalon 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 00/23] Add DLB PMD Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-30 9:40 ` [dpdk-dev] [PATCH v8 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 14/23] event/dlb: add port link Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-30 9:41 ` [dpdk-dev] [PATCH v8 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-30 12:41 ` [dpdk-dev] [PATCH v9 00/23] Add DLB PMD Timothy McDaniel 2020-10-30 12:41 ` [dpdk-dev] [PATCH v9 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-30 12:41 ` [dpdk-dev] [PATCH v9 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-30 12:41 ` [dpdk-dev] [PATCH v9 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 14/23] event/dlb: add port link Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-30 12:42 ` [dpdk-dev] [PATCH v9 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 00/23] Add DLB PMD Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-30 19:57 ` Eads, Gage 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-30 19:51 ` Eads, Gage 2020-10-31 18:21 ` McDaniel, Timothy 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-30 20:05 ` Eads, Gage 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-30 19:54 ` Eads, Gage 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-30 19:50 ` Eads, Gage 2020-10-31 18:22 ` McDaniel, Timothy 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 14/23] event/dlb: add port link Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-30 18:27 ` Timothy McDaniel [this message] 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 18/23] event/dlb: add dequeue and its burst variants Timothy McDaniel 2020-10-30 19:51 ` Eads, Gage 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-30 19:51 ` Eads, Gage 2020-10-30 18:27 ` [dpdk-dev] [PATCH v10 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 00/23] Add DLB PMD Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 14/23] event/dlb: add port link Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-30 23:41 ` [dpdk-dev] [PATCH v11 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 00/23] Add DLB PMD Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 14/23] event/dlb: add port link Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-31 1:19 ` [dpdk-dev] [PATCH v12 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-31 2:12 ` [dpdk-dev] [PATCH v13 00/23] Add DLB PMD Timothy McDaniel 2020-10-31 2:12 ` [dpdk-dev] [PATCH v13 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-31 2:12 ` [dpdk-dev] [PATCH v13 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-31 2:12 ` [dpdk-dev] [PATCH v13 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-31 2:12 ` [dpdk-dev] [PATCH v13 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-31 2:12 ` [dpdk-dev] [PATCH v13 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 14/23] event/dlb: add port link Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-31 2:13 ` [dpdk-dev] [PATCH v13 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-31 12:49 ` [dpdk-dev] [PATCH v13 00/23] Add DLB PMD Jerin Jacob 2020-10-31 18:20 ` McDaniel, Timothy 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 " Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-10-31 21:48 ` David Marchand 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 05/23] event/dlb: add inline functions Timothy McDaniel 2020-10-31 21:54 ` David Marchand 2020-11-01 16:04 ` McDaniel, Timothy 2020-11-01 16:21 ` McDaniel, Timothy 2020-11-01 18:01 ` David Marchand 2020-11-01 18:07 ` McDaniel, Timothy 2020-11-01 18:11 ` David Marchand 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 09/23] event/dlb: add xstats Timothy McDaniel 2020-10-31 21:59 ` David Marchand 2020-11-01 16:48 ` McDaniel, Timothy 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 12/23] event/dlb: add queue setup Timothy McDaniel 2020-10-31 22:02 ` David Marchand 2020-11-01 16:55 ` McDaniel, Timothy 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 13/23] event/dlb: add port setup Timothy McDaniel 2020-10-31 22:03 ` David Marchand 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 14/23] event/dlb: add port link Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-10-31 22:08 ` David Marchand 2020-11-01 17:04 ` McDaniel, Timothy 2020-10-31 18:17 ` [dpdk-dev] [PATCH v14 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-10-31 22:11 ` David Marchand 2020-10-31 18:18 ` [dpdk-dev] [PATCH v14 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-10-31 18:18 ` [dpdk-dev] [PATCH v14 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-10-31 22:15 ` [dpdk-dev] [PATCH v14 00/23] Add DLB PMD David Marchand 2020-10-31 22:25 ` McDaniel, Timothy 2020-11-01 9:16 ` David Marchand 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 " Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 05/23] event/dlb: add inline functions Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 09/23] event/dlb: add xstats Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 12/23] event/dlb: add queue setup Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 13/23] event/dlb: add port setup Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 14/23] event/dlb: add port link Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-11-01 19:26 ` [dpdk-dev] [PATCH v15 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-11-01 21:10 ` [dpdk-dev] [PATCH v15 00/23] Add DLB PMD David Marchand 2020-11-01 21:26 ` McDaniel, Timothy 2020-11-02 9:56 ` David Marchand 2020-11-10 12:51 ` David Marchand 2020-11-10 15:29 ` McDaniel, Timothy 2020-11-10 15:51 ` David Marchand 2020-11-10 15:59 ` McDaniel, Timothy 2020-11-11 20:29 ` McDaniel, Timothy 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 " Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 01/23] event/dlb: add documentation and meson infrastructure Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 02/23] event/dlb: add dynamic logging Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 03/23] event/dlb: add private data structures and constants Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 04/23] event/dlb: add definitions shared with LKM or shared code Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 05/23] event/dlb: add inline functions Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 06/23] event/dlb: add eventdev probe Timothy McDaniel 2020-11-01 23:29 ` [dpdk-dev] [PATCH v16 07/23] event/dlb: add flexible interface Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 08/23] event/dlb: add probe-time hardware init Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 09/23] event/dlb: add xstats Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 10/23] event/dlb: add infos get and configure Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 11/23] event/dlb: add queue and port default conf Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 12/23] event/dlb: add queue setup Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 13/23] event/dlb: add port setup Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 14/23] event/dlb: add port link Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 15/23] event/dlb: add port unlink and port unlinks in progress Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 16/23] event/dlb: add eventdev start Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 17/23] event/dlb: add enqueue and its burst variants Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 18/23] event/dlb: add dequeue " Timothy McDaniel 2020-11-02 10:15 ` Burakov, Anatoly 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 19/23] event/dlb: add eventdev stop and close Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 20/23] event/dlb: add PMD's token pop public interface Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 21/23] event/dlb: add PMD self-tests Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 22/23] event/dlb: add queue and port release Timothy McDaniel 2020-11-01 23:30 ` [dpdk-dev] [PATCH v16 23/23] event/dlb: add timeout ticks entry point Timothy McDaniel 2020-11-02 14:07 ` [dpdk-dev] [PATCH v16 00/23] Add DLB PMD Jerin Jacob 2020-06-12 21:24 ` [dpdk-dev] [PATCH 02/27] eventdev: do not pass disable_implicit_release bit to trace macro McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 03/27] event/dlb: add shared code version 10.7.9 McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 04/27] event/dlb: add make and meson build infrastructure McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 05/27] event/dlb: add DLB documentation McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 06/27] event/dlb: add dynamic logging McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 07/27] event/dlb: add private data structures and constants McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 08/27] event/dlb: add definitions shared with LKM or shared code McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 09/27] event/dlb: inline functions used in multiple files McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 10/27] event/dlb: add PFPMD-specific interface layer to shared code McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 11/27] event/dlb: add flexible PMD to device interfaces McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 12/27] event/dlb: add the PMD's public interfaces McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 13/27] event/dlb: add xstats support McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 14/27] event/dlb: add PMD self-tests McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 15/27] event/dlb: add probe McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 16/27] event/dlb: add infos_get and configure McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 17/27] event/dlb: add queue_def_conf and port_def_conf McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 18/27] event/dlb: add queue setup McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 19/27] event/dlb: add port_setup McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 20/27] event/dlb: add port_link McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 21/27] event/dlb: add queue_release and port_release McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 22/27] event/dlb: add port_unlink and port_unlinks_in_progress McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 23/27] event/dlb: add eventdev_start McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 24/27] event/dlb: add timout_ticks, dump, xstats, and selftest McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 25/27] event/dlb: add enqueue and its burst variants McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 26/27] event/dlb: add dequeue, dequeue_burst, and variants McDaniel, Timothy 2020-06-12 21:24 ` [dpdk-dev] [PATCH 27/27] event/dlb: add eventdev_stop and eventdev_close 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=1604082458-15368-18-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 \ --cc=thomas@monjalon.net \ /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
DPDK patches and discussions This inbox may be cloned and mirrored by anyone: git clone --mirror https://inbox.dpdk.org/dev/0 dev/git/0.git # If you have public-inbox 1.1+ installed, you may # initialize and index your mirror using the following commands: public-inbox-init -V2 dev dev/ https://inbox.dpdk.org/dev \ dev@dpdk.org public-inbox-index dev Example config snippet for mirrors. Newsgroup available over NNTP: nntp://inbox.dpdk.org/inbox.dpdk.dev AGPL code for this site: git clone https://public-inbox.org/public-inbox.git