- * [dpdk-dev] [PATCH v2 1/2] 10G PMD: New vectorized functions for RX/TX
  2014-05-12  9:56 [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Bruce Richardson
@ 2014-05-12  9:56 ` Bruce Richardson
  2014-05-12  9:56 ` [dpdk-dev] [PATCH v2 2/2] 10G PMD: enable vector PMD compile for 64b linux Bruce Richardson
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 9+ messages in thread
From: Bruce Richardson @ 2014-05-12  9:56 UTC (permalink / raw)
  To: dev
New file containing optimized receive and transmit functions which
use 128bit vector instructions to improve performance. When conditions
permit, these functions will be enabled at runtime by the device
initialization routines already in the PMD.
Signed-off-by: Bruce Richardson <bruce.richardson@intel.com>
---
 lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c |  726 +++++++++++++++++++++++++++++++++
 1 files changed, 726 insertions(+), 0 deletions(-)
 create mode 100644 lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c
diff --git a/lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c b/lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c
new file mode 100644
index 0000000..1cfdbe9
--- /dev/null
+++ b/lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c
@@ -0,0 +1,726 @@
+/*-
+ *   BSD LICENSE
+ * 
+ *   Copyright(c) 2010-2014 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ * 
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ * 
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ * 
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/queue.h>
+
+#include <endian.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <errno.h>
+#include <stdint.h>
+#include <stdarg.h>
+#include <unistd.h>
+#include <inttypes.h>
+
+#include <rte_byteorder.h>
+#include <rte_common.h>
+#include <rte_cycles.h>
+#include <rte_log.h>
+#include <rte_debug.h>
+#include <rte_interrupts.h>
+#include <rte_pci.h>
+#include <rte_memory.h>
+#include <rte_memzone.h>
+#include <rte_launch.h>
+#include <rte_tailq.h>
+#include <rte_eal.h>
+#include <rte_per_lcore.h>
+#include <rte_lcore.h>
+#include <rte_atomic.h>
+#include <rte_branch_prediction.h>
+#include <rte_ring.h>
+#include <rte_mempool.h>
+#include <rte_malloc.h>
+#include <rte_mbuf.h>
+#include <rte_ether.h>
+#include <rte_ethdev.h>
+#include <rte_prefetch.h>
+#include <rte_udp.h>
+#include <rte_tcp.h>
+#include <rte_sctp.h>
+#include <rte_string_fns.h>
+#include <rte_errno.h>
+
+#include "ixgbe_logs.h"
+#include "ixgbe/ixgbe_api.h"
+#include "ixgbe/ixgbe_vf.h"
+#include "ixgbe_ethdev.h"
+#include "ixgbe/ixgbe_dcb.h"
+
+#include "ixgbe_rxtx.h"
+
+#include <emmintrin.h>
+#include <xmmintrin.h>
+#include <nmmintrin.h>
+
+#ifndef __INTEL_COMPILER
+#pragma GCC diagnostic ignored "-Wcast-qual"
+#endif
+
+static struct rte_mbuf mb_def = {
+
+	.ol_flags = 0,
+	{
+		.pkt = {
+			.data_len = 0,
+			.pkt_len = 0,
+
+			.vlan_macip = {
+				.data = 0,
+			},
+			.hash = {
+				.rss = 0,
+			},
+
+			.nb_segs = 1,
+			.in_port = 0,
+
+			.next = NULL,
+			.data = NULL,
+		},
+	},
+};
+
+static inline void
+ixgbe_rxq_rearm (struct igb_rx_queue * rxq)
+{
+	int i;
+	uint16_t rx_id;
+	volatile union ixgbe_adv_rx_desc *rxdp;
+	struct igb_rx_entry *rxep = &rxq->sw_ring[rxq->rxrearm_start];
+	struct rte_mbuf *mb0, *mb1;
+	__m128i def_low;
+	__m128i hdr_room = _mm_set_epi64x(RTE_PKTMBUF_HEADROOM, RTE_PKTMBUF_HEADROOM);
+
+	/* Pull 'n' more MBUFs into the software ring */
+	if (rte_mempool_get_bulk(rxq->mb_pool, 
+				 (void *)rxep, RTE_IXGBE_RXQ_REARM_THRESH) < 0)
+		return;
+
+	rxdp = rxq->rx_ring + rxq->rxrearm_start;
+
+	def_low = _mm_load_si128((__m128i *)&(mb_def.pkt));
+
+	/* Initialize the mbufs in vector, process 2 mbufs in one loop */
+	for (i = 0; i < RTE_IXGBE_RXQ_REARM_THRESH; i += 2, rxep+=2)
+	{
+		__m128i dma_addr0, dma_addr1;
+		__m128i vaddr0, vaddr1;
+
+		mb0 = rxep[0].mbuf;
+		mb1 = rxep[1].mbuf;
+
+		/* load buf_addr(lo 64bit) and buf_physaddr(hi 64bit) */
+		vaddr0 = _mm_loadu_si128((__m128i *)&(mb0->buf_addr));
+		vaddr1 = _mm_loadu_si128((__m128i *)&(mb1->buf_addr));
+		
+		/* calc va/pa of pkt data point */
+		vaddr0 = _mm_add_epi64(vaddr0, hdr_room);
+		vaddr1 = _mm_add_epi64(vaddr1, hdr_room);
+
+		/* convert pa to dma_addr hdr/data */
+		dma_addr0 = _mm_unpackhi_epi64(vaddr0, vaddr0);
+		dma_addr1 = _mm_unpackhi_epi64(vaddr1, vaddr1);
+
+		/* fill va into t0 def pkt template */
+		vaddr0 = _mm_unpacklo_epi64(def_low, vaddr0);
+		vaddr1 = _mm_unpacklo_epi64(def_low, vaddr1);
+
+		/* flush desc with pa dma_addr */
+		_mm_store_si128((__m128i *)&rxdp++->read, dma_addr0);
+		_mm_store_si128((__m128i *)&rxdp++->read, dma_addr1);
+
+		/* flush mbuf with pkt template */
+		_mm_store_si128 ((__m128i *)&mb0->pkt, vaddr0);
+		_mm_store_si128 ((__m128i *)&mb1->pkt, vaddr1);
+
+		/* update refcnt per pkt */
+		rte_mbuf_refcnt_set(mb0, 1);
+		rte_mbuf_refcnt_set(mb1, 1);
+	}
+
+	rxq->rxrearm_start += RTE_IXGBE_RXQ_REARM_THRESH;
+	if (rxq->rxrearm_start >= rxq->nb_rx_desc)
+		rxq->rxrearm_start = 0;
+
+	rxq->rxrearm_nb -= RTE_IXGBE_RXQ_REARM_THRESH;
+
+	rx_id = (uint16_t) ((rxq->rxrearm_start == 0) ?
+			     (rxq->nb_rx_desc - 1) : (rxq->rxrearm_start - 1));
+
+	/* Update the tail pointer on the NIC */
+	IXGBE_PCI_REG_WRITE(rxq->rdt_reg_addr, rx_id);
+}
+
+#ifndef RTE_IXGBE_RX_OLFLAGS_DISABLE
+#define OLFLAGS_MASK     ((uint16_t)(PKT_RX_VLAN_PKT | PKT_RX_IPV4_HDR |\
+				     PKT_RX_IPV4_HDR_EXT | PKT_RX_IPV6_HDR |\
+				     PKT_RX_IPV6_HDR_EXT))
+#define OLFLAGS_MASK_V   (((uint64_t)OLFLAGS_MASK << 48) | ((uint64_t)OLFLAGS_MASK << 32) | \
+			  ((uint64_t)OLFLAGS_MASK << 16) | ((uint64_t)OLFLAGS_MASK))
+#define PTYPE_SHIFT    (1)
+#define VTAG_SHIFT     (3)
+
+static inline void
+desc_to_olflags_v(__m128i descs[4], struct rte_mbuf **rx_pkts)
+{
+	__m128i ptype0, ptype1, vtag0, vtag1;
+	union {
+		uint16_t e[4];
+		uint64_t dword;
+	}vol;
+	
+	ptype0 = _mm_unpacklo_epi16(descs[0], descs[1]);
+	ptype1 = _mm_unpacklo_epi16(descs[2], descs[3]);
+	vtag0 = _mm_unpackhi_epi16(descs[0], descs[1]);
+	vtag1 = _mm_unpackhi_epi16(descs[2], descs[3]);
+
+	ptype1 = _mm_unpacklo_epi32(ptype0, ptype1);
+	vtag1 = _mm_unpacklo_epi32(vtag0, vtag1);
+
+	ptype1 = _mm_slli_epi16(ptype1, PTYPE_SHIFT);
+	vtag1 = _mm_srli_epi16(vtag1, VTAG_SHIFT);
+
+	ptype1 = _mm_or_si128(ptype1, vtag1);
+	vol.dword = _mm_cvtsi128_si64(ptype1) & OLFLAGS_MASK_V;
+
+	rx_pkts[0]->ol_flags = vol.e[0];
+	rx_pkts[1]->ol_flags = vol.e[1];	
+	rx_pkts[2]->ol_flags = vol.e[2];
+	rx_pkts[3]->ol_flags = vol.e[3];
+}
+#else
+#define desc_to_olflags_v(desc, rx_pkts)            do{} while(0)
+#endif
+
+/*
+ * vPMD receive routine, now only accept (nb_pkts == RTE_IXGBE_VPMD_RX_BURST) in one loop
+ * Notice:
+ * - nb_pkts < RTE_IXGBE_VPMD_RX_BURST, just return no packet
+ * - nb_pkts > RTE_IXGBE_VPMD_RX_BURST, only scan RTE_IXGBE_VPMD_RX_BURST numbers of DD bit
+ * - don't support ol_flags for rss and csum err
+ */
+uint16_t
+ixgbe_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
+		uint16_t nb_pkts)
+{
+	volatile union ixgbe_adv_rx_desc * rxdp;
+	struct igb_rx_queue * rxq = rx_queue;
+	struct igb_rx_entry *sw_ring;	
+	uint16_t nb_pkts_recd;
+	int pos;
+	uint64_t var;
+	__m128i shuf_msk;
+	__m128i in_port;
+	__m128i dd_check; 
+
+	if (unlikely(nb_pkts < RTE_IXGBE_VPMD_RX_BURST))
+		return 0;
+
+	/* Just the act of getting into the function from the application is
+	 * going to cost about 7 cycles */
+	rxdp = rxq->rx_ring + rxq->rx_tail;
+
+	_mm_prefetch((const void *)rxdp, _MM_HINT_T0);
+
+	/* See if we need to rearm the RX queue - gives the prefetch a bit of time to act */
+	if (rxq->rxrearm_nb > RTE_IXGBE_RXQ_REARM_THRESH)
+		ixgbe_rxq_rearm(rxq);
+
+	/* Before we start moving massive data around, check to see if there is actually a
+	 * packet available */
+	if (! (rxdp->wb.upper.status_error & rte_cpu_to_le_32(IXGBE_RXDADV_STAT_DD)))
+		return 0;
+
+	/* 4 packets DD mask */
+	dd_check = _mm_set_epi64x(0x0000000100000001LL, 0x0000000100000001LL);
+
+	/* mask to shuffle from desc. to mbuf */
+	shuf_msk = _mm_set_epi8(
+		7,6,5,4,    /* octet 4~7, 32bits rss */
+		0xFF,0xFF,  /* skip high 16 bits vlan_macip, zero out */
+		15,14,      /* octet 14~15, low 16 bits vlan_macip */
+		0xFF,0xFF,  /* skip high 16 bits pkt_len, zero out */
+		13,12,      /* octet 12~13, low 16 bits pkt_len */
+		0xFF,0xFF,  /* skip nb_segs and in_port, zero out */
+		13,12       /* octet 12~13, 16 bits data_len */
+		);
+
+
+	/* Cache is empty -> need to scan the buffer rings, but first move the next
+	 * 'n' mbufs into the cache */
+	sw_ring = &rxq->sw_ring[rxq->rx_tail];
+	
+	/* in_port, nb_seg = 1, crc_len */
+	in_port = rxq->misc_info; 
+
+	/*
+	 * A. load 4 packet in one loop
+	 * B. copy 4 mbuf point from swring to rx_pkts
+	 * C. calc the number of DD bits among the 4 packets  
+	 * D. fill info. from desc to mbuf 
+	 */
+	for (pos = 0, nb_pkts_recd = 0; pos < RTE_IXGBE_VPMD_RX_BURST; 
+	     pos += RTE_IXGBE_DESCS_PER_LOOP, rxdp += RTE_IXGBE_DESCS_PER_LOOP)
+	{
+		__m128i descs[RTE_IXGBE_DESCS_PER_LOOP];
+		__m128i pkt_mb1, pkt_mb2, pkt_mb3, pkt_mb4;
+		__m128i zero, staterr, sterr_tmp1, sterr_tmp2;
+	        __m128i mbp1, mbp2; /* two mbuf pointer in one XMM reg. */
+
+		/* B.1 load 1 mbuf point */
+		mbp1 = _mm_loadu_si128((__m128i *)&sw_ring[pos]);
+
+		/* Read desc statuses backwards to avoid race condition */
+		/* A.1 load 4 pkts desc */
+		descs[3] = _mm_loadu_si128((__m128i *)(rxdp + 3));
+
+		/* B.2 copy 2 mbuf point into rx_pkts  */
+		_mm_store_si128((__m128i *)&rx_pkts[pos], mbp1);
+
+		/* B.1 load 1 mbuf point */
+		mbp2 = _mm_loadu_si128((__m128i *)&sw_ring[pos+2]);
+
+		descs[2] = _mm_loadu_si128((__m128i *)(rxdp + 2));
+		/* B.1 load 2 mbuf point */
+		descs[1] = _mm_loadu_si128((__m128i *)(rxdp + 1));
+		descs[0] = _mm_loadu_si128((__m128i *)(rxdp));	
+
+		/* B.2 copy 2 mbuf point into rx_pkts  */
+		_mm_store_si128((__m128i *)&rx_pkts[pos+2], mbp2);
+
+		/* avoid compiler reorder optimization */
+		rte_compiler_barrier();
+
+		/* D.1 pkt 3,4 convert format from desc to pktmbuf */
+		pkt_mb4 = _mm_shuffle_epi8(descs[3], shuf_msk);
+		pkt_mb3 = _mm_shuffle_epi8(descs[2], shuf_msk);
+
+		/* C.1 4=>2 filter staterr info only */
+		sterr_tmp2 = _mm_unpackhi_epi32(descs[3], descs[2]);
+		/* C.1 4=>2 filter staterr info only */
+		sterr_tmp1 = _mm_unpackhi_epi32(descs[1], descs[0]);
+
+		/* set ol_flags with packet type and vlan tag */
+		desc_to_olflags_v(descs, &rx_pkts[pos]);
+
+		/* D.2 pkt 3,4 set in_port/nb_seg and remove crc */
+		pkt_mb4 = _mm_add_epi16(pkt_mb4, in_port);
+		pkt_mb3 = _mm_add_epi16(pkt_mb3, in_port);
+
+		/* D.1 pkt 1,2 convert format from desc to pktmbuf */
+		pkt_mb2 = _mm_shuffle_epi8(descs[1], shuf_msk);
+		pkt_mb1 = _mm_shuffle_epi8(descs[0], shuf_msk);		
+
+		/* C.2 get 4 pkts staterr value  */
+		zero = _mm_xor_si128(dd_check, dd_check);
+		staterr = _mm_unpacklo_epi32(sterr_tmp1, sterr_tmp2);
+
+		/* D.3 copy final 3,4 data to rx_pkts */
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos+3]->pkt.data_len), pkt_mb4);
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos+2]->pkt.data_len), pkt_mb3);
+
+		/* D.2 pkt 1,2 set in_port/nb_seg and remove crc */
+		pkt_mb2 = _mm_add_epi16(pkt_mb2, in_port);
+		pkt_mb1 = _mm_add_epi16(pkt_mb1, in_port);
+
+		/* C.3 calc avaialbe number of desc */
+		staterr = _mm_and_si128(staterr, dd_check);
+		staterr = _mm_packs_epi32(staterr, zero);
+
+		/* D.3 copy final 1,2 data to rx_pkts */
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos+1]->pkt.data_len), pkt_mb2);
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos]->pkt.data_len), pkt_mb1);
+
+		/* C.4 calc avaialbe number of desc */
+		var = _mm_popcnt_u64(_mm_cvtsi128_si64(staterr));
+		nb_pkts_recd += var;
+		if (likely(var != RTE_IXGBE_DESCS_PER_LOOP))
+			break;
+	}
+
+	/* Update our internal tail pointer */
+	rxq->rx_tail = (uint16_t)(rxq->rx_tail + nb_pkts_recd);
+	rxq->rx_tail = (uint16_t)(rxq->rx_tail & (rxq->nb_rx_desc - 1));
+	rxq->rxrearm_nb = (uint16_t)(rxq->rxrearm_nb + nb_pkts_recd);
+
+	return nb_pkts_recd;
+}
+
+static inline void 
+vtx1(volatile union ixgbe_adv_tx_desc *txdp, 
+     struct rte_mbuf *pkt, __m128i flags)
+{
+	__m128i t0, t1, offset, ols, ba, ctl;
+ 
+	/* load buf_addr/buf_physaddr in t0 */
+	t0 = _mm_loadu_si128((__m128i *)&(pkt->buf_addr));
+	/* load data, ... pkt_len in t1 */
+	t1 = _mm_loadu_si128((__m128i *)&(pkt->pkt.data));
+		
+	/* calc offset = (data - buf_adr) */
+	offset = _mm_sub_epi64(t1, t0);
+
+	/* cmd_type_len: pkt_len |= DCMD_DTYP_FLAGS */
+	ctl = _mm_or_si128(t1, flags); 
+
+	/* reorder as buf_physaddr/buf_addr */
+	offset = _mm_shuffle_epi32(offset, 0x4E);
+
+	/* olinfo_stats: pkt_len << IXGBE_ADVTXD_PAYLEN_SHIFT */
+	ols = _mm_slli_epi32(t1, IXGBE_ADVTXD_PAYLEN_SHIFT);
+		
+	/* buffer_addr = buf_physaddr + offset */
+	ba = _mm_add_epi64(t0, offset);
+	
+	/* format cmd_type_len/olinfo_status */
+	ctl = _mm_unpackhi_epi32(ctl, ols);
+
+	/* format buf_physaddr/cmd_type_len */
+	ba = _mm_unpackhi_epi64(ba, ctl);
+
+	/* write desc */
+	_mm_store_si128((__m128i *)&txdp->read, ba);
+}
+
+static inline void 
+vtx(volatile union ixgbe_adv_tx_desc *txdp, 
+    struct rte_mbuf **pkt, uint16_t nb_pkts,  __m128i flags)
+{
+	int i;
+	for (i = 0; i < nb_pkts; ++i, ++txdp, ++pkt)
+		vtx1(txdp, *pkt, flags);
+}
+
+static inline int __attribute__((always_inline))
+ixgbe_tx_free_bufs(struct igb_tx_queue *txq)
+{
+	struct igb_tx_entry_v *txep;
+	struct igb_tx_entry_seq *txsp;
+	uint32_t status;
+	uint32_t n, k;
+#ifdef RTE_MBUF_SCATTER_GATHER
+	uint32_t i;
+	int nb_free = 0;
+	struct rte_mbuf *m, *free[RTE_IXGBE_TX_MAX_FREE_BUF_SZ];
+#endif
+
+	/* check DD bit on threshold descriptor */
+	status = txq->tx_ring[txq->tx_next_dd].wb.status;
+	if (! (status & IXGBE_ADVTXD_STAT_DD))
+		return 0;
+
+	n = txq->tx_rs_thresh;
+
+	/*
+	 * first buffer to free from S/W ring is at index
+	 * tx_next_dd - (tx_rs_thresh-1)
+	 */
+	txep = &((struct igb_tx_entry_v*)txq->sw_ring)[txq->tx_next_dd - (n - 1)];
+	txsp = &txq->sw_ring_seq[txq->tx_next_dd - (n - 1)];
+
+	while (n > 0) {
+		k = RTE_MIN(n, txsp[n-1].same_pool);
+#ifdef RTE_MBUF_SCATTER_GATHER
+		for (i = 0; i < k; i++) {
+			if((m = __rte_pktmbuf_prefree_seg((txep+n-k+i)->mbuf)) != NULL)
+			        free[nb_free++] = m;
+		}
+		rte_mempool_put_bulk((void*)txsp[n-1].pool, (void**)free, nb_free);
+#else
+		rte_mempool_put_bulk((void*)txsp[n-1].pool, (void**)(txep+n-k), k);
+#endif
+		n -= k;
+	}
+
+	/* buffers were freed, update counters */
+	txq->nb_tx_free = (uint16_t)(txq->nb_tx_free + txq->tx_rs_thresh);
+	txq->tx_next_dd = (uint16_t)(txq->tx_next_dd + txq->tx_rs_thresh);
+	if (txq->tx_next_dd >= txq->nb_tx_desc)
+		txq->tx_next_dd = (uint16_t)(txq->tx_rs_thresh - 1);
+
+	return txq->tx_rs_thresh;
+}
+
+static inline void __attribute__((always_inline))
+tx_backlog_entry(struct igb_tx_entry_v *txep, 
+		 struct igb_tx_entry_seq *txsp,
+		 struct rte_mbuf **tx_pkts, uint16_t nb_pkts)
+{
+	int i;
+	for (i = 0; i < (int)nb_pkts; ++i) {
+		txep[i].mbuf = tx_pkts[i];
+		/* check and update sequence number */
+		txsp[i].pool = tx_pkts[i]->pool;
+		if (txsp[i-1].pool == tx_pkts[i]->pool)
+			txsp[i].same_pool = txsp[i-1].same_pool + 1;
+		else
+			txsp[i].same_pool = 1;
+	}
+}
+
+uint16_t
+ixgbe_xmit_pkts_vec(void *tx_queue, struct rte_mbuf **tx_pkts,
+		       uint16_t nb_pkts)
+{
+	struct igb_tx_queue *txq = (struct igb_tx_queue *)tx_queue;
+	volatile union ixgbe_adv_tx_desc *txdp;
+	struct igb_tx_entry_v *txep;
+	struct igb_tx_entry_seq *txsp;
+	uint16_t n, nb_commit, tx_id;
+	__m128i flags = _mm_set_epi32(DCMD_DTYP_FLAGS,0,0,0);
+	__m128i rs = _mm_set_epi32(IXGBE_ADVTXD_DCMD_RS|DCMD_DTYP_FLAGS,0,0,0);
+	int i;
+
+	if (unlikely(nb_pkts > RTE_IXGBE_VPMD_TX_BURST))
+		nb_pkts = RTE_IXGBE_VPMD_TX_BURST;
+
+	if (txq->nb_tx_free < txq->tx_free_thresh)
+		ixgbe_tx_free_bufs(txq);
+
+	nb_commit = nb_pkts = (uint16_t)RTE_MIN(txq->nb_tx_free, nb_pkts);
+	if (unlikely(nb_pkts == 0))
+		return 0;
+
+	tx_id = txq->tx_tail;
+	txdp = &txq->tx_ring[tx_id];
+	txep = &((struct igb_tx_entry_v*)txq->sw_ring)[tx_id];
+	txsp = &txq->sw_ring_seq[tx_id];
+
+	txq->nb_tx_free = (uint16_t)(txq->nb_tx_free - nb_pkts);
+
+	if (nb_commit >= (n = (uint16_t)(txq->nb_tx_desc - tx_id))) {
+
+		tx_backlog_entry(txep, txsp, tx_pkts, n);
+
+		for (i = 0; i < n - 1; ++i, ++tx_pkts, ++txdp)
+			vtx1(txdp, *tx_pkts, flags);
+
+		vtx1(txdp, *tx_pkts++, rs);
+
+		nb_commit = (uint16_t)(nb_commit - n);
+
+		tx_id = 0;
+		txq->tx_next_rs = (uint16_t)(txq->tx_rs_thresh - 1);
+
+		/* avoid reach the end of ring */
+		txdp = &(txq->tx_ring[tx_id]);
+		txep = &(((struct igb_tx_entry_v*)txq->sw_ring)[tx_id]);
+		txsp = &(txq->sw_ring_seq[tx_id]);
+	}
+
+	tx_backlog_entry(txep, txsp, tx_pkts, nb_commit);
+
+	vtx(txdp, tx_pkts, nb_commit, flags);
+
+	tx_id = (uint16_t)(tx_id + nb_commit);
+	if (tx_id > txq->tx_next_rs) {
+		txq->tx_ring[txq->tx_next_rs].read.cmd_type_len |=
+			rte_cpu_to_le_32(IXGBE_ADVTXD_DCMD_RS);
+		txq->tx_next_rs = (uint16_t)(txq->tx_next_rs +
+			txq->tx_rs_thresh);
+	}
+	
+	txq->tx_tail = tx_id;
+
+	IXGBE_PCI_REG_WRITE(txq->tdt_reg_addr, txq->tx_tail);
+
+	return nb_pkts;
+}
+
+static void
+ixgbe_tx_queue_release_mbufs(struct igb_tx_queue *txq)
+{
+	unsigned i;
+	struct igb_tx_entry_v *txe;
+	struct igb_tx_entry_seq *txs;
+	uint16_t nb_free, max_desc;
+
+	if (txq->sw_ring != NULL) {
+		/* release the used mbufs in sw_ring */
+		nb_free = txq->nb_tx_free; 
+		max_desc = (uint16_t)(txq->nb_tx_desc - 1);
+		for (i = txq->tx_next_dd - (txq->tx_rs_thresh - 1); 
+		     nb_free < max_desc && i != txq->tx_tail;
+		     i = (i + 1) & max_desc) {
+			txe = (struct igb_tx_entry_v*)&txq->sw_ring[i];			
+			if (txe->mbuf != NULL)
+				rte_pktmbuf_free_seg(txe->mbuf);
+		}
+		/* reset tx_entry */
+		for (i = 0; i < txq->nb_tx_desc; i++) {
+			txe = (struct igb_tx_entry_v*)&txq->sw_ring[i];
+			txe->mbuf = NULL;
+
+			txs = &txq->sw_ring_seq[i];
+			txs->pool = NULL;
+			txs->same_pool = 0;
+		}
+	}
+}
+
+static void
+ixgbe_tx_free_swring(struct igb_tx_queue *txq)
+{
+	if (txq == NULL)
+		return;
+ 
+	if (txq->sw_ring != NULL) {
+		rte_free((struct igb_rx_entry*)txq->sw_ring - 1);
+		txq->sw_ring = NULL;
+	}
+
+	if (txq->sw_ring_seq != NULL) {
+		rte_free(txq->sw_ring_seq - 1);
+		txq->sw_ring_seq = NULL;
+	}
+}
+
+static void
+ixgbe_reset_tx_queue(struct igb_tx_queue *txq)
+{
+	static const union ixgbe_adv_tx_desc zeroed_desc = { .read = {
+			.buffer_addr = 0}};
+	struct igb_tx_entry_v *txe = (struct igb_tx_entry_v*)txq->sw_ring;
+	struct igb_tx_entry_seq *txs = txq->sw_ring_seq;
+	uint16_t i;
+
+	/* Zero out HW ring memory */
+	for (i = 0; i < txq->nb_tx_desc; i++) {
+		txq->tx_ring[i] = zeroed_desc;
+	}
+
+	/* Initialize SW ring entries */
+	for (i = 0; i < txq->nb_tx_desc; i++) {
+		volatile union ixgbe_adv_tx_desc *txd = &txq->tx_ring[i];
+		txd->wb.status = IXGBE_TXD_STAT_DD;
+		txe[i].mbuf = NULL;
+		txs[i].pool = NULL;
+		txs[i].same_pool = 0;
+	}
+
+	txq->tx_next_dd = (uint16_t)(txq->tx_rs_thresh - 1);
+	txq->tx_next_rs = (uint16_t)(txq->tx_rs_thresh - 1);
+
+	txq->tx_tail = 0;
+	txq->nb_tx_used = 0;
+	/*
+	 * Always allow 1 descriptor to be un-allocated to avoid
+	 * a H/W race condition
+	 */
+	txq->last_desc_cleaned = (uint16_t)(txq->nb_tx_desc - 1);
+	txq->nb_tx_free = (uint16_t)(txq->nb_tx_desc - 1);
+	txq->ctx_curr = 0;
+	memset((void*)&txq->ctx_cache, 0,
+		IXGBE_CTX_NUM * sizeof(struct ixgbe_advctx_info));
+}
+
+static struct ixgbe_txq_ops vec_txq_ops = {
+	.release_mbufs = ixgbe_tx_queue_release_mbufs,
+	.free_swring = ixgbe_tx_free_swring,
+	.reset = ixgbe_reset_tx_queue,
+};
+
+int ixgbe_txq_vec_setup(struct igb_tx_queue *txq, 
+			unsigned int socket_id)
+{
+	uint16_t nb_desc;
+
+	if (txq->sw_ring == NULL)
+		return -1;
+
+	/* request addtional one entry for continous sequence check */
+	nb_desc = (uint16_t)(txq->nb_tx_desc + 1);
+
+	txq->sw_ring_seq = rte_zmalloc_socket("txq->sw_ring_seq",
+				sizeof(struct igb_tx_entry_seq) * nb_desc,
+				CACHE_LINE_SIZE, socket_id);
+	if (txq->sw_ring_seq == NULL) 
+		return -1;
+
+
+	/* leave the first one for overflow */
+	txq->sw_ring = (struct igb_tx_entry*)
+		((struct igb_tx_entry_v*)txq->sw_ring + 1);
+	txq->sw_ring_seq += 1;
+	txq->ops = &vec_txq_ops;
+
+	return 0;
+}
+
+int ixgbe_rxq_vec_setup(struct igb_rx_queue *rxq,
+			__rte_unused unsigned int socket_id)
+{
+	rxq->misc_info = 
+		_mm_set_epi16(
+			0,0,0,0,0,
+			(uint16_t)-rxq->crc_len, /* sub crc on pkt_len */
+			(uint16_t)(rxq->port_id << 8 | 1),
+			/* 8b port_id and 8b nb_seg*/ 
+			(uint16_t)-rxq->crc_len  /* sub crc on data_len */
+			); 
+	
+	return 0;
+}
+
+int ixgbe_rx_vec_condition_check(struct rte_eth_dev *dev)
+{
+#ifndef RTE_LIBRTE_IEEE1588
+	struct rte_eth_rxmode *rxmode = &dev->data->dev_conf.rxmode;
+	struct rte_fdir_conf *fconf = &dev->data->dev_conf.fdir_conf;
+
+#ifdef RTE_IXGBE_RX_OLFLAGS_DISABLE
+	/* whithout rx ol_flags, no VP flag report */
+	if (rxmode->hw_vlan_strip != 0 ||
+	    rxmode->hw_vlan_extend != 0)
+		return -1;
+#endif
+
+	/* no fdir support */
+	if (fconf->mode != RTE_FDIR_MODE_NONE)
+		return -1;
+
+	/* 
+	 * - no csum error report support
+	 * - no header split support
+	 */
+	if (rxmode->hw_ip_checksum == 1 ||
+	    rxmode->header_split == 1)
+		return -1;
+	
+	return 0;
+#else
+	return -1;
+#endif
+}
-- 
1.7.0.7
^ permalink raw reply	[flat|nested] 9+ messages in thread
- * [dpdk-dev] [PATCH v2 2/2] 10G PMD: enable vector PMD compile for 64b linux
  2014-05-12  9:56 [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Bruce Richardson
  2014-05-12  9:56 ` [dpdk-dev] [PATCH v2 1/2] 10G PMD: New vectorized functions for RX/TX Bruce Richardson
@ 2014-05-12  9:56 ` Bruce Richardson
  2014-06-06  7:11 ` [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Cao, Waterman
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 9+ messages in thread
From: Bruce Richardson @ 2014-05-12  9:56 UTC (permalink / raw)
  To: dev
Add in and set the necessary compile-time options to enable the
vector PMD to be used in 64-bit linux.
Signed-off-by: Bruce Richardson <bruce.richardson@intel.com>
---
 config/defconfig_x86_64-default-linuxapp-gcc |    2 ++
 config/defconfig_x86_64-default-linuxapp-icc |    2 ++
 2 files changed, 4 insertions(+), 0 deletions(-)
diff --git a/config/defconfig_x86_64-default-linuxapp-gcc b/config/defconfig_x86_64-default-linuxapp-gcc
index f11ffbf..00c7c73 100644
--- a/config/defconfig_x86_64-default-linuxapp-gcc
+++ b/config/defconfig_x86_64-default-linuxapp-gcc
@@ -177,6 +177,8 @@ CONFIG_RTE_LIBRTE_IXGBE_DEBUG_DRIVER=n
 CONFIG_RTE_LIBRTE_IXGBE_PF_DISABLE_STRIP_CRC=n
 CONFIG_RTE_LIBRTE_IXGBE_RX_ALLOW_BULK_ALLOC=y
 CONFIG_RTE_LIBRTE_IXGBE_ALLOW_UNSUPPORTED_SFP=n
+CONFIG_RTE_IXGBE_INC_VECTOR=y
+CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE=n
 
 #
 # Compile burst-oriented VIRTIO PMD driver
diff --git a/config/defconfig_x86_64-default-linuxapp-icc b/config/defconfig_x86_64-default-linuxapp-icc
index 4eaca4c..e166b29 100644
--- a/config/defconfig_x86_64-default-linuxapp-icc
+++ b/config/defconfig_x86_64-default-linuxapp-icc
@@ -177,6 +177,8 @@ CONFIG_RTE_LIBRTE_IXGBE_DEBUG_DRIVER=n
 CONFIG_RTE_LIBRTE_IXGBE_PF_DISABLE_STRIP_CRC=n
 CONFIG_RTE_LIBRTE_IXGBE_RX_ALLOW_BULK_ALLOC=y
 CONFIG_RTE_LIBRTE_IXGBE_ALLOW_UNSUPPORTED_SFP=n
+CONFIG_RTE_IXGBE_INC_VECTOR=y
+CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE=n
 
 #
 # Compile burst-oriented VIRTIO PMD driver
-- 
1.7.0.7
^ permalink raw reply	[flat|nested] 9+ messages in thread
- * Re: [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions
  2014-05-12  9:56 [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Bruce Richardson
  2014-05-12  9:56 ` [dpdk-dev] [PATCH v2 1/2] 10G PMD: New vectorized functions for RX/TX Bruce Richardson
  2014-05-12  9:56 ` [dpdk-dev] [PATCH v2 2/2] 10G PMD: enable vector PMD compile for 64b linux Bruce Richardson
@ 2014-06-06  7:11 ` Cao, Waterman
  2014-06-11 22:19 ` Thomas Monjalon
  2014-06-13 22:52 ` [dpdk-dev] [PATCH v3] 10G PMD: New vectorized functions for RX/TX Bruce Richardson
  4 siblings, 0 replies; 9+ messages in thread
From: Cao, Waterman @ 2014-06-06  7:11 UTC (permalink / raw)
  To: Richardson, Bruce, dev
Tested-by: Waterman Cao <waterman.cao@intel.com>
This patch set has been tested by Intel.
We performed L2FWD, L3FWD and PMD test suite on new library, all cases passed.
Also we run performance test, result met our design expectation.
Please see test environment and configuration:
Each of the 10Gb Ethernet* ports of the DUT is directly connected in
full-duplex to a different port of the peer traffic generator.
The core configuration description is:
- 2C/1T: 2 Physical Cores, 1 Logical Core per physical core using core #2 and #4 (socket 0, 2nd and 3rd physical cores).
^ permalink raw reply	[flat|nested] 9+ messages in thread 
- * Re: [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions
  2014-05-12  9:56 [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Bruce Richardson
                   ` (2 preceding siblings ...)
  2014-06-06  7:11 ` [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Cao, Waterman
@ 2014-06-11 22:19 ` Thomas Monjalon
  2014-06-11 22:27   ` Richardson, Bruce
  2014-06-13 22:52 ` [dpdk-dev] [PATCH v3] 10G PMD: New vectorized functions for RX/TX Bruce Richardson
  4 siblings, 1 reply; 9+ messages in thread
From: Thomas Monjalon @ 2014-06-11 22:19 UTC (permalink / raw)
  To: Bruce Richardson; +Cc: dev
Hi Bruce,
You are introducing a new option CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE.
I feel a comment in the code would be helpful here.
Sorry for not having checked before but there are some checkpatch issues
in this PMD.
Other simple cleanup: I think some include directives are useless.
Could you rework a new version please?
Thanks
-- 
Thomas
^ permalink raw reply	[flat|nested] 9+ messages in thread
- * Re: [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions
  2014-06-11 22:19 ` Thomas Monjalon
@ 2014-06-11 22:27   ` Richardson, Bruce
  2014-06-11 22:33     ` Thomas Monjalon
  0 siblings, 1 reply; 9+ messages in thread
From: Richardson, Bruce @ 2014-06-11 22:27 UTC (permalink / raw)
  To: Thomas Monjalon; +Cc: dev
> -----Original Message-----
> From: Thomas Monjalon [mailto:thomas.monjalon@6wind.com]
> Sent: Wednesday, June 11, 2014 3:19 PM
> To: Richardson, Bruce
> Cc: dev@dpdk.org
> Subject: Re: [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX
> functions
> 
> Hi Bruce,
> 
> You are introducing a new option CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE.
> I feel a comment in the code would be helpful here.
Comment in the code, or in the config file where the option is defined?
> 
> Sorry for not having checked before but there are some checkpatch issues
> in this PMD.
> Other simple cleanup: I think some include directives are useless.
> 
> Could you rework a new version please?
> 
Ack. I'll see what I can do.
^ permalink raw reply	[flat|nested] 9+ messages in thread 
- * Re: [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions
  2014-06-11 22:27   ` Richardson, Bruce
@ 2014-06-11 22:33     ` Thomas Monjalon
  0 siblings, 0 replies; 9+ messages in thread
From: Thomas Monjalon @ 2014-06-11 22:33 UTC (permalink / raw)
  To: Richardson, Bruce; +Cc: dev
2014-06-11 22:27, Richardson, Bruce:
> > From: Thomas Monjalon [mailto:thomas.monjalon@6wind.com]
> > You are introducing a new option CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE.
> > I feel a comment in the code would be helpful here.
> 
> Comment in the code, or in the config file where the option is defined?
There are several config files so I think it's more appropriated in the code.
And it's not in DPDK habit to add comments in config files.
We are used to grep in code ;)
-- 
Thomas
^ permalink raw reply	[flat|nested] 9+ messages in thread 
 
 
- * [dpdk-dev] [PATCH v3] 10G PMD: New vectorized functions for RX/TX
  2014-05-12  9:56 [dpdk-dev] [PATCH v2 0/2] 10G PMD: vectorized RX and TX functions Bruce Richardson
                   ` (3 preceding siblings ...)
  2014-06-11 22:19 ` Thomas Monjalon
@ 2014-06-13 22:52 ` Bruce Richardson
  2014-06-16  8:25   ` Thomas Monjalon
  4 siblings, 1 reply; 9+ messages in thread
From: Bruce Richardson @ 2014-06-13 22:52 UTC (permalink / raw)
  To: dev
From: "Richardson, Bruce" <bruce.richardson@intel.com>
New file containing optimized receive and transmit functions which
use 128bit vector instructions to improve performance. When conditions
permit, these functions will be enabled at runtime by the device
initialization routines already in the PMD.
The compilation of the vectorized RX and TX code paths is controlled by
a new setting in the build time configuration for the IXGBE driver. Also
added is a setting which allows an optional further performance increase
by disabling the use of the olflags field on packet RX.
Changes:
* fixed issues highlighted by checkpatch
* added additional comment in commit description and code re. olflags
  compile-time config setting
Signed-off-by: Bruce Richardson <bruce.richardson@intel.com>
---
 config/common_bsdapp                      |   2 +
 config/common_linuxapp                    |   2 +
 config/defconfig_i686-native-linuxapp-gcc |   5 +
 config/defconfig_i686-native-linuxapp-icc |   5 +
 lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c     | 704 ++++++++++++++++++++++++++++++
 5 files changed, 718 insertions(+)
 create mode 100644 lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c
diff --git a/config/common_bsdapp b/config/common_bsdapp
index ec7b159..7802885 100644
--- a/config/common_bsdapp
+++ b/config/common_bsdapp
@@ -163,6 +163,8 @@ CONFIG_RTE_LIBRTE_IXGBE_DEBUG_DRIVER=n
 CONFIG_RTE_LIBRTE_IXGBE_PF_DISABLE_STRIP_CRC=n
 CONFIG_RTE_LIBRTE_IXGBE_RX_ALLOW_BULK_ALLOC=y
 CONFIG_RTE_LIBRTE_IXGBE_ALLOW_UNSUPPORTED_SFP=n
+CONFIG_RTE_IXGBE_INC_VECTOR=n
+CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE=n
 
 #
 # Compile burst-oriented VIRTIO PMD driver
diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7c143eb..663779d 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -182,6 +182,8 @@ CONFIG_RTE_LIBRTE_IXGBE_DEBUG_DRIVER=n
 CONFIG_RTE_LIBRTE_IXGBE_PF_DISABLE_STRIP_CRC=n
 CONFIG_RTE_LIBRTE_IXGBE_RX_ALLOW_BULK_ALLOC=y
 CONFIG_RTE_LIBRTE_IXGBE_ALLOW_UNSUPPORTED_SFP=n
+CONFIG_RTE_IXGBE_INC_VECTOR=y
+CONFIG_RTE_IXGBE_RX_OLFLAGS_DISABLE=n
 
 #
 # Compile burst-oriented VIRTIO PMD driver
diff --git a/config/defconfig_i686-native-linuxapp-gcc b/config/defconfig_i686-native-linuxapp-gcc
index 8cad745..61d8cc7 100644
--- a/config/defconfig_i686-native-linuxapp-gcc
+++ b/config/defconfig_i686-native-linuxapp-gcc
@@ -66,3 +66,8 @@ CONFIG_RTE_TOOLCHAIN_GCC=y
 # KNI is not supported on 32-bit
 #
 CONFIG_RTE_LIBRTE_KNI=n
+
+#
+# Vectorized PMD is not supported on 32-bit
+#
+CONFIG_RTE_IXGBE_INC_VECTOR=n
diff --git a/config/defconfig_i686-native-linuxapp-icc b/config/defconfig_i686-native-linuxapp-icc
index 7cb9beb..1afef18 100644
--- a/config/defconfig_i686-native-linuxapp-icc
+++ b/config/defconfig_i686-native-linuxapp-icc
@@ -66,3 +66,8 @@ CONFIG_RTE_TOOLCHAIN_ICC=y
 # KNI is not supported on 32-bit
 #
 CONFIG_RTE_LIBRTE_KNI=n
+
+#
+# Vectorized PMD is not supported on 32-bit
+#
+CONFIG_RTE_IXGBE_INC_VECTOR=n
diff --git a/lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c b/lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c
new file mode 100644
index 0000000..216fd49
--- /dev/null
+++ b/lib/librte_pmd_ixgbe/ixgbe_rxtx_vec.c
@@ -0,0 +1,704 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2014 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <rte_ethdev.h>
+#include <rte_malloc.h>
+
+#include "ixgbe_ethdev.h"
+#include "ixgbe_rxtx.h"
+
+#include <nmmintrin.h>
+
+#ifndef __INTEL_COMPILER
+#pragma GCC diagnostic ignored "-Wcast-qual"
+#endif
+
+static struct rte_mbuf mb_def = {
+
+	.ol_flags = 0,
+	{
+		.pkt = {
+			.data_len = 0,
+			.pkt_len = 0,
+
+			.vlan_macip = {
+				.data = 0,
+			},
+			.hash = {
+				.rss = 0,
+			},
+
+			.nb_segs = 1,
+			.in_port = 0,
+
+			.next = NULL,
+			.data = NULL,
+		},
+	},
+};
+
+static inline void
+ixgbe_rxq_rearm(struct igb_rx_queue *rxq)
+{
+	int i;
+	uint16_t rx_id;
+	volatile union ixgbe_adv_rx_desc *rxdp;
+	struct igb_rx_entry *rxep = &rxq->sw_ring[rxq->rxrearm_start];
+	struct rte_mbuf *mb0, *mb1;
+	__m128i def_low;
+	__m128i hdr_room = _mm_set_epi64x(RTE_PKTMBUF_HEADROOM,
+			RTE_PKTMBUF_HEADROOM);
+
+	/* Pull 'n' more MBUFs into the software ring */
+	if (rte_mempool_get_bulk(rxq->mb_pool,
+				 (void *)rxep, RTE_IXGBE_RXQ_REARM_THRESH) < 0)
+		return;
+
+	rxdp = rxq->rx_ring + rxq->rxrearm_start;
+
+	def_low = _mm_load_si128((__m128i *)&(mb_def.pkt));
+
+	/* Initialize the mbufs in vector, process 2 mbufs in one loop */
+	for (i = 0; i < RTE_IXGBE_RXQ_REARM_THRESH; i += 2, rxep += 2) {
+		__m128i dma_addr0, dma_addr1;
+		__m128i vaddr0, vaddr1;
+
+		mb0 = rxep[0].mbuf;
+		mb1 = rxep[1].mbuf;
+
+		/* load buf_addr(lo 64bit) and buf_physaddr(hi 64bit) */
+		vaddr0 = _mm_loadu_si128((__m128i *)&(mb0->buf_addr));
+		vaddr1 = _mm_loadu_si128((__m128i *)&(mb1->buf_addr));
+
+		/* calc va/pa of pkt data point */
+		vaddr0 = _mm_add_epi64(vaddr0, hdr_room);
+		vaddr1 = _mm_add_epi64(vaddr1, hdr_room);
+
+		/* convert pa to dma_addr hdr/data */
+		dma_addr0 = _mm_unpackhi_epi64(vaddr0, vaddr0);
+		dma_addr1 = _mm_unpackhi_epi64(vaddr1, vaddr1);
+
+		/* fill va into t0 def pkt template */
+		vaddr0 = _mm_unpacklo_epi64(def_low, vaddr0);
+		vaddr1 = _mm_unpacklo_epi64(def_low, vaddr1);
+
+		/* flush desc with pa dma_addr */
+		_mm_store_si128((__m128i *)&rxdp++->read, dma_addr0);
+		_mm_store_si128((__m128i *)&rxdp++->read, dma_addr1);
+
+		/* flush mbuf with pkt template */
+		_mm_store_si128((__m128i *)&mb0->pkt, vaddr0);
+		_mm_store_si128((__m128i *)&mb1->pkt, vaddr1);
+
+		/* update refcnt per pkt */
+		rte_mbuf_refcnt_set(mb0, 1);
+		rte_mbuf_refcnt_set(mb1, 1);
+	}
+
+	rxq->rxrearm_start += RTE_IXGBE_RXQ_REARM_THRESH;
+	if (rxq->rxrearm_start >= rxq->nb_rx_desc)
+		rxq->rxrearm_start = 0;
+
+	rxq->rxrearm_nb -= RTE_IXGBE_RXQ_REARM_THRESH;
+
+	rx_id = (uint16_t) ((rxq->rxrearm_start == 0) ?
+			     (rxq->nb_rx_desc - 1) : (rxq->rxrearm_start - 1));
+
+	/* Update the tail pointer on the NIC */
+	IXGBE_PCI_REG_WRITE(rxq->rdt_reg_addr, rx_id);
+}
+
+/* Handling the offload flags (olflags) field takes computation
+ * time when receiving packets. Therefore we provide a flag to disable
+ * the processing of the olflags field when they are not needed. This
+ * gives improved performance, at the cost of losing the offload info
+ * in the received packet
+ */
+#ifndef RTE_IXGBE_RX_OLFLAGS_DISABLE
+
+#define OLFLAGS_MASK     ((uint16_t)(PKT_RX_VLAN_PKT | PKT_RX_IPV4_HDR |\
+				     PKT_RX_IPV4_HDR_EXT | PKT_RX_IPV6_HDR |\
+				     PKT_RX_IPV6_HDR_EXT))
+#define OLFLAGS_MASK_V   (((uint64_t)OLFLAGS_MASK << 48) | \
+			  ((uint64_t)OLFLAGS_MASK << 32) | \
+			  ((uint64_t)OLFLAGS_MASK << 16) | \
+			  ((uint64_t)OLFLAGS_MASK))
+#define PTYPE_SHIFT    (1)
+#define VTAG_SHIFT     (3)
+
+static inline void
+desc_to_olflags_v(__m128i descs[4], struct rte_mbuf **rx_pkts)
+{
+	__m128i ptype0, ptype1, vtag0, vtag1;
+	union {
+		uint16_t e[4];
+		uint64_t dword;
+	} vol;
+
+	ptype0 = _mm_unpacklo_epi16(descs[0], descs[1]);
+	ptype1 = _mm_unpacklo_epi16(descs[2], descs[3]);
+	vtag0 = _mm_unpackhi_epi16(descs[0], descs[1]);
+	vtag1 = _mm_unpackhi_epi16(descs[2], descs[3]);
+
+	ptype1 = _mm_unpacklo_epi32(ptype0, ptype1);
+	vtag1 = _mm_unpacklo_epi32(vtag0, vtag1);
+
+	ptype1 = _mm_slli_epi16(ptype1, PTYPE_SHIFT);
+	vtag1 = _mm_srli_epi16(vtag1, VTAG_SHIFT);
+
+	ptype1 = _mm_or_si128(ptype1, vtag1);
+	vol.dword = _mm_cvtsi128_si64(ptype1) & OLFLAGS_MASK_V;
+
+	rx_pkts[0]->ol_flags = vol.e[0];
+	rx_pkts[1]->ol_flags = vol.e[1];
+	rx_pkts[2]->ol_flags = vol.e[2];
+	rx_pkts[3]->ol_flags = vol.e[3];
+}
+#else
+#define desc_to_olflags_v(desc, rx_pkts) do {} while (0)
+#endif
+
+/*
+ * vPMD receive routine, now only accept (nb_pkts == RTE_IXGBE_VPMD_RX_BURST)
+ * in one loop
+ *
+ * Notice:
+ * - nb_pkts < RTE_IXGBE_VPMD_RX_BURST, just return no packet
+ * - nb_pkts > RTE_IXGBE_VPMD_RX_BURST, only scan RTE_IXGBE_VPMD_RX_BURST
+ *   numbers of DD bit
+ * - don't support ol_flags for rss and csum err
+ */
+uint16_t
+ixgbe_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
+		uint16_t nb_pkts)
+{
+	volatile union ixgbe_adv_rx_desc *rxdp;
+	struct igb_rx_queue *rxq = rx_queue;
+	struct igb_rx_entry *sw_ring;
+	uint16_t nb_pkts_recd;
+	int pos;
+	uint64_t var;
+	__m128i shuf_msk;
+	__m128i in_port;
+	__m128i dd_check;
+
+	if (unlikely(nb_pkts < RTE_IXGBE_VPMD_RX_BURST))
+		return 0;
+
+	/* Just the act of getting into the function from the application is
+	 * going to cost about 7 cycles */
+	rxdp = rxq->rx_ring + rxq->rx_tail;
+
+	_mm_prefetch((const void *)rxdp, _MM_HINT_T0);
+
+	/* See if we need to rearm the RX queue - gives the prefetch a bit
+	 * of time to act */
+	if (rxq->rxrearm_nb > RTE_IXGBE_RXQ_REARM_THRESH)
+		ixgbe_rxq_rearm(rxq);
+
+	/* Before we start moving massive data around, check to see if
+	 * there is actually a packet available */
+	if (!(rxdp->wb.upper.status_error &
+				rte_cpu_to_le_32(IXGBE_RXDADV_STAT_DD)))
+		return 0;
+
+	/* 4 packets DD mask */
+	dd_check = _mm_set_epi64x(0x0000000100000001LL, 0x0000000100000001LL);
+
+	/* mask to shuffle from desc. to mbuf */
+	shuf_msk = _mm_set_epi8(
+		7, 6, 5, 4,  /* octet 4~7, 32bits rss */
+		0xFF, 0xFF,  /* skip high 16 bits vlan_macip, zero out */
+		15, 14,      /* octet 14~15, low 16 bits vlan_macip */
+		0xFF, 0xFF,  /* skip high 16 bits pkt_len, zero out */
+		13, 12,      /* octet 12~13, low 16 bits pkt_len */
+		0xFF, 0xFF,  /* skip nb_segs and in_port, zero out */
+		13, 12       /* octet 12~13, 16 bits data_len */
+		);
+
+
+	/* Cache is empty -> need to scan the buffer rings, but first move
+	 * the next 'n' mbufs into the cache */
+	sw_ring = &rxq->sw_ring[rxq->rx_tail];
+
+	/* in_port, nb_seg = 1, crc_len */
+	in_port = rxq->misc_info;
+
+	/*
+	 * A. load 4 packet in one loop
+	 * B. copy 4 mbuf point from swring to rx_pkts
+	 * C. calc the number of DD bits among the 4 packets
+	 * D. fill info. from desc to mbuf
+	 */
+	for (pos = 0, nb_pkts_recd = 0; pos < RTE_IXGBE_VPMD_RX_BURST;
+			pos += RTE_IXGBE_DESCS_PER_LOOP,
+			rxdp += RTE_IXGBE_DESCS_PER_LOOP) {
+		__m128i descs[RTE_IXGBE_DESCS_PER_LOOP];
+		__m128i pkt_mb1, pkt_mb2, pkt_mb3, pkt_mb4;
+		__m128i zero, staterr, sterr_tmp1, sterr_tmp2;
+		__m128i mbp1, mbp2; /* two mbuf pointer in one XMM reg. */
+
+		/* B.1 load 1 mbuf point */
+		mbp1 = _mm_loadu_si128((__m128i *)&sw_ring[pos]);
+
+		/* Read desc statuses backwards to avoid race condition */
+		/* A.1 load 4 pkts desc */
+		descs[3] = _mm_loadu_si128((__m128i *)(rxdp + 3));
+
+		/* B.2 copy 2 mbuf point into rx_pkts  */
+		_mm_store_si128((__m128i *)&rx_pkts[pos], mbp1);
+
+		/* B.1 load 1 mbuf point */
+		mbp2 = _mm_loadu_si128((__m128i *)&sw_ring[pos+2]);
+
+		descs[2] = _mm_loadu_si128((__m128i *)(rxdp + 2));
+		/* B.1 load 2 mbuf point */
+		descs[1] = _mm_loadu_si128((__m128i *)(rxdp + 1));
+		descs[0] = _mm_loadu_si128((__m128i *)(rxdp));
+
+		/* B.2 copy 2 mbuf point into rx_pkts  */
+		_mm_store_si128((__m128i *)&rx_pkts[pos+2], mbp2);
+
+		/* avoid compiler reorder optimization */
+		rte_compiler_barrier();
+
+		/* D.1 pkt 3,4 convert format from desc to pktmbuf */
+		pkt_mb4 = _mm_shuffle_epi8(descs[3], shuf_msk);
+		pkt_mb3 = _mm_shuffle_epi8(descs[2], shuf_msk);
+
+		/* C.1 4=>2 filter staterr info only */
+		sterr_tmp2 = _mm_unpackhi_epi32(descs[3], descs[2]);
+		/* C.1 4=>2 filter staterr info only */
+		sterr_tmp1 = _mm_unpackhi_epi32(descs[1], descs[0]);
+
+		/* set ol_flags with packet type and vlan tag */
+		desc_to_olflags_v(descs, &rx_pkts[pos]);
+
+		/* D.2 pkt 3,4 set in_port/nb_seg and remove crc */
+		pkt_mb4 = _mm_add_epi16(pkt_mb4, in_port);
+		pkt_mb3 = _mm_add_epi16(pkt_mb3, in_port);
+
+		/* D.1 pkt 1,2 convert format from desc to pktmbuf */
+		pkt_mb2 = _mm_shuffle_epi8(descs[1], shuf_msk);
+		pkt_mb1 = _mm_shuffle_epi8(descs[0], shuf_msk);
+
+		/* C.2 get 4 pkts staterr value  */
+		zero = _mm_xor_si128(dd_check, dd_check);
+		staterr = _mm_unpacklo_epi32(sterr_tmp1, sterr_tmp2);
+
+		/* D.3 copy final 3,4 data to rx_pkts */
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos+3]->pkt.data_len),
+				pkt_mb4);
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos+2]->pkt.data_len),
+				pkt_mb3);
+
+		/* D.2 pkt 1,2 set in_port/nb_seg and remove crc */
+		pkt_mb2 = _mm_add_epi16(pkt_mb2, in_port);
+		pkt_mb1 = _mm_add_epi16(pkt_mb1, in_port);
+
+		/* C.3 calc avaialbe number of desc */
+		staterr = _mm_and_si128(staterr, dd_check);
+		staterr = _mm_packs_epi32(staterr, zero);
+
+		/* D.3 copy final 1,2 data to rx_pkts */
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos+1]->pkt.data_len),
+				pkt_mb2);
+		_mm_storeu_si128((__m128i *)&(rx_pkts[pos]->pkt.data_len),
+				pkt_mb1);
+
+		/* C.4 calc avaialbe number of desc */
+		var = _mm_popcnt_u64(_mm_cvtsi128_si64(staterr));
+		nb_pkts_recd += var;
+		if (likely(var != RTE_IXGBE_DESCS_PER_LOOP))
+			break;
+	}
+
+	/* Update our internal tail pointer */
+	rxq->rx_tail = (uint16_t)(rxq->rx_tail + nb_pkts_recd);
+	rxq->rx_tail = (uint16_t)(rxq->rx_tail & (rxq->nb_rx_desc - 1));
+	rxq->rxrearm_nb = (uint16_t)(rxq->rxrearm_nb + nb_pkts_recd);
+
+	return nb_pkts_recd;
+}
+
+static inline void
+vtx1(volatile union ixgbe_adv_tx_desc *txdp,
+		struct rte_mbuf *pkt, __m128i flags)
+{
+	__m128i t0, t1, offset, ols, ba, ctl;
+
+	/* load buf_addr/buf_physaddr in t0 */
+	t0 = _mm_loadu_si128((__m128i *)&(pkt->buf_addr));
+	/* load data, ... pkt_len in t1 */
+	t1 = _mm_loadu_si128((__m128i *)&(pkt->pkt.data));
+
+	/* calc offset = (data - buf_adr) */
+	offset = _mm_sub_epi64(t1, t0);
+
+	/* cmd_type_len: pkt_len |= DCMD_DTYP_FLAGS */
+	ctl = _mm_or_si128(t1, flags);
+
+	/* reorder as buf_physaddr/buf_addr */
+	offset = _mm_shuffle_epi32(offset, 0x4E);
+
+	/* olinfo_stats: pkt_len << IXGBE_ADVTXD_PAYLEN_SHIFT */
+	ols = _mm_slli_epi32(t1, IXGBE_ADVTXD_PAYLEN_SHIFT);
+
+	/* buffer_addr = buf_physaddr + offset */
+	ba = _mm_add_epi64(t0, offset);
+
+	/* format cmd_type_len/olinfo_status */
+	ctl = _mm_unpackhi_epi32(ctl, ols);
+
+	/* format buf_physaddr/cmd_type_len */
+	ba = _mm_unpackhi_epi64(ba, ctl);
+
+	/* write desc */
+	_mm_store_si128((__m128i *)&txdp->read, ba);
+}
+
+static inline void
+vtx(volatile union ixgbe_adv_tx_desc *txdp,
+		struct rte_mbuf **pkt, uint16_t nb_pkts,  __m128i flags)
+{
+	int i;
+	for (i = 0; i < nb_pkts; ++i, ++txdp, ++pkt)
+		vtx1(txdp, *pkt, flags);
+}
+
+static inline int __attribute__((always_inline))
+ixgbe_tx_free_bufs(struct igb_tx_queue *txq)
+{
+	struct igb_tx_entry_v *txep;
+	struct igb_tx_entry_seq *txsp;
+	uint32_t status;
+	uint32_t n, k;
+#ifdef RTE_MBUF_SCATTER_GATHER
+	uint32_t i;
+	int nb_free = 0;
+	struct rte_mbuf *m, *free[RTE_IXGBE_TX_MAX_FREE_BUF_SZ];
+#endif
+
+	/* check DD bit on threshold descriptor */
+	status = txq->tx_ring[txq->tx_next_dd].wb.status;
+	if (!(status & IXGBE_ADVTXD_STAT_DD))
+		return 0;
+
+	n = txq->tx_rs_thresh;
+
+	/*
+	 * first buffer to free from S/W ring is at index
+	 * tx_next_dd - (tx_rs_thresh-1)
+	 */
+	txep = &((struct igb_tx_entry_v *)txq->sw_ring)[txq->tx_next_dd -
+			(n - 1)];
+	txsp = &txq->sw_ring_seq[txq->tx_next_dd - (n - 1)];
+
+	while (n > 0) {
+		k = RTE_MIN(n, txsp[n-1].same_pool);
+#ifdef RTE_MBUF_SCATTER_GATHER
+		for (i = 0; i < k; i++) {
+			if ((m = __rte_pktmbuf_prefree_seg((txep+n-k+i)->mbuf))
+					!= NULL)
+				free[nb_free++] = m;
+		}
+		rte_mempool_put_bulk((void *)txsp[n-1].pool,
+				(void **)free, nb_free);
+#else
+		rte_mempool_put_bulk((void *)txsp[n-1].pool,
+				(void **)(txep+n-k), k);
+#endif
+		n -= k;
+	}
+
+	/* buffers were freed, update counters */
+	txq->nb_tx_free = (uint16_t)(txq->nb_tx_free + txq->tx_rs_thresh);
+	txq->tx_next_dd = (uint16_t)(txq->tx_next_dd + txq->tx_rs_thresh);
+	if (txq->tx_next_dd >= txq->nb_tx_desc)
+		txq->tx_next_dd = (uint16_t)(txq->tx_rs_thresh - 1);
+
+	return txq->tx_rs_thresh;
+}
+
+static inline void __attribute__((always_inline))
+tx_backlog_entry(struct igb_tx_entry_v *txep,
+		 struct igb_tx_entry_seq *txsp,
+		 struct rte_mbuf **tx_pkts, uint16_t nb_pkts)
+{
+	int i;
+	for (i = 0; i < (int)nb_pkts; ++i) {
+		txep[i].mbuf = tx_pkts[i];
+		/* check and update sequence number */
+		txsp[i].pool = tx_pkts[i]->pool;
+		if (txsp[i-1].pool == tx_pkts[i]->pool)
+			txsp[i].same_pool = txsp[i-1].same_pool + 1;
+		else
+			txsp[i].same_pool = 1;
+	}
+}
+
+uint16_t
+ixgbe_xmit_pkts_vec(void *tx_queue, struct rte_mbuf **tx_pkts,
+		       uint16_t nb_pkts)
+{
+	struct igb_tx_queue *txq = (struct igb_tx_queue *)tx_queue;
+	volatile union ixgbe_adv_tx_desc *txdp;
+	struct igb_tx_entry_v *txep;
+	struct igb_tx_entry_seq *txsp;
+	uint16_t n, nb_commit, tx_id;
+	__m128i flags = _mm_set_epi32(DCMD_DTYP_FLAGS, 0, 0, 0);
+	__m128i rs = _mm_set_epi32(IXGBE_ADVTXD_DCMD_RS|DCMD_DTYP_FLAGS,
+			0, 0, 0);
+	int i;
+
+	if (unlikely(nb_pkts > RTE_IXGBE_VPMD_TX_BURST))
+		nb_pkts = RTE_IXGBE_VPMD_TX_BURST;
+
+	if (txq->nb_tx_free < txq->tx_free_thresh)
+		ixgbe_tx_free_bufs(txq);
+
+	nb_commit = nb_pkts = (uint16_t)RTE_MIN(txq->nb_tx_free, nb_pkts);
+	if (unlikely(nb_pkts == 0))
+		return 0;
+
+	tx_id = txq->tx_tail;
+	txdp = &txq->tx_ring[tx_id];
+	txep = &((struct igb_tx_entry_v *)txq->sw_ring)[tx_id];
+	txsp = &txq->sw_ring_seq[tx_id];
+
+	txq->nb_tx_free = (uint16_t)(txq->nb_tx_free - nb_pkts);
+
+	if (nb_commit >= (n = (uint16_t)(txq->nb_tx_desc - tx_id))) {
+
+		tx_backlog_entry(txep, txsp, tx_pkts, n);
+
+		for (i = 0; i < n - 1; ++i, ++tx_pkts, ++txdp)
+			vtx1(txdp, *tx_pkts, flags);
+
+		vtx1(txdp, *tx_pkts++, rs);
+
+		nb_commit = (uint16_t)(nb_commit - n);
+
+		tx_id = 0;
+		txq->tx_next_rs = (uint16_t)(txq->tx_rs_thresh - 1);
+
+		/* avoid reach the end of ring */
+		txdp = &(txq->tx_ring[tx_id]);
+		txep = &(((struct igb_tx_entry_v *)txq->sw_ring)[tx_id]);
+		txsp = &(txq->sw_ring_seq[tx_id]);
+	}
+
+	tx_backlog_entry(txep, txsp, tx_pkts, nb_commit);
+
+	vtx(txdp, tx_pkts, nb_commit, flags);
+
+	tx_id = (uint16_t)(tx_id + nb_commit);
+	if (tx_id > txq->tx_next_rs) {
+		txq->tx_ring[txq->tx_next_rs].read.cmd_type_len |=
+			rte_cpu_to_le_32(IXGBE_ADVTXD_DCMD_RS);
+		txq->tx_next_rs = (uint16_t)(txq->tx_next_rs +
+			txq->tx_rs_thresh);
+	}
+
+	txq->tx_tail = tx_id;
+
+	IXGBE_PCI_REG_WRITE(txq->tdt_reg_addr, txq->tx_tail);
+
+	return nb_pkts;
+}
+
+static void
+ixgbe_tx_queue_release_mbufs(struct igb_tx_queue *txq)
+{
+	unsigned i;
+	struct igb_tx_entry_v *txe;
+	struct igb_tx_entry_seq *txs;
+	uint16_t nb_free, max_desc;
+
+	if (txq->sw_ring != NULL) {
+		/* release the used mbufs in sw_ring */
+		nb_free = txq->nb_tx_free;
+		max_desc = (uint16_t)(txq->nb_tx_desc - 1);
+		for (i = txq->tx_next_dd - (txq->tx_rs_thresh - 1);
+		     nb_free < max_desc && i != txq->tx_tail;
+		     i = (i + 1) & max_desc) {
+			txe = (struct igb_tx_entry_v *)&txq->sw_ring[i];
+			if (txe->mbuf != NULL)
+				rte_pktmbuf_free_seg(txe->mbuf);
+		}
+		/* reset tx_entry */
+		for (i = 0; i < txq->nb_tx_desc; i++) {
+			txe = (struct igb_tx_entry_v *)&txq->sw_ring[i];
+			txe->mbuf = NULL;
+
+			txs = &txq->sw_ring_seq[i];
+			txs->pool = NULL;
+			txs->same_pool = 0;
+		}
+	}
+}
+
+static void
+ixgbe_tx_free_swring(struct igb_tx_queue *txq)
+{
+	if (txq == NULL)
+		return;
+
+	if (txq->sw_ring != NULL) {
+		rte_free((struct igb_rx_entry *)txq->sw_ring - 1);
+		txq->sw_ring = NULL;
+	}
+
+	if (txq->sw_ring_seq != NULL) {
+		rte_free(txq->sw_ring_seq - 1);
+		txq->sw_ring_seq = NULL;
+	}
+}
+
+static void
+ixgbe_reset_tx_queue(struct igb_tx_queue *txq)
+{
+	static const union ixgbe_adv_tx_desc zeroed_desc = { .read = {
+			.buffer_addr = 0} };
+	struct igb_tx_entry_v *txe = (struct igb_tx_entry_v *)txq->sw_ring;
+	struct igb_tx_entry_seq *txs = txq->sw_ring_seq;
+	uint16_t i;
+
+	/* Zero out HW ring memory */
+	for (i = 0; i < txq->nb_tx_desc; i++)
+		txq->tx_ring[i] = zeroed_desc;
+
+	/* Initialize SW ring entries */
+	for (i = 0; i < txq->nb_tx_desc; i++) {
+		volatile union ixgbe_adv_tx_desc *txd = &txq->tx_ring[i];
+		txd->wb.status = IXGBE_TXD_STAT_DD;
+		txe[i].mbuf = NULL;
+		txs[i].pool = NULL;
+		txs[i].same_pool = 0;
+	}
+
+	txq->tx_next_dd = (uint16_t)(txq->tx_rs_thresh - 1);
+	txq->tx_next_rs = (uint16_t)(txq->tx_rs_thresh - 1);
+
+	txq->tx_tail = 0;
+	txq->nb_tx_used = 0;
+	/*
+	 * Always allow 1 descriptor to be un-allocated to avoid
+	 * a H/W race condition
+	 */
+	txq->last_desc_cleaned = (uint16_t)(txq->nb_tx_desc - 1);
+	txq->nb_tx_free = (uint16_t)(txq->nb_tx_desc - 1);
+	txq->ctx_curr = 0;
+	memset((void *)&txq->ctx_cache, 0,
+		IXGBE_CTX_NUM * sizeof(struct ixgbe_advctx_info));
+}
+
+static struct ixgbe_txq_ops vec_txq_ops = {
+	.release_mbufs = ixgbe_tx_queue_release_mbufs,
+	.free_swring = ixgbe_tx_free_swring,
+	.reset = ixgbe_reset_tx_queue,
+};
+
+int ixgbe_txq_vec_setup(struct igb_tx_queue *txq,
+			unsigned int socket_id)
+{
+	uint16_t nb_desc;
+
+	if (txq->sw_ring == NULL)
+		return -1;
+
+	/* request addtional one entry for continous sequence check */
+	nb_desc = (uint16_t)(txq->nb_tx_desc + 1);
+
+	txq->sw_ring_seq = rte_zmalloc_socket("txq->sw_ring_seq",
+				sizeof(struct igb_tx_entry_seq) * nb_desc,
+				CACHE_LINE_SIZE, socket_id);
+	if (txq->sw_ring_seq == NULL)
+		return -1;
+
+
+	/* leave the first one for overflow */
+	txq->sw_ring = (struct igb_tx_entry *)
+		((struct igb_tx_entry_v *)txq->sw_ring + 1);
+	txq->sw_ring_seq += 1;
+	txq->ops = &vec_txq_ops;
+
+	return 0;
+}
+
+int ixgbe_rxq_vec_setup(struct igb_rx_queue *rxq,
+			__rte_unused unsigned int socket_id)
+{
+	rxq->misc_info =
+		_mm_set_epi16(
+			0, 0, 0, 0, 0,
+			(uint16_t)-rxq->crc_len, /* sub crc on pkt_len */
+			(uint16_t)(rxq->port_id << 8 | 1),
+			/* 8b port_id and 8b nb_seg*/
+			(uint16_t)-rxq->crc_len  /* sub crc on data_len */
+			);
+
+	return 0;
+}
+
+int ixgbe_rx_vec_condition_check(struct rte_eth_dev *dev)
+{
+#ifndef RTE_LIBRTE_IEEE1588
+	struct rte_eth_rxmode *rxmode = &dev->data->dev_conf.rxmode;
+	struct rte_fdir_conf *fconf = &dev->data->dev_conf.fdir_conf;
+
+#ifdef RTE_IXGBE_RX_OLFLAGS_DISABLE
+	/* whithout rx ol_flags, no VP flag report */
+	if (rxmode->hw_vlan_strip != 0 ||
+	    rxmode->hw_vlan_extend != 0)
+		return -1;
+#endif
+
+	/* no fdir support */
+	if (fconf->mode != RTE_FDIR_MODE_NONE)
+		return -1;
+
+	/*
+	 * - no csum error report support
+	 * - no header split support
+	 */
+	if (rxmode->hw_ip_checksum == 1 ||
+	    rxmode->header_split == 1)
+		return -1;
+
+	return 0;
+#else
+	return -1;
+#endif
+}
-- 
1.9.3
^ permalink raw reply	[flat|nested] 9+ messages in thread
- * Re: [dpdk-dev] [PATCH v3] 10G PMD: New vectorized functions for RX/TX
  2014-06-13 22:52 ` [dpdk-dev] [PATCH v3] 10G PMD: New vectorized functions for RX/TX Bruce Richardson
@ 2014-06-16  8:25   ` Thomas Monjalon
  0 siblings, 0 replies; 9+ messages in thread
From: Thomas Monjalon @ 2014-06-16  8:25 UTC (permalink / raw)
  To: Bruce Richardson; +Cc: dev
2014-06-13 23:52, Bruce Richardson:
> New file containing optimized receive and transmit functions which
> use 128bit vector instructions to improve performance. When conditions
> permit, these functions will be enabled at runtime by the device
> initialization routines already in the PMD.
> 
> The compilation of the vectorized RX and TX code paths is controlled by
> a new setting in the build time configuration for the IXGBE driver. Also
> added is a setting which allows an optional further performance increase
> by disabling the use of the olflags field on packet RX.
Acked-by: Thomas Monjalon <thomas.monjalon@6wind.com>
I've adjusted code for 2 "ASSIGN_IN_IF" checkpatch errors
and applied for version 1.7.0.
Thanks
-- 
Thomas
^ permalink raw reply	[flat|nested] 9+ messages in thread