summaryrefslogtreecommitdiffstats
path: root/src/dpdk/drivers/net/virtio/virtio_rxtx.c
diff options
context:
space:
mode:
Diffstat (limited to 'src/dpdk/drivers/net/virtio/virtio_rxtx.c')
-rw-r--r--src/dpdk/drivers/net/virtio/virtio_rxtx.c588
1 files changed, 350 insertions, 238 deletions
diff --git a/src/dpdk/drivers/net/virtio/virtio_rxtx.c b/src/dpdk/drivers/net/virtio/virtio_rxtx.c
index 724517e2..cab6e8fc 100644
--- a/src/dpdk/drivers/net/virtio/virtio_rxtx.c
+++ b/src/dpdk/drivers/net/virtio/virtio_rxtx.c
@@ -50,6 +50,11 @@
#include <rte_string_fns.h>
#include <rte_errno.h>
#include <rte_byteorder.h>
+#include <rte_cpuflags.h>
+#include <rte_net.h>
+#include <rte_ip.h>
+#include <rte_udp.h>
+#include <rte_tcp.h>
#include "virtio_logs.h"
#include "virtio_ethdev.h"
@@ -67,9 +72,14 @@
#define VIRTIO_SIMPLE_FLAGS ((uint32_t)ETH_TXQ_FLAGS_NOMULTSEGS | \
ETH_TXQ_FLAGS_NOOFFLOADS)
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
-static int use_simple_rxtx;
-#endif
+int
+virtio_dev_rx_queue_done(void *rxq, uint16_t offset)
+{
+ struct virtnet_rx *rxvq = rxq;
+ struct virtqueue *vq = rxvq->vq;
+
+ return VIRTQUEUE_NUSED(vq) >= offset;
+}
static void
vq_ring_free_chain(struct virtqueue *vq, uint16_t desc_idx)
@@ -123,7 +133,7 @@ virtqueue_dequeue_burst_rx(struct virtqueue *vq, struct rte_mbuf **rx_pkts,
cookie = (struct rte_mbuf *)vq->vq_descx[desc_idx].cookie;
if (unlikely(cookie == NULL)) {
- PMD_DRV_LOG(ERR, "vring descriptor with no mbuf cookie at %u\n",
+ PMD_DRV_LOG(ERR, "vring descriptor with no mbuf cookie at %u",
vq->vq_used_cons_idx);
break;
}
@@ -208,18 +218,76 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
return 0;
}
+/* When doing TSO, the IP length is not included in the pseudo header
+ * checksum of the packet given to the PMD, but for virtio it is
+ * expected.
+ */
+static void
+virtio_tso_fix_cksum(struct rte_mbuf *m)
+{
+ /* common case: header is not fragmented */
+ if (likely(rte_pktmbuf_data_len(m) >= m->l2_len + m->l3_len +
+ m->l4_len)) {
+ struct ipv4_hdr *iph;
+ struct ipv6_hdr *ip6h;
+ struct tcp_hdr *th;
+ uint16_t prev_cksum, new_cksum, ip_len, ip_paylen;
+ uint32_t tmp;
+
+ iph = rte_pktmbuf_mtod_offset(m, struct ipv4_hdr *, m->l2_len);
+ th = RTE_PTR_ADD(iph, m->l3_len);
+ if ((iph->version_ihl >> 4) == 4) {
+ iph->hdr_checksum = 0;
+ iph->hdr_checksum = rte_ipv4_cksum(iph);
+ ip_len = iph->total_length;
+ ip_paylen = rte_cpu_to_be_16(rte_be_to_cpu_16(ip_len) -
+ m->l3_len);
+ } else {
+ ip6h = (struct ipv6_hdr *)iph;
+ ip_paylen = ip6h->payload_len;
+ }
+
+ /* calculate the new phdr checksum not including ip_paylen */
+ prev_cksum = th->cksum;
+ tmp = prev_cksum;
+ tmp += ip_paylen;
+ tmp = (tmp & 0xffff) + (tmp >> 16);
+ new_cksum = tmp;
+
+ /* replace it in the packet */
+ th->cksum = new_cksum;
+ }
+}
+
+static inline int
+tx_offload_enabled(struct virtio_hw *hw)
+{
+ return vtpci_with_feature(hw, VIRTIO_NET_F_CSUM) ||
+ vtpci_with_feature(hw, VIRTIO_NET_F_HOST_TSO4) ||
+ vtpci_with_feature(hw, VIRTIO_NET_F_HOST_TSO6);
+}
+
+/* avoid write operation when necessary, to lessen cache issues */
+#define ASSIGN_UNLESS_EQUAL(var, val) do { \
+ if ((var) != (val)) \
+ (var) = (val); \
+} while (0)
+
static inline void
virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
uint16_t needed, int use_indirect, int can_push)
{
+ struct virtio_tx_region *txr = txvq->virtio_net_hdr_mz->addr;
struct vq_desc_extra *dxp;
struct virtqueue *vq = txvq->vq;
struct vring_desc *start_dp;
uint16_t seg_num = cookie->nb_segs;
uint16_t head_idx, idx;
uint16_t head_size = vq->hw->vtnet_hdr_size;
- unsigned long offs;
+ struct virtio_net_hdr *hdr;
+ int offload;
+ offload = tx_offload_enabled(vq->hw);
head_idx = vq->vq_desc_head_idx;
idx = head_idx;
dxp = &vq->vq_descx[idx];
@@ -229,10 +297,18 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
start_dp = vq->vq_ring.desc;
if (can_push) {
- /* put on zero'd transmit header (no offloads) */
- void *hdr = rte_pktmbuf_prepend(cookie, head_size);
-
- memset(hdr, 0, head_size);
+ /* prepend cannot fail, checked by caller */
+ hdr = (struct virtio_net_hdr *)
+ rte_pktmbuf_prepend(cookie, head_size);
+ /* if offload disabled, it is not zeroed below, do it now */
+ if (offload == 0) {
+ ASSIGN_UNLESS_EQUAL(hdr->csum_start, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->csum_offset, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->flags, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->gso_type, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->gso_size, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->hdr_len, 0);
+ }
} else if (use_indirect) {
/* setup tx ring slot to point to indirect
* descriptor list stored in reserved region.
@@ -240,14 +316,11 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
* the first slot in indirect ring is already preset
* to point to the header in reserved region
*/
- struct virtio_tx_region *txr = txvq->virtio_net_hdr_mz->addr;
-
- offs = idx * sizeof(struct virtio_tx_region)
- + offsetof(struct virtio_tx_region, tx_indir);
-
- start_dp[idx].addr = txvq->virtio_net_hdr_mem + offs;
+ start_dp[idx].addr = txvq->virtio_net_hdr_mem +
+ RTE_PTR_DIFF(&txr[idx].tx_indir, txr);
start_dp[idx].len = (seg_num + 1) * sizeof(struct vring_desc);
start_dp[idx].flags = VRING_DESC_F_INDIRECT;
+ hdr = (struct virtio_net_hdr *)&txr[idx].tx_hdr;
/* loop below will fill in rest of the indirect elements */
start_dp = txr[idx].tx_indir;
@@ -256,15 +329,59 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
/* setup first tx ring slot to point to header
* stored in reserved region.
*/
- offs = idx * sizeof(struct virtio_tx_region)
- + offsetof(struct virtio_tx_region, tx_hdr);
-
- start_dp[idx].addr = txvq->virtio_net_hdr_mem + offs;
+ start_dp[idx].addr = txvq->virtio_net_hdr_mem +
+ RTE_PTR_DIFF(&txr[idx].tx_hdr, txr);
start_dp[idx].len = vq->hw->vtnet_hdr_size;
start_dp[idx].flags = VRING_DESC_F_NEXT;
+ hdr = (struct virtio_net_hdr *)&txr[idx].tx_hdr;
+
idx = start_dp[idx].next;
}
+ /* Checksum Offload / TSO */
+ if (offload) {
+ if (cookie->ol_flags & PKT_TX_TCP_SEG)
+ cookie->ol_flags |= PKT_TX_TCP_CKSUM;
+
+ switch (cookie->ol_flags & PKT_TX_L4_MASK) {
+ case PKT_TX_UDP_CKSUM:
+ hdr->csum_start = cookie->l2_len + cookie->l3_len;
+ hdr->csum_offset = offsetof(struct udp_hdr,
+ dgram_cksum);
+ hdr->flags = VIRTIO_NET_HDR_F_NEEDS_CSUM;
+ break;
+
+ case PKT_TX_TCP_CKSUM:
+ hdr->csum_start = cookie->l2_len + cookie->l3_len;
+ hdr->csum_offset = offsetof(struct tcp_hdr, cksum);
+ hdr->flags = VIRTIO_NET_HDR_F_NEEDS_CSUM;
+ break;
+
+ default:
+ ASSIGN_UNLESS_EQUAL(hdr->csum_start, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->csum_offset, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->flags, 0);
+ break;
+ }
+
+ /* TCP Segmentation Offload */
+ if (cookie->ol_flags & PKT_TX_TCP_SEG) {
+ virtio_tso_fix_cksum(cookie);
+ hdr->gso_type = (cookie->ol_flags & PKT_TX_IPV6) ?
+ VIRTIO_NET_HDR_GSO_TCPV6 :
+ VIRTIO_NET_HDR_GSO_TCPV4;
+ hdr->gso_size = cookie->tso_segsz;
+ hdr->hdr_len =
+ cookie->l2_len +
+ cookie->l3_len +
+ cookie->l4_len;
+ } else {
+ ASSIGN_UNLESS_EQUAL(hdr->gso_type, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->gso_size, 0);
+ ASSIGN_UNLESS_EQUAL(hdr->hdr_len, 0);
+ }
+ }
+
do {
start_dp[idx].addr = VIRTIO_MBUF_DATA_DMA_ADDR(cookie, vq);
start_dp[idx].len = cookie->data_len;
@@ -282,207 +399,120 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
vq_update_avail_ring(vq, head_idx);
}
-static void
-virtio_dev_vring_start(struct virtqueue *vq)
-{
- int size = vq->vq_nentries;
- struct vring *vr = &vq->vq_ring;
- uint8_t *ring_mem = vq->vq_ring_virt_mem;
-
- PMD_INIT_FUNC_TRACE();
-
- /*
- * Reinitialise since virtio port might have been stopped and restarted
- */
- memset(vq->vq_ring_virt_mem, 0, vq->vq_ring_size);
- vring_init(vr, size, ring_mem, VIRTIO_PCI_VRING_ALIGN);
- vq->vq_used_cons_idx = 0;
- vq->vq_desc_head_idx = 0;
- vq->vq_avail_idx = 0;
- vq->vq_desc_tail_idx = (uint16_t)(vq->vq_nentries - 1);
- vq->vq_free_cnt = vq->vq_nentries;
- memset(vq->vq_descx, 0, sizeof(struct vq_desc_extra) * vq->vq_nentries);
-
- vring_desc_init(vr->desc, size);
-
- /*
- * Disable device(host) interrupting guest
- */
- virtqueue_disable_intr(vq);
-}
-
void
virtio_dev_cq_start(struct rte_eth_dev *dev)
{
struct virtio_hw *hw = dev->data->dev_private;
if (hw->cvq && hw->cvq->vq) {
- virtio_dev_vring_start(hw->cvq->vq);
VIRTQUEUE_DUMP((struct virtqueue *)hw->cvq->vq);
}
}
-void
-virtio_dev_rxtx_start(struct rte_eth_dev *dev)
+int
+virtio_dev_rx_queue_setup(struct rte_eth_dev *dev,
+ uint16_t queue_idx,
+ uint16_t nb_desc,
+ unsigned int socket_id __rte_unused,
+ __rte_unused const struct rte_eth_rxconf *rx_conf,
+ struct rte_mempool *mp)
{
- /*
- * Start receive and transmit vrings
- * - Setup vring structure for all queues
- * - Initialize descriptor for the rx vring
- * - Allocate blank mbufs for the each rx descriptor
- *
- */
- uint16_t i;
+ uint16_t vtpci_queue_idx = 2 * queue_idx + VTNET_SQ_RQ_QUEUE_IDX;
+ struct virtio_hw *hw = dev->data->dev_private;
+ struct virtqueue *vq = hw->vqs[vtpci_queue_idx];
+ struct virtnet_rx *rxvq;
+ int error, nbufs;
+ struct rte_mbuf *m;
uint16_t desc_idx;
PMD_INIT_FUNC_TRACE();
- /* Start rx vring. */
- for (i = 0; i < dev->data->nb_rx_queues; i++) {
- struct virtnet_rx *rxvq = dev->data->rx_queues[i];
- struct virtqueue *vq = rxvq->vq;
- int error, nbufs;
- struct rte_mbuf *m;
-
- virtio_dev_vring_start(vq);
- if (rxvq->mpool == NULL) {
- rte_exit(EXIT_FAILURE,
- "Cannot allocate mbufs for rx virtqueue");
- }
-
- /* Allocate blank mbufs for the each rx descriptor */
- nbufs = 0;
- error = ENOSPC;
-
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
- if (use_simple_rxtx) {
- for (desc_idx = 0; desc_idx < vq->vq_nentries;
- desc_idx++) {
- vq->vq_ring.avail->ring[desc_idx] = desc_idx;
- vq->vq_ring.desc[desc_idx].flags =
- VRING_DESC_F_WRITE;
- }
- }
-#endif
- memset(&rxvq->fake_mbuf, 0, sizeof(rxvq->fake_mbuf));
- for (desc_idx = 0; desc_idx < RTE_PMD_VIRTIO_RX_MAX_BURST;
- desc_idx++) {
- vq->sw_ring[vq->vq_nentries + desc_idx] =
- &rxvq->fake_mbuf;
- }
-
- while (!virtqueue_full(vq)) {
- m = rte_mbuf_raw_alloc(rxvq->mpool);
- if (m == NULL)
- break;
+ if (nb_desc == 0 || nb_desc > vq->vq_nentries)
+ nb_desc = vq->vq_nentries;
+ vq->vq_free_cnt = RTE_MIN(vq->vq_free_cnt, nb_desc);
- /******************************************
- * Enqueue allocated buffers *
- *******************************************/
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
- if (use_simple_rxtx)
- error = virtqueue_enqueue_recv_refill_simple(vq, m);
- else
-#endif
- error = virtqueue_enqueue_recv_refill(vq, m);
- if (error) {
- rte_pktmbuf_free(m);
- break;
- }
- nbufs++;
- }
+ rxvq = &vq->rxq;
+ rxvq->queue_id = queue_idx;
+ rxvq->mpool = mp;
+ if (rxvq->mpool == NULL) {
+ rte_exit(EXIT_FAILURE,
+ "Cannot allocate mbufs for rx virtqueue");
+ }
+ dev->data->rx_queues[queue_idx] = rxvq;
- vq_update_avail_idx(vq);
- PMD_INIT_LOG(DEBUG, "Allocated %d bufs", nbufs);
+ /* Allocate blank mbufs for the each rx descriptor */
+ nbufs = 0;
+ error = ENOSPC;
- VIRTQUEUE_DUMP(vq);
+ if (hw->use_simple_rxtx) {
+ for (desc_idx = 0; desc_idx < vq->vq_nentries;
+ desc_idx++) {
+ vq->vq_ring.avail->ring[desc_idx] = desc_idx;
+ vq->vq_ring.desc[desc_idx].flags =
+ VRING_DESC_F_WRITE;
+ }
}
- /* Start tx vring. */
- for (i = 0; i < dev->data->nb_tx_queues; i++) {
- struct virtnet_tx *txvq = dev->data->tx_queues[i];
- struct virtqueue *vq = txvq->vq;
-
- virtio_dev_vring_start(vq);
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
- if (use_simple_rxtx) {
- uint16_t mid_idx = vq->vq_nentries >> 1;
-
- for (desc_idx = 0; desc_idx < mid_idx; desc_idx++) {
- vq->vq_ring.avail->ring[desc_idx] =
- desc_idx + mid_idx;
- vq->vq_ring.desc[desc_idx + mid_idx].next =
- desc_idx;
- vq->vq_ring.desc[desc_idx + mid_idx].addr =
- txvq->virtio_net_hdr_mem +
- offsetof(struct virtio_tx_region, tx_hdr);
- vq->vq_ring.desc[desc_idx + mid_idx].len =
- vq->hw->vtnet_hdr_size;
- vq->vq_ring.desc[desc_idx + mid_idx].flags =
- VRING_DESC_F_NEXT;
- vq->vq_ring.desc[desc_idx].flags = 0;
- }
- for (desc_idx = mid_idx; desc_idx < vq->vq_nentries;
- desc_idx++)
- vq->vq_ring.avail->ring[desc_idx] = desc_idx;
- }
-#endif
- VIRTQUEUE_DUMP(vq);
+ memset(&rxvq->fake_mbuf, 0, sizeof(rxvq->fake_mbuf));
+ for (desc_idx = 0; desc_idx < RTE_PMD_VIRTIO_RX_MAX_BURST;
+ desc_idx++) {
+ vq->sw_ring[vq->vq_nentries + desc_idx] =
+ &rxvq->fake_mbuf;
}
-}
-int
-virtio_dev_rx_queue_setup(struct rte_eth_dev *dev,
- uint16_t queue_idx,
- uint16_t nb_desc,
- unsigned int socket_id,
- __rte_unused const struct rte_eth_rxconf *rx_conf,
- struct rte_mempool *mp)
-{
- uint16_t vtpci_queue_idx = 2 * queue_idx + VTNET_SQ_RQ_QUEUE_IDX;
- struct virtnet_rx *rxvq;
- int ret;
+ while (!virtqueue_full(vq)) {
+ m = rte_mbuf_raw_alloc(rxvq->mpool);
+ if (m == NULL)
+ break;
- PMD_INIT_FUNC_TRACE();
- ret = virtio_dev_queue_setup(dev, VTNET_RQ, queue_idx, vtpci_queue_idx,
- nb_desc, socket_id, (void **)&rxvq);
- if (ret < 0) {
- PMD_INIT_LOG(ERR, "rvq initialization failed");
- return ret;
+ /* Enqueue allocated buffers */
+ if (hw->use_simple_rxtx)
+ error = virtqueue_enqueue_recv_refill_simple(vq, m);
+ else
+ error = virtqueue_enqueue_recv_refill(vq, m);
+
+ if (error) {
+ rte_pktmbuf_free(m);
+ break;
+ }
+ nbufs++;
}
- /* Create mempool for rx mbuf allocation */
- rxvq->mpool = mp;
+ vq_update_avail_idx(vq);
- dev->data->rx_queues[queue_idx] = rxvq;
+ PMD_INIT_LOG(DEBUG, "Allocated %d bufs", nbufs);
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
virtio_rxq_vec_setup(rxvq);
-#endif
+
+ VIRTQUEUE_DUMP(vq);
return 0;
}
-void
-virtio_dev_rx_queue_release(void *rxq)
+static void
+virtio_update_rxtx_handler(struct rte_eth_dev *dev,
+ const struct rte_eth_txconf *tx_conf)
{
- struct virtnet_rx *rxvq = rxq;
- struct virtqueue *vq;
- const struct rte_memzone *mz;
-
- if (rxvq == NULL)
- return;
-
- /*
- * rxvq is freed when vq is freed, and as mz should be freed after the
- * del_queue, so we reserve the mz pointer first.
- */
- vq = rxvq->vq;
- mz = rxvq->mz;
+ uint8_t use_simple_rxtx = 0;
+ struct virtio_hw *hw = dev->data->dev_private;
- virtio_dev_queue_release(vq);
- rte_memzone_free(mz);
+#if defined RTE_ARCH_X86
+ if (rte_cpu_get_flag_enabled(RTE_CPUFLAG_SSE3))
+ use_simple_rxtx = 1;
+#elif defined RTE_ARCH_ARM64 || defined CONFIG_RTE_ARCH_ARM
+ if (rte_cpu_get_flag_enabled(RTE_CPUFLAG_NEON))
+ use_simple_rxtx = 1;
+#endif
+ /* Use simple rx/tx func if single segment and no offloads */
+ if (use_simple_rxtx &&
+ (tx_conf->txq_flags & VIRTIO_SIMPLE_FLAGS) == VIRTIO_SIMPLE_FLAGS &&
+ !vtpci_with_feature(hw, VIRTIO_NET_F_MRG_RXBUF)) {
+ PMD_INIT_LOG(INFO, "Using simple rx/tx path");
+ dev->tx_pkt_burst = virtio_xmit_pkts_simple;
+ dev->rx_pkt_burst = virtio_recv_pkts_vec;
+ hw->use_simple_rxtx = use_simple_rxtx;
+ }
}
/*
@@ -496,45 +526,26 @@ int
virtio_dev_tx_queue_setup(struct rte_eth_dev *dev,
uint16_t queue_idx,
uint16_t nb_desc,
- unsigned int socket_id,
+ unsigned int socket_id __rte_unused,
const struct rte_eth_txconf *tx_conf)
{
uint8_t vtpci_queue_idx = 2 * queue_idx + VTNET_SQ_TQ_QUEUE_IDX;
-
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
struct virtio_hw *hw = dev->data->dev_private;
-#endif
+ struct virtqueue *vq = hw->vqs[vtpci_queue_idx];
struct virtnet_tx *txvq;
- struct virtqueue *vq;
uint16_t tx_free_thresh;
- int ret;
+ uint16_t desc_idx;
PMD_INIT_FUNC_TRACE();
- if ((tx_conf->txq_flags & ETH_TXQ_FLAGS_NOXSUMS)
- != ETH_TXQ_FLAGS_NOXSUMS) {
- PMD_INIT_LOG(ERR, "TX checksum offload not supported\n");
- return -EINVAL;
- }
+ virtio_update_rxtx_handler(dev, tx_conf);
-#ifdef RTE_MACHINE_CPUFLAG_SSSE3
- /* Use simple rx/tx func if single segment and no offloads */
- if ((tx_conf->txq_flags & VIRTIO_SIMPLE_FLAGS) == VIRTIO_SIMPLE_FLAGS &&
- !vtpci_with_feature(hw, VIRTIO_NET_F_MRG_RXBUF)) {
- PMD_INIT_LOG(INFO, "Using simple rx/tx path");
- dev->tx_pkt_burst = virtio_xmit_pkts_simple;
- dev->rx_pkt_burst = virtio_recv_pkts_vec;
- use_simple_rxtx = 1;
- }
-#endif
+ if (nb_desc == 0 || nb_desc > vq->vq_nentries)
+ nb_desc = vq->vq_nentries;
+ vq->vq_free_cnt = RTE_MIN(vq->vq_free_cnt, nb_desc);
- ret = virtio_dev_queue_setup(dev, VTNET_TQ, queue_idx, vtpci_queue_idx,
- nb_desc, socket_id, (void **)&txvq);
- if (ret < 0) {
- PMD_INIT_LOG(ERR, "tvq initialization failed");
- return ret;
- }
- vq = txvq->vq;
+ txvq = &vq->txq;
+ txvq->queue_id = queue_idx;
tx_free_thresh = tx_conf->tx_free_thresh;
if (tx_free_thresh == 0)
@@ -552,32 +563,32 @@ virtio_dev_tx_queue_setup(struct rte_eth_dev *dev,
vq->vq_free_thresh = tx_free_thresh;
- dev->data->tx_queues[queue_idx] = txvq;
- return 0;
-}
-
-void
-virtio_dev_tx_queue_release(void *txq)
-{
- struct virtnet_tx *txvq = txq;
- struct virtqueue *vq;
- const struct rte_memzone *mz;
- const struct rte_memzone *hdr_mz;
-
- if (txvq == NULL)
- return;
+ if (hw->use_simple_rxtx) {
+ uint16_t mid_idx = vq->vq_nentries >> 1;
+
+ for (desc_idx = 0; desc_idx < mid_idx; desc_idx++) {
+ vq->vq_ring.avail->ring[desc_idx] =
+ desc_idx + mid_idx;
+ vq->vq_ring.desc[desc_idx + mid_idx].next =
+ desc_idx;
+ vq->vq_ring.desc[desc_idx + mid_idx].addr =
+ txvq->virtio_net_hdr_mem +
+ offsetof(struct virtio_tx_region, tx_hdr);
+ vq->vq_ring.desc[desc_idx + mid_idx].len =
+ vq->hw->vtnet_hdr_size;
+ vq->vq_ring.desc[desc_idx + mid_idx].flags =
+ VRING_DESC_F_NEXT;
+ vq->vq_ring.desc[desc_idx].flags = 0;
+ }
+ for (desc_idx = mid_idx; desc_idx < vq->vq_nentries;
+ desc_idx++)
+ vq->vq_ring.avail->ring[desc_idx] = desc_idx;
+ }
- /*
- * txvq is freed when vq is freed, and as mz should be freed after the
- * del_queue, so we reserve the mz pointer first.
- */
- vq = txvq->vq;
- mz = txvq->mz;
- hdr_mz = txvq->virtio_net_hdr_mz;
+ VIRTQUEUE_DUMP(vq);
- virtio_dev_queue_release(vq);
- rte_memzone_free(mz);
- rte_memzone_free(hdr_mz);
+ dev->data->tx_queues[queue_idx] = txvq;
+ return 0;
}
static void
@@ -627,6 +638,86 @@ virtio_update_packet_stats(struct virtnet_stats *stats, struct rte_mbuf *mbuf)
}
}
+/* Optionally fill offload information in structure */
+static int
+virtio_rx_offload(struct rte_mbuf *m, struct virtio_net_hdr *hdr)
+{
+ struct rte_net_hdr_lens hdr_lens;
+ uint32_t hdrlen, ptype;
+ int l4_supported = 0;
+
+ /* nothing to do */
+ if (hdr->flags == 0 && hdr->gso_type == VIRTIO_NET_HDR_GSO_NONE)
+ return 0;
+
+ m->ol_flags |= PKT_RX_IP_CKSUM_UNKNOWN;
+
+ ptype = rte_net_get_ptype(m, &hdr_lens, RTE_PTYPE_ALL_MASK);
+ m->packet_type = ptype;
+ if ((ptype & RTE_PTYPE_L4_MASK) == RTE_PTYPE_L4_TCP ||
+ (ptype & RTE_PTYPE_L4_MASK) == RTE_PTYPE_L4_UDP ||
+ (ptype & RTE_PTYPE_L4_MASK) == RTE_PTYPE_L4_SCTP)
+ l4_supported = 1;
+
+ if (hdr->flags & VIRTIO_NET_HDR_F_NEEDS_CSUM) {
+ hdrlen = hdr_lens.l2_len + hdr_lens.l3_len + hdr_lens.l4_len;
+ if (hdr->csum_start <= hdrlen && l4_supported) {
+ m->ol_flags |= PKT_RX_L4_CKSUM_NONE;
+ } else {
+ /* Unknown proto or tunnel, do sw cksum. We can assume
+ * the cksum field is in the first segment since the
+ * buffers we provided to the host are large enough.
+ * In case of SCTP, this will be wrong since it's a CRC
+ * but there's nothing we can do.
+ */
+ uint16_t csum, off;
+
+ rte_raw_cksum_mbuf(m, hdr->csum_start,
+ rte_pktmbuf_pkt_len(m) - hdr->csum_start,
+ &csum);
+ if (likely(csum != 0xffff))
+ csum = ~csum;
+ off = hdr->csum_offset + hdr->csum_start;
+ if (rte_pktmbuf_data_len(m) >= off + 1)
+ *rte_pktmbuf_mtod_offset(m, uint16_t *,
+ off) = csum;
+ }
+ } else if (hdr->flags & VIRTIO_NET_HDR_F_DATA_VALID && l4_supported) {
+ m->ol_flags |= PKT_RX_L4_CKSUM_GOOD;
+ }
+
+ /* GSO request, save required information in mbuf */
+ if (hdr->gso_type != VIRTIO_NET_HDR_GSO_NONE) {
+ /* Check unsupported modes */
+ if ((hdr->gso_type & VIRTIO_NET_HDR_GSO_ECN) ||
+ (hdr->gso_size == 0)) {
+ return -EINVAL;
+ }
+
+ /* Update mss lengthes in mbuf */
+ m->tso_segsz = hdr->gso_size;
+ switch (hdr->gso_type & ~VIRTIO_NET_HDR_GSO_ECN) {
+ case VIRTIO_NET_HDR_GSO_TCPV4:
+ case VIRTIO_NET_HDR_GSO_TCPV6:
+ m->ol_flags |= PKT_RX_LRO | \
+ PKT_RX_L4_CKSUM_NONE;
+ break;
+ default:
+ return -EINVAL;
+ }
+ }
+
+ return 0;
+}
+
+static inline int
+rx_offload_enabled(struct virtio_hw *hw)
+{
+ return vtpci_with_feature(hw, VIRTIO_NET_F_GUEST_CSUM) ||
+ vtpci_with_feature(hw, VIRTIO_NET_F_GUEST_TSO4) ||
+ vtpci_with_feature(hw, VIRTIO_NET_F_GUEST_TSO6);
+}
+
#define VIRTIO_MBUF_BURST_SZ 64
#define DESC_PER_CACHELINE (RTE_CACHE_LINE_SIZE / sizeof(struct vring_desc))
uint16_t
@@ -642,6 +733,8 @@ virtio_recv_pkts(void *rx_queue, struct rte_mbuf **rx_pkts, uint16_t nb_pkts)
int error;
uint32_t i, nb_enqueued;
uint32_t hdr_size;
+ int offload;
+ struct virtio_net_hdr *hdr;
nb_used = VIRTQUEUE_NUSED(vq);
@@ -659,6 +752,7 @@ virtio_recv_pkts(void *rx_queue, struct rte_mbuf **rx_pkts, uint16_t nb_pkts)
nb_rx = 0;
nb_enqueued = 0;
hdr_size = hw->vtnet_hdr_size;
+ offload = rx_offload_enabled(hw);
for (i = 0; i < num ; i++) {
rxm = rcv_pkts[i];
@@ -683,9 +777,18 @@ virtio_recv_pkts(void *rx_queue, struct rte_mbuf **rx_pkts, uint16_t nb_pkts)
rxm->pkt_len = (uint32_t)(len[i] - hdr_size);
rxm->data_len = (uint16_t)(len[i] - hdr_size);
+ hdr = (struct virtio_net_hdr *)((char *)rxm->buf_addr +
+ RTE_PKTMBUF_HEADROOM - hdr_size);
+
if (hw->vlan_strip)
rte_vlan_strip(rxm);
+ if (offload && virtio_rx_offload(rxm, hdr) < 0) {
+ virtio_discard_rxbuf(vq, rxm);
+ rxvq->stats.errors++;
+ continue;
+ }
+
VIRTIO_DUMP_PACKET(rxm, rxm->data_len);
rx_pkts[nb_rx++] = rxm;
@@ -745,6 +848,7 @@ virtio_recv_mergeable_pkts(void *rx_queue,
uint16_t extra_idx;
uint32_t seg_res;
uint32_t hdr_size;
+ int offload;
nb_used = VIRTQUEUE_NUSED(vq);
@@ -760,6 +864,7 @@ virtio_recv_mergeable_pkts(void *rx_queue,
extra_idx = 0;
seg_res = 0;
hdr_size = hw->vtnet_hdr_size;
+ offload = rx_offload_enabled(hw);
while (i < nb_used) {
struct virtio_net_hdr_mrg_rxbuf *header;
@@ -805,6 +910,12 @@ virtio_recv_mergeable_pkts(void *rx_queue,
rx_pkts[nb_rx] = rxm;
prev = rxm;
+ if (offload && virtio_rx_offload(rxm, &header->hdr) < 0) {
+ virtio_discard_rxbuf(vq, rxm);
+ rxvq->stats.errors++;
+ continue;
+ }
+
seg_res = seg_num - 1;
while (seg_res != 0) {
@@ -925,7 +1036,8 @@ virtio_xmit_pkts(void *tx_queue, struct rte_mbuf **tx_pkts, uint16_t nb_pkts)
}
/* optimize ring usage */
- if (vtpci_with_feature(hw, VIRTIO_F_ANY_LAYOUT) &&
+ if ((vtpci_with_feature(hw, VIRTIO_F_ANY_LAYOUT) ||
+ vtpci_with_feature(hw, VIRTIO_F_VERSION_1)) &&
rte_mbuf_refcnt_read(txm) == 1 &&
RTE_MBUF_DIRECT(txm) &&
txm->nb_segs == 1 &&