]> git.proxmox.com Git - mirror_ovs.git/blobdiff - lib/netdev-dpdk.c
dpif-netdev: Avoid reordering of packets in a batch with same megaflow
[mirror_ovs.git] / lib / netdev-dpdk.c
index af9843a943f1b752afe59d7954d12238dc6036f3..f91aa27cd350852a528232679898ca30f8fe95bd 100644 (file)
@@ -38,7 +38,9 @@
 #include <rte_pci.h>
 #include <rte_vhost.h>
 #include <rte_version.h>
+#include <rte_flow.h>
 
+#include "cmap.h"
 #include "dirs.h"
 #include "dp-packet.h"
 #include "dpdk.h"
@@ -51,6 +53,7 @@
 #include "openvswitch/list.h"
 #include "openvswitch/ofp-print.h"
 #include "openvswitch/vlog.h"
+#include "openvswitch/match.h"
 #include "ovs-numa.h"
 #include "ovs-thread.h"
 #include "ovs-rcu.h"
@@ -60,6 +63,7 @@
 #include "sset.h"
 #include "unaligned.h"
 #include "timeval.h"
+#include "uuid.h"
 #include "unixctl.h"
 
 enum {VIRTIO_RXQ, VIRTIO_TXQ, VIRTIO_QNUM};
@@ -91,13 +95,24 @@ static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
 #define NETDEV_DPDK_MBUF_ALIGN      1024
 #define NETDEV_DPDK_MAX_PKT_LEN     9728
 
-/* Min number of packets in the mempool.  OVS tries to allocate a mempool with
- * roughly estimated number of mbufs: if this fails (because the system doesn't
- * have enough hugepages) we keep halving the number until the allocation
- * succeeds or we reach MIN_NB_MBUF */
+/* Max and min number of packets in the mempool. OVS tries to allocate a
+ * mempool with MAX_NB_MBUF: if this fails (because the system doesn't have
+ * enough hugepages) we keep halving the number until the allocation succeeds
+ * or we reach MIN_NB_MBUF */
+
+#define MAX_NB_MBUF          (4096 * 64)
 #define MIN_NB_MBUF          (4096 * 4)
 #define MP_CACHE_SZ          RTE_MEMPOOL_CACHE_MAX_SIZE
 
+/* MAX_NB_MBUF can be divided by 2 many times, until MIN_NB_MBUF */
+BUILD_ASSERT_DECL(MAX_NB_MBUF % ROUND_DOWN_POW2(MAX_NB_MBUF / MIN_NB_MBUF)
+                  == 0);
+
+/* The smallest possible NB_MBUF that we're going to try should be a multiple
+ * of MP_CACHE_SZ. This is advised by DPDK documentation. */
+BUILD_ASSERT_DECL((MAX_NB_MBUF / ROUND_DOWN_POW2(MAX_NB_MBUF / MIN_NB_MBUF))
+                  % MP_CACHE_SZ == 0);
+
 /*
  * DPDK XSTATS Counter names definition
  */
@@ -144,6 +159,7 @@ static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
 
 /* DPDK library uses uint16_t for port_id. */
 typedef uint16_t dpdk_port_t;
+#define DPDK_PORT_ID_FMT "%"PRIu16
 
 #define VHOST_ENQ_RETRY_NUM 8
 #define IF_NAME_SZ (PATH_MAX > IFNAMSIZ ? PATH_MAX : IFNAMSIZ)
@@ -169,6 +185,17 @@ static const struct rte_eth_conf port_conf = {
     },
 };
 
+/*
+ * A mapping from ufid to dpdk rte_flow.
+ */
+static struct cmap ufid_to_rte_flow = CMAP_INITIALIZER;
+
+struct ufid_to_rte_flow_data {
+    struct cmap_node node;
+    ovs_u128 ufid;
+    struct rte_flow *rte_flow;
+};
+
 /*
  * These callbacks allow virtio-net devices to be added to vhost ports when
  * configuration has been fully completed.
@@ -271,7 +298,7 @@ struct dpdk_qos_ops {
      * For all QoS implementations it should always be non-null.
      */
     int (*qos_run)(struct qos_conf *qos_conf, struct rte_mbuf **pkts,
-                   int pkt_cnt, bool may_steal);
+                   int pkt_cnt, bool should_steal);
 };
 
 /* dpdk_qos_ops for each type of user space QoS implementation */
@@ -295,6 +322,18 @@ static struct ovs_list dpdk_list OVS_GUARDED_BY(dpdk_mutex)
 static struct ovs_mutex dpdk_mp_mutex OVS_ACQ_AFTER(dpdk_mutex)
     = OVS_MUTEX_INITIALIZER;
 
+/* Contains all 'struct dpdk_mp's. */
+static struct ovs_list dpdk_mp_list OVS_GUARDED_BY(dpdk_mp_mutex)
+    = OVS_LIST_INITIALIZER(&dpdk_mp_list);
+
+struct dpdk_mp {
+     struct rte_mempool *mp;
+     int mtu;
+     int socket_id;
+     int refcount;
+     struct ovs_list list_node OVS_GUARDED_BY(dpdk_mp_mutex);
+ };
+
 /* There should be one 'struct dpdk_tx_queue' created for
  * each cpu core. */
 struct dpdk_tx_queue {
@@ -330,6 +369,7 @@ struct ingress_policer {
 
 enum dpdk_hw_ol_features {
     NETDEV_RX_CHECKSUM_OFFLOAD = 1 << 0,
+    NETDEV_RX_HW_CRC_STRIP = 1 << 1,
 };
 
 /*
@@ -355,6 +395,8 @@ struct netdev_dpdk {
 
         /* If true, device was attached by rte_eth_dev_attach(). */
         bool attached;
+        /* If true, rte_eth_dev_start() was successfully called */
+        bool started;
         struct eth_addr hwaddr;
         int mtu;
         int socket_id;
@@ -362,16 +404,15 @@ struct netdev_dpdk {
         int max_packet_len;
         enum dpdk_dev_type type;
         enum netdev_flags flags;
+        int link_reset_cnt;
         char *devargs;  /* Device arguments for dpdk ports */
         struct dpdk_tx_queue *tx_q;
         struct rte_eth_link link;
-        int link_reset_cnt;
-        /* 4 pad bytes here. */
     );
 
     PADDED_MEMBERS_CACHELINE_MARKER(CACHE_LINE_SIZE, cacheline1,
         struct ovs_mutex mutex OVS_ACQ_AFTER(dpdk_mutex);
-        struct rte_mempool *mp;
+        struct dpdk_mp *dpdk_mp;
 
         /* virtio identifier for vhost devices */
         ovsrcu_index vid;
@@ -433,6 +474,12 @@ struct netdev_dpdk {
         /* DPDK-ETH hardware offload features,
          * from the enum set 'dpdk_hw_ol_features' */
         uint32_t hw_ol_features;
+
+        /* Properties for link state change detection mode.
+         * If lsc_interrupt_mode is set to false, poll mode is used,
+         * otherwise interrupt mode is used. */
+        bool requested_lsc_interrupt_mode;
+        bool lsc_interrupt_mode;
     );
 
     PADDED_MEMBERS(CACHE_LINE_SIZE,
@@ -510,36 +557,110 @@ ovs_rte_pktmbuf_init(struct rte_mempool *mp OVS_UNUSED,
     dp_packet_init_dpdk((struct dp_packet *) pkt, pkt->buf_len);
 }
 
-/* Returns a valid pointer when either of the following is true:
- *  - a new mempool was just created;
- *  - a matching mempool already exists. */
-static struct rte_mempool *
-dpdk_mp_create(struct netdev_dpdk *dev, int mtu)
+static int
+dpdk_mp_full(const struct rte_mempool *mp) OVS_REQUIRES(dpdk_mp_mutex)
+{
+    /* At this point we want to know if all the mbufs are back
+     * in the mempool. rte_mempool_full() is not atomic but it's
+     * the best available and as we are no longer requesting mbufs
+     * from the mempool, it means mbufs will not move from
+     * 'mempool ring' --> 'mempool cache'. In rte_mempool_full()
+     * the ring is counted before caches, so we won't get false
+     * positives in this use case and we handle false negatives.
+     *
+     * If future implementations of rte_mempool_full() were to change
+     * it could be possible for a false positive. Even that would
+     * likely be ok, as there are additional checks during mempool
+     * freeing but it would make things racey.
+     */
+    return rte_mempool_full(mp);
+}
+
+/* Free unused mempools. */
+static void
+dpdk_mp_sweep(void) OVS_REQUIRES(dpdk_mp_mutex)
+{
+    struct dpdk_mp *dmp, *next;
+
+    LIST_FOR_EACH_SAFE (dmp, next, list_node, &dpdk_mp_list) {
+        if (!dmp->refcount && dpdk_mp_full(dmp->mp)) {
+            VLOG_DBG("Freeing mempool \"%s\"", dmp->mp->name);
+            ovs_list_remove(&dmp->list_node);
+            rte_mempool_free(dmp->mp);
+            rte_free(dmp);
+        }
+    }
+}
+
+/* Calculating the required number of mbufs differs depending on the
+ * mempool model being used. Check if per port memory is in use before
+ * calculating.
+ */
+static uint32_t
+dpdk_calculate_mbufs(struct netdev_dpdk *dev, int mtu, bool per_port_mp)
+{
+    uint32_t n_mbufs;
+
+    if (!per_port_mp) {
+        /* Shared memory are being used.
+         * XXX: this is a really rough method of provisioning memory.
+         * It's impossible to determine what the exact memory requirements are
+         * when the number of ports and rxqs that utilize a particular mempool
+         * can change dynamically at runtime. For now, use this rough
+         * heurisitic.
+         */
+        if (mtu >= ETHER_MTU) {
+            n_mbufs = MAX_NB_MBUF;
+        } else {
+            n_mbufs = MIN_NB_MBUF;
+        }
+    } else {
+        /* Per port memory is being used.
+         * XXX: rough estimation of number of mbufs required for this port:
+         * <packets required to fill the device rxqs>
+         * + <packets that could be stuck on other ports txqs>
+         * + <packets in the pmd threads>
+         * + <additional memory for corner cases>
+         */
+        n_mbufs = dev->requested_n_rxq * dev->requested_rxq_size
+                  + dev->requested_n_txq * dev->requested_txq_size
+                  + MIN(RTE_MAX_LCORE, dev->requested_n_rxq) * NETDEV_MAX_BURST
+                  + MIN_NB_MBUF;
+    }
+
+    return n_mbufs;
+}
+
+static struct dpdk_mp *
+dpdk_mp_create(struct netdev_dpdk *dev, int mtu, bool per_port_mp)
 {
     char mp_name[RTE_MEMPOOL_NAMESIZE];
     const char *netdev_name = netdev_get_name(&dev->up);
     int socket_id = dev->requested_socket_id;
     uint32_t n_mbufs;
     uint32_t hash = hash_string(netdev_name, 0);
-    struct rte_mempool *mp = NULL;
+    struct dpdk_mp *dmp = NULL;
+    int ret;
 
-    /*
-     * XXX: rough estimation of number of mbufs required for this port:
-     * <packets required to fill the device rxqs>
-     * + <packets that could be stuck on other ports txqs>
-     * + <packets in the pmd threads>
-     * + <additional memory for corner cases>
-     */
-    n_mbufs = dev->requested_n_rxq * dev->requested_rxq_size
-              + dev->requested_n_txq * dev->requested_txq_size
-              + MIN(RTE_MAX_LCORE, dev->requested_n_rxq) * NETDEV_MAX_BURST
-              + MIN_NB_MBUF;
+    dmp = dpdk_rte_mzalloc(sizeof *dmp);
+    if (!dmp) {
+        return NULL;
+    }
+    dmp->socket_id = socket_id;
+    dmp->mtu = mtu;
+    dmp->refcount = 1;
+
+    n_mbufs = dpdk_calculate_mbufs(dev, mtu, per_port_mp);
 
-    ovs_mutex_lock(&dpdk_mp_mutex);
     do {
         /* Full DPDK memory pool name must be unique and cannot be
-         * longer than RTE_MEMPOOL_NAMESIZE. */
-        int ret = snprintf(mp_name, RTE_MEMPOOL_NAMESIZE,
+         * longer than RTE_MEMPOOL_NAMESIZE. Note that for the shared
+         * mempool case this can result in one device using a mempool
+         * which references a different device in it's name. However as
+         * mempool names are hashed, the device name will not be readable
+         * so this is not an issue for tasks such as debugging.
+         */
+        ret = snprintf(mp_name, RTE_MEMPOOL_NAMESIZE,
                            "ovs%08x%02d%05d%07u",
                            hash, socket_id, mtu, n_mbufs);
         if (ret < 0 || ret >= RTE_MEMPOOL_NAMESIZE) {
@@ -555,82 +676,159 @@ dpdk_mp_create(struct netdev_dpdk *dev, int mtu)
                   netdev_name, n_mbufs, socket_id,
                   dev->requested_n_rxq, dev->requested_n_txq);
 
-        mp = rte_pktmbuf_pool_create(mp_name, n_mbufs, MP_CACHE_SZ,
-                 sizeof (struct dp_packet) - sizeof (struct rte_mbuf),
-                 MBUF_SIZE(mtu) - sizeof(struct dp_packet), socket_id);
+        dmp->mp = rte_pktmbuf_pool_create(mp_name, n_mbufs,
+                                          MP_CACHE_SZ,
+                                          sizeof (struct dp_packet)
+                                          - sizeof (struct rte_mbuf),
+                                          MBUF_SIZE(mtu)
+                                          - sizeof(struct dp_packet),
+                                          socket_id);
 
-        if (mp) {
+        if (dmp->mp) {
             VLOG_DBG("Allocated \"%s\" mempool with %u mbufs",
                      mp_name, n_mbufs);
             /* rte_pktmbuf_pool_create has done some initialization of the
-             * rte_mbuf part of each dp_packet. Some OvS specific fields
-             * of the packet still need to be initialized by
-             * ovs_rte_pktmbuf_init. */
-            rte_mempool_obj_iter(mp, ovs_rte_pktmbuf_init, NULL);
+             * rte_mbuf part of each dp_packet, while ovs_rte_pktmbuf_init
+             * initializes some OVS specific fields of dp_packet.
+             */
+            rte_mempool_obj_iter(dmp->mp, ovs_rte_pktmbuf_init, NULL);
+            return dmp;
         } else if (rte_errno == EEXIST) {
             /* A mempool with the same name already exists.  We just
              * retrieve its pointer to be returned to the caller. */
-            mp = rte_mempool_lookup(mp_name);
+            dmp->mp = rte_mempool_lookup(mp_name);
             /* As the mempool create returned EEXIST we can expect the
              * lookup has returned a valid pointer.  If for some reason
              * that's not the case we keep track of it. */
             VLOG_DBG("A mempool with name \"%s\" already exists at %p.",
-                     mp_name, mp);
+                     mp_name, dmp->mp);
+            return dmp;
         } else {
-            VLOG_ERR("Failed mempool \"%s\" create request of %u mbufs",
-                     mp_name, n_mbufs);
+            VLOG_DBG("Failed to create mempool \"%s\" with a request of "
+                     "%u mbufs, retrying with %u mbufs",
+                     mp_name, n_mbufs, n_mbufs / 2);
+        }
+    } while (!dmp->mp && rte_errno == ENOMEM && (n_mbufs /= 2) >= MIN_NB_MBUF);
+
+    VLOG_ERR("Failed to create mempool \"%s\" with a request of %u mbufs",
+             mp_name, n_mbufs);
+
+    rte_free(dmp);
+    return NULL;
+}
+
+static struct dpdk_mp *
+dpdk_mp_get(struct netdev_dpdk *dev, int mtu, bool per_port_mp)
+{
+    struct dpdk_mp *dmp, *next;
+    bool reuse = false;
+
+    ovs_mutex_lock(&dpdk_mp_mutex);
+    /* Check if shared memory is being used, if so check existing mempools
+     * to see if reuse is possible. */
+    if (!per_port_mp) {
+        LIST_FOR_EACH (dmp, list_node, &dpdk_mp_list) {
+            if (dmp->socket_id == dev->requested_socket_id
+                && dmp->mtu == mtu) {
+                VLOG_DBG("Reusing mempool \"%s\"", dmp->mp->name);
+                dmp->refcount++;
+                reuse = true;
+                break;
+            }
+        }
+    }
+    /* Sweep mempools after reuse or before create. */
+    dpdk_mp_sweep();
+
+    if (!reuse) {
+        dmp = dpdk_mp_create(dev, mtu, per_port_mp);
+        if (dmp) {
+            /* Shared memory will hit the reuse case above so will not
+             * request a mempool that already exists but we need to check
+             * for the EEXIST case for per port memory case. Compare the
+             * mempool returned by dmp to each entry in dpdk_mp_list. If a
+             * match is found, free dmp as a new entry is not required, set
+             * dmp to point to the existing entry and increment the refcount
+             * to avoid being freed at a later stage.
+             */
+            if (per_port_mp && rte_errno == EEXIST) {
+                LIST_FOR_EACH (next, list_node, &dpdk_mp_list) {
+                    if (dmp->mp == next->mp) {
+                        rte_free(dmp);
+                        dmp = next;
+                        dmp->refcount++;
+                    }
+                }
+            } else {
+                ovs_list_push_back(&dpdk_mp_list, &dmp->list_node);
+            }
         }
-    } while (!mp && rte_errno == ENOMEM && (n_mbufs /= 2) >= MIN_NB_MBUF);
+    }
+
 
     ovs_mutex_unlock(&dpdk_mp_mutex);
-    return mp;
+
+    return dmp;
 }
 
-/* Release an existing mempool. */
+/* Decrement reference to a mempool. */
 static void
-dpdk_mp_free(struct rte_mempool *mp)
+dpdk_mp_put(struct dpdk_mp *dmp)
 {
-    if (!mp) {
+    if (!dmp) {
         return;
     }
 
     ovs_mutex_lock(&dpdk_mp_mutex);
-    VLOG_DBG("Releasing \"%s\" mempool", mp->name);
-    rte_mempool_free(mp);
+    ovs_assert(dmp->refcount);
+    dmp->refcount--;
     ovs_mutex_unlock(&dpdk_mp_mutex);
 }
 
-/* Tries to allocate a new mempool - or re-use an existing one where
- * appropriate - on requested_socket_id with a size determined by
- * requested_mtu and requested Rx/Tx queues.
- * On success - or when re-using an existing mempool - the new configuration
- * will be applied.
+/* Depending on the memory model being used this function tries to
+ * identify and reuse an existing mempool or tries to allocate a new
+ * mempool on requested_socket_id with mbuf size corresponding to the
+ * requested_mtu. On success, a new configuration will be applied.
  * On error, device will be left unchanged. */
 static int
 netdev_dpdk_mempool_configure(struct netdev_dpdk *dev)
     OVS_REQUIRES(dev->mutex)
 {
     uint32_t buf_size = dpdk_buf_size(dev->requested_mtu);
-    struct rte_mempool *mp;
+    struct dpdk_mp *dmp;
     int ret = 0;
+    bool per_port_mp = dpdk_per_port_memory();
 
-    mp = dpdk_mp_create(dev, FRAME_LEN_TO_MTU(buf_size));
-    if (!mp) {
+    /* With shared memory we do not need to configure a mempool if the MTU
+     * and socket ID have not changed, the previous configuration is still
+     * valid so return 0 */
+    if (!per_port_mp && dev->mtu == dev->requested_mtu
+        && dev->socket_id == dev->requested_socket_id) {
+        return ret;
+    }
+
+    dmp = dpdk_mp_get(dev, FRAME_LEN_TO_MTU(buf_size), per_port_mp);
+    if (!dmp) {
         VLOG_ERR("Failed to create memory pool for netdev "
                  "%s, with MTU %d on socket %d: %s\n",
                  dev->up.name, dev->requested_mtu, dev->requested_socket_id,
                  rte_strerror(rte_errno));
         ret = rte_errno;
     } else {
-        /* If a new MTU was requested and its rounded value equals the one
-         * that is currently used, then the existing mempool is returned. */
-        if (dev->mp != mp) {
-            /* A new mempool was created, release the previous one. */
-            dpdk_mp_free(dev->mp);
-        } else {
-            ret = EEXIST;
+        /* Check for any pre-existing dpdk_mp for the device before accessing
+         * the associated mempool.
+         */
+        if (dev->dpdk_mp != NULL) {
+            /* A new MTU was requested, decrement the reference count for the
+             * devices current dpdk_mp. This is required even if a pointer to
+             * same dpdk_mp is returned by dpdk_mp_get. The refcount for dmp
+             * has already been incremented by dpdk_mp_get at this stage so it
+             * must be decremented to keep an accurate refcount for the
+             * dpdk_mp.
+             */
+            dpdk_mp_put(dev->dpdk_mp);
         }
-        dev->mp = mp;
+        dev->dpdk_mp = dmp;
         dev->mtu = dev->requested_mtu;
         dev->socket_id = dev->requested_socket_id;
         dev->max_packet_len = MTU_TO_FRAME_LEN(dev->mtu);
@@ -652,12 +850,14 @@ check_link_status(struct netdev_dpdk *dev)
         dev->link_reset_cnt++;
         dev->link = link;
         if (dev->link.link_status) {
-            VLOG_DBG_RL(&rl, "Port %"PRIu8" Link Up - speed %u Mbps - %s",
+            VLOG_DBG_RL(&rl,
+                        "Port "DPDK_PORT_ID_FMT" Link Up - speed %u Mbps - %s",
                         dev->port_id, (unsigned) dev->link.link_speed,
-                        (dev->link.link_duplex == ETH_LINK_FULL_DUPLEX) ?
-                         ("full-duplex") : ("half-duplex"));
+                        (dev->link.link_duplex == ETH_LINK_FULL_DUPLEX)
+                        ? "full-duplex" : "half-duplex");
         } else {
-            VLOG_DBG_RL(&rl, "Port %"PRIu8" Link Down", dev->port_id);
+            VLOG_DBG_RL(&rl, "Port "DPDK_PORT_ID_FMT" Link Down",
+                        dev->port_id);
         }
     }
 }
@@ -686,20 +886,35 @@ dpdk_watchdog(void *dummy OVS_UNUSED)
 }
 
 static int
-dpdk_eth_dev_queue_setup(struct netdev_dpdk *dev, int n_rxq, int n_txq)
+dpdk_eth_dev_port_config(struct netdev_dpdk *dev, int n_rxq, int n_txq)
 {
     int diag = 0;
     int i;
     struct rte_eth_conf conf = port_conf;
-
-    /* For some NICs (e.g. Niantic), scatter_rx mode needs to be explicitly
-     * enabled. */
+    struct rte_eth_dev_info info;
+    uint16_t conf_mtu;
+
+    /* As of DPDK 17.11.1 a few PMDs require to explicitly enable
+     * scatter to support jumbo RX. Checking the offload capabilities
+     * is not an option as PMDs are not required yet to report
+     * them. The only reliable info is the driver name and knowledge
+     * (testing or code review). Listing all such PMDs feels harder
+     * than highlighting the one known not to need scatter */
     if (dev->mtu > ETHER_MTU) {
-        conf.rxmode.enable_scatter = 1;
+        rte_eth_dev_info_get(dev->port_id, &info);
+        if (strncmp(info.driver_name, "net_nfp", 7)) {
+            conf.rxmode.enable_scatter = 1;
+        }
     }
 
+    conf.intr_conf.lsc = dev->lsc_interrupt_mode;
     conf.rxmode.hw_ip_checksum = (dev->hw_ol_features &
                                   NETDEV_RX_CHECKSUM_OFFLOAD) != 0;
+
+    if (dev->hw_ol_features & NETDEV_RX_HW_CRC_STRIP) {
+        conf.rxmode.hw_strip_crc = 1;
+    }
+
     /* A device may report more queues than it makes available (this has
      * been observed for Intel xl710, which reserves some of them for
      * SRIOV):  rte_eth_*_queue_setup will fail if a queue is not
@@ -719,16 +934,26 @@ dpdk_eth_dev_queue_setup(struct netdev_dpdk *dev, int n_rxq, int n_txq)
 
         diag = rte_eth_dev_set_mtu(dev->port_id, dev->mtu);
         if (diag) {
-            VLOG_ERR("Interface %s MTU (%d) setup error: %s",
-                    dev->up.name, dev->mtu, rte_strerror(-diag));
-            break;
+            /* A device may not support rte_eth_dev_set_mtu, in this case
+             * flag a warning to the user and include the devices configured
+             * MTU value that will be used instead. */
+            if (-ENOTSUP == diag) {
+                rte_eth_dev_get_mtu(dev->port_id, &conf_mtu);
+                VLOG_WARN("Interface %s does not support MTU configuration, "
+                          "max packet size supported is %"PRIu16".",
+                          dev->up.name, conf_mtu);
+            } else {
+                VLOG_ERR("Interface %s MTU (%d) setup error: %s",
+                         dev->up.name, dev->mtu, rte_strerror(-diag));
+                break;
+            }
         }
 
         for (i = 0; i < n_txq; i++) {
             diag = rte_eth_tx_queue_setup(dev->port_id, i, dev->txq_size,
                                           dev->socket_id, NULL);
             if (diag) {
-                VLOG_INFO("Interface %s txq(%d) setup error: %s",
+                VLOG_INFO("Interface %s unable to setup txq(%d): %s",
                           dev->up.name, i, rte_strerror(-diag));
                 break;
             }
@@ -742,9 +967,10 @@ dpdk_eth_dev_queue_setup(struct netdev_dpdk *dev, int n_rxq, int n_txq)
 
         for (i = 0; i < n_rxq; i++) {
             diag = rte_eth_rx_queue_setup(dev->port_id, i, dev->rxq_size,
-                                          dev->socket_id, NULL, dev->mp);
+                                          dev->socket_id, NULL,
+                                          dev->dpdk_mp->mp);
             if (diag) {
-                VLOG_INFO("Interface %s rxq(%d) setup error: %s",
+                VLOG_INFO("Interface %s unable to setup rxq(%d): %s",
                           dev->up.name, i, rte_strerror(-diag));
                 break;
             }
@@ -769,7 +995,7 @@ static void
 dpdk_eth_flow_ctrl_setup(struct netdev_dpdk *dev) OVS_REQUIRES(dev->mutex)
 {
     if (rte_eth_dev_flow_ctrl_set(dev->port_id, &dev->fc_conf)) {
-        VLOG_WARN("Failed to enable flow control on device %"PRIu8,
+        VLOG_WARN("Failed to enable flow control on device "DPDK_PORT_ID_FMT,
                   dev->port_id);
     }
 }
@@ -789,10 +1015,17 @@ dpdk_eth_dev_init(struct netdev_dpdk *dev)
 
     rte_eth_dev_info_get(dev->port_id, &info);
 
+    if (strstr(info.driver_name, "vf") != NULL) {
+        VLOG_INFO("Virtual function detected, HW_CRC_STRIP will be enabled");
+        dev->hw_ol_features |= NETDEV_RX_HW_CRC_STRIP;
+    } else {
+        dev->hw_ol_features &= ~NETDEV_RX_HW_CRC_STRIP;
+    }
+
     if ((info.rx_offload_capa & rx_chksm_offload_capa) !=
             rx_chksm_offload_capa) {
-        VLOG_WARN("Rx checksum offload is not supported on port %"PRIu8,
-                        dev->port_id);
+        VLOG_WARN("Rx checksum offload is not supported on port "
+                  DPDK_PORT_ID_FMT, dev->port_id);
         dev->hw_ol_features &= ~NETDEV_RX_CHECKSUM_OFFLOAD;
     } else {
         dev->hw_ol_features |= NETDEV_RX_CHECKSUM_OFFLOAD;
@@ -801,10 +1034,13 @@ dpdk_eth_dev_init(struct netdev_dpdk *dev)
     n_rxq = MIN(info.max_rx_queues, dev->up.n_rxq);
     n_txq = MIN(info.max_tx_queues, dev->up.n_txq);
 
-    diag = dpdk_eth_dev_queue_setup(dev, n_rxq, n_txq);
+    diag = dpdk_eth_dev_port_config(dev, n_rxq, n_txq);
     if (diag) {
-        VLOG_ERR("Interface %s(rxq:%d txq:%d) configure error: %s",
-                 dev->up.name, n_rxq, n_txq, rte_strerror(-diag));
+        VLOG_ERR("Interface %s(rxq:%d txq:%d lsc interrupt mode:%s) "
+                 "configure error: %s",
+                 dev->up.name, n_rxq, n_txq,
+                 dev->lsc_interrupt_mode ? "true" : "false",
+                 rte_strerror(-diag));
         return -diag;
     }
 
@@ -814,28 +1050,21 @@ dpdk_eth_dev_init(struct netdev_dpdk *dev)
                  rte_strerror(-diag));
         return -diag;
     }
+    dev->started = true;
 
     rte_eth_promiscuous_enable(dev->port_id);
     rte_eth_allmulticast_enable(dev->port_id);
 
     memset(&eth_addr, 0x0, sizeof(eth_addr));
     rte_eth_macaddr_get(dev->port_id, &eth_addr);
-    VLOG_INFO_RL(&rl, "Port %"PRIu8": "ETH_ADDR_FMT,
-                    dev->port_id, ETH_ADDR_BYTES_ARGS(eth_addr.addr_bytes));
+    VLOG_INFO_RL(&rl, "Port "DPDK_PORT_ID_FMT": "ETH_ADDR_FMT,
+                 dev->port_id, ETH_ADDR_BYTES_ARGS(eth_addr.addr_bytes));
 
     memcpy(dev->hwaddr.ea, eth_addr.addr_bytes, ETH_ADDR_LEN);
     rte_eth_link_get_nowait(dev->port_id, &dev->link);
 
-    mbp_priv = rte_mempool_get_priv(dev->mp);
+    mbp_priv = rte_mempool_get_priv(dev->dpdk_mp->mp);
     dev->buf_size = mbp_priv->mbuf_data_room_size - RTE_PKTMBUF_HEADROOM;
-
-    /* Get the Flow control configuration for DPDK-ETH */
-    diag = rte_eth_dev_flow_ctrl_get(dev->port_id, &dev->fc_conf);
-    if (diag) {
-        VLOG_DBG("cannot get flow control parameters on port=%"PRIu8", err=%d",
-                 dev->port_id, diag);
-    }
-
     return 0;
 }
 
@@ -897,6 +1126,7 @@ common_construct(struct netdev *netdev, dpdk_port_t port_no,
     dev->flags = 0;
     dev->requested_mtu = ETHER_MTU;
     dev->max_packet_len = MTU_TO_FRAME_LEN(dev->mtu);
+    dev->requested_lsc_interrupt_mode = 0;
     ovsrcu_index_init(&dev->vid, -1);
     dev->vhost_reconfigured = false;
     dev->attached = false;
@@ -1079,7 +1309,7 @@ common_destruct(struct netdev_dpdk *dev)
     OVS_EXCLUDED(dev->mutex)
 {
     rte_free(dev->tx_q);
-    dpdk_mp_free(dev->mp);
+    dpdk_mp_put(dev->dpdk_mp);
 
     ovs_list_remove(&dev->list_node);
     free(ovsrcu_get_protected(struct ingress_policer *,
@@ -1096,6 +1326,7 @@ netdev_dpdk_destruct(struct netdev *netdev)
     ovs_mutex_lock(&dpdk_mutex);
 
     rte_eth_dev_stop(dev->port_id);
+    dev->started = false;
 
     if (dev->attached) {
         rte_eth_dev_close(dev->port_id);
@@ -1222,7 +1453,8 @@ netdev_dpdk_configure_xstats(struct netdev_dpdk *dev)
                 rte_eth_xstats_get_names(dev->port_id, NULL, 0);
 
         if (dev->rte_xstats_names_size < 0) {
-            VLOG_WARN("Cannot get XSTATS for port: %"PRIu8, dev->port_id);
+            VLOG_WARN("Cannot get XSTATS for port: "DPDK_PORT_ID_FMT,
+                      dev->port_id);
             dev->rte_xstats_names_size = 0;
         } else {
             /* Reserve memory for xstats names and values */
@@ -1237,12 +1469,12 @@ netdev_dpdk_configure_xstats(struct netdev_dpdk *dev)
                                                  dev->rte_xstats_names_size);
 
                 if (rte_xstats_len < 0) {
-                    VLOG_WARN("Cannot get XSTATS names for port: %"PRIu8,
-                               dev->port_id);
+                    VLOG_WARN("Cannot get XSTATS names for port: "
+                              DPDK_PORT_ID_FMT, dev->port_id);
                     goto out;
                 } else if (rte_xstats_len != dev->rte_xstats_names_size) {
-                    VLOG_WARN("XSTATS size doesn't match for port: %"PRIu8,
-                              dev->port_id);
+                    VLOG_WARN("XSTATS size doesn't match for port: "
+                              DPDK_PORT_ID_FMT, dev->port_id);
                     goto out;
                 }
 
@@ -1274,8 +1506,8 @@ netdev_dpdk_configure_xstats(struct netdev_dpdk *dev)
                     dev->rte_xstats_ids_size = xstats_no;
                     ret = true;
                 } else {
-                    VLOG_WARN("Can't get XSTATS IDs for port: %"PRIu8,
-                              dev->port_id);
+                    VLOG_WARN("Can't get XSTATS IDs for port: "
+                              DPDK_PORT_ID_FMT, dev->port_id);
                 }
 
                 free(rte_xstats);
@@ -1320,6 +1552,8 @@ netdev_dpdk_get_config(const struct netdev *netdev, struct smap *args)
         } else {
             smap_add(args, "rx_csum_offload", "false");
         }
+        smap_add(args, "lsc_interrupt_mode",
+                 dev->lsc_interrupt_mode ? "true" : "false");
     }
     ovs_mutex_unlock(&dev->mutex);
 
@@ -1443,7 +1677,7 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
                        char **errp)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-    bool rx_fc_en, tx_fc_en, autoneg;
+    bool rx_fc_en, tx_fc_en, autoneg, lsc_interrupt_mode;
     enum rte_eth_fc_mode fc_mode;
     static const enum rte_eth_fc_mode fc_mode_set[2][2] = {
         {RTE_FC_NONE,     RTE_FC_TX_PAUSE},
@@ -1520,6 +1754,12 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
         goto out;
     }
 
+    lsc_interrupt_mode = smap_get_bool(args, "dpdk-lsc-interrupt", false);
+    if (dev->requested_lsc_interrupt_mode != lsc_interrupt_mode) {
+        dev->requested_lsc_interrupt_mode = lsc_interrupt_mode;
+        netdev_request_reconfigure(netdev);
+    }
+
     rx_fc_en = smap_get_bool(args, "rx-flow-ctrl", false);
     tx_fc_en = smap_get_bool(args, "tx-flow-ctrl", false);
     autoneg = smap_get_bool(args, "flow-ctrl-autoneg", false);
@@ -1528,6 +1768,12 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
     if (dev->fc_conf.mode != fc_mode || autoneg != dev->fc_conf.autoneg) {
         dev->fc_conf.mode = fc_mode;
         dev->fc_conf.autoneg = autoneg;
+        /* Get the Flow control configuration for DPDK-ETH */
+        err = rte_eth_dev_flow_ctrl_get(dev->port_id, &dev->fc_conf);
+        if (err) {
+            VLOG_WARN("Cannot get flow control parameters on port "
+                DPDK_PORT_ID_FMT", err=%d", dev->port_id, err);
+        }
         dpdk_eth_flow_ctrl_setup(dev);
     }
 
@@ -1697,7 +1943,7 @@ netdev_dpdk_policer_pkt_handle(struct rte_meter_srtcm *meter,
 static int
 netdev_dpdk_policer_run(struct rte_meter_srtcm *meter,
                         struct rte_mbuf **pkts, int pkt_cnt,
-                        bool may_steal)
+                        bool should_steal)
 {
     int i = 0;
     int cnt = 0;
@@ -1713,7 +1959,7 @@ netdev_dpdk_policer_run(struct rte_meter_srtcm *meter,
             }
             cnt++;
         } else {
-            if (may_steal) {
+            if (should_steal) {
                 rte_pktmbuf_free(pkt);
             }
         }
@@ -1724,13 +1970,13 @@ netdev_dpdk_policer_run(struct rte_meter_srtcm *meter,
 
 static int
 ingress_policer_run(struct ingress_policer *policer, struct rte_mbuf **pkts,
-                    int pkt_cnt, bool may_steal)
+                    int pkt_cnt, bool should_steal)
 {
     int cnt = 0;
 
     rte_spinlock_lock(&policer->policer_lock);
     cnt = netdev_dpdk_policer_run(&policer->in_policer, pkts,
-                                  pkt_cnt, may_steal);
+                                  pkt_cnt, should_steal);
     rte_spinlock_unlock(&policer->policer_lock);
 
     return cnt;
@@ -1808,13 +2054,13 @@ netdev_dpdk_vhost_update_rx_counters(struct netdev_stats *stats,
  */
 static int
 netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
-                           struct dp_packet_batch *batch)
+                           struct dp_packet_batch *batch, int *qfill)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(rxq->netdev);
     struct ingress_policer *policer = netdev_dpdk_get_ingress_policer(dev);
     uint16_t nb_rx = 0;
     uint16_t dropped = 0;
-    int qid = rxq->queue_id;
+    int qid = rxq->queue_id * VIRTIO_QNUM + VIRTIO_TXQ;
     int vid = netdev_dpdk_get_vid(dev);
 
     if (OVS_UNLIKELY(vid < 0 || !dev->vhost_reconfigured
@@ -1822,14 +2068,23 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
         return EAGAIN;
     }
 
-    nb_rx = rte_vhost_dequeue_burst(vid, qid * VIRTIO_QNUM + VIRTIO_TXQ,
-                                    dev->mp,
+    nb_rx = rte_vhost_dequeue_burst(vid, qid, dev->dpdk_mp->mp,
                                     (struct rte_mbuf **) batch->packets,
                                     NETDEV_MAX_BURST);
     if (!nb_rx) {
         return EAGAIN;
     }
 
+    if (qfill) {
+        if (nb_rx == NETDEV_MAX_BURST) {
+            /* The DPDK API returns a uint32_t which often has invalid bits in
+             * the upper 16-bits. Need to restrict the value to uint16_t. */
+            *qfill = rte_vhost_rx_queue_count(vid, qid) & UINT16_MAX;
+        } else {
+            *qfill = 0;
+        }
+    }
+
     if (policer) {
         dropped = nb_rx;
         nb_rx = ingress_policer_run(policer,
@@ -1850,7 +2105,8 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
 }
 
 static int
-netdev_dpdk_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch)
+netdev_dpdk_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch,
+                     int *qfill)
 {
     struct netdev_rxq_dpdk *rx = netdev_rxq_dpdk_cast(rxq);
     struct netdev_dpdk *dev = netdev_dpdk_cast(rxq->netdev);
@@ -1887,18 +2143,26 @@ netdev_dpdk_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch)
     batch->count = nb_rx;
     dp_packet_batch_init_packet_fields(batch);
 
+    if (qfill) {
+        if (nb_rx == NETDEV_MAX_BURST) {
+            *qfill = rte_eth_rx_queue_count(rx->port_id, rxq->queue_id);
+        } else {
+            *qfill = 0;
+        }
+    }
+
     return 0;
 }
 
 static inline int
 netdev_dpdk_qos_run(struct netdev_dpdk *dev, struct rte_mbuf **pkts,
-                    int cnt, bool may_steal)
+                    int cnt, bool should_steal)
 {
     struct qos_conf *qos_conf = ovsrcu_get(struct qos_conf *, &dev->qos_conf);
 
     if (qos_conf) {
         rte_spinlock_lock(&qos_conf->lock);
-        cnt = qos_conf->ops->qos_run(qos_conf, pkts, cnt, may_steal);
+        cnt = qos_conf->ops->qos_run(qos_conf, pkts, cnt, should_steal);
         rte_spinlock_unlock(&qos_conf->lock);
     }
 
@@ -2043,7 +2307,7 @@ dpdk_do_tx_copy(struct netdev *netdev, int qid, struct dp_packet_batch *batch)
             continue;
         }
 
-        pkts[txcnt] = rte_pktmbuf_alloc(dev->mp);
+        pkts[txcnt] = rte_pktmbuf_alloc(dev->dpdk_mp->mp);
         if (OVS_UNLIKELY(!pkts[txcnt])) {
             dropped += cnt - i;
             break;
@@ -2318,7 +2582,8 @@ netdev_dpdk_get_stats(const struct netdev *netdev, struct netdev_stats *stats)
     int rte_xstats_len, rte_xstats_new_len, rte_xstats_ret;
 
     if (rte_eth_stats_get(dev->port_id, &rte_stats)) {
-        VLOG_ERR("Can't get ETH statistics for port: %"PRIu8, dev->port_id);
+        VLOG_ERR("Can't get ETH statistics for port: "DPDK_PORT_ID_FMT,
+                 dev->port_id);
         ovs_mutex_unlock(&dev->mutex);
         return EPROTO;
     }
@@ -2326,7 +2591,8 @@ netdev_dpdk_get_stats(const struct netdev *netdev, struct netdev_stats *stats)
     /* Get length of statistics */
     rte_xstats_len = rte_eth_xstats_get_names(dev->port_id, NULL, 0);
     if (rte_xstats_len < 0) {
-        VLOG_WARN("Cannot get XSTATS values for port: %"PRIu8, dev->port_id);
+        VLOG_WARN("Cannot get XSTATS values for port: "DPDK_PORT_ID_FMT,
+                  dev->port_id);
         goto out;
     }
     /* Reserve memory for xstats names and values */
@@ -2338,7 +2604,8 @@ netdev_dpdk_get_stats(const struct netdev *netdev, struct netdev_stats *stats)
                                                   rte_xstats_names,
                                                   rte_xstats_len);
     if (rte_xstats_new_len != rte_xstats_len) {
-        VLOG_WARN("Cannot get XSTATS names for port: %"PRIu8, dev->port_id);
+        VLOG_WARN("Cannot get XSTATS names for port: "DPDK_PORT_ID_FMT,
+                  dev->port_id);
         goto out;
     }
     /* Retreive xstats values */
@@ -2349,7 +2616,8 @@ netdev_dpdk_get_stats(const struct netdev *netdev, struct netdev_stats *stats)
         netdev_dpdk_convert_xstats(stats, rte_xstats, rte_xstats_names,
                                    rte_xstats_len);
     } else {
-        VLOG_WARN("Cannot get XSTATS values for port: %"PRIu8, dev->port_id);
+        VLOG_WARN("Cannot get XSTATS values for port: "DPDK_PORT_ID_FMT,
+                  dev->port_id);
     }
 
 out:
@@ -2413,7 +2681,7 @@ netdev_dpdk_get_custom_stats(const struct netdev *netdev,
                 custom_stats->counters[i].value = values[i];
             }
         } else {
-            VLOG_WARN("Cannot get XSTATS values for port: %"PRIu8,
+            VLOG_WARN("Cannot get XSTATS values for port: "DPDK_PORT_ID_FMT,
                       dev->port_id);
             custom_stats->counters = NULL;
             custom_stats->size = 0;
@@ -2467,6 +2735,9 @@ netdev_dpdk_get_features(const struct netdev *netdev,
         if (link.link_speed == ETH_SPEED_NUM_10G) {
             *current = NETDEV_F_10GB_FD;
         }
+        if (link.link_speed == ETH_SPEED_NUM_40G) {
+            *current = NETDEV_F_40GB_FD;
+        }
     }
 
     if (link.link_autoneg) {
@@ -2726,11 +2997,10 @@ netdev_dpdk_vhost_user_get_status(const struct netdev *netdev,
 
     for (int i = 0; i < vring_num; i++) {
         struct rte_vhost_vring vring;
-        char vhost_vring[16];
 
         rte_vhost_get_vhost_vring(vid, i, &vring);
-        snprintf(vhost_vring, 16, "vring_%d_size", i);
-        smap_add_format(args, vhost_vring, "%d", vring.size);
+        smap_add_nocopy(args, xasprintf("vring_%d_size", i),
+                        xasprintf("%d", vring.size));
     }
 
     ovs_mutex_unlock(&dev->mutex);
@@ -2751,7 +3021,7 @@ netdev_dpdk_get_status(const struct netdev *netdev, struct smap *args)
     rte_eth_dev_info_get(dev->port_id, &dev_info);
     ovs_mutex_unlock(&dev->mutex);
 
-    smap_add_format(args, "port_no", "%d", dev->port_id);
+    smap_add_format(args, "port_no", DPDK_PORT_ID_FMT, dev->port_id);
     smap_add_format(args, "numa_id", "%d",
                            rte_eth_dev_socket_id(dev->port_id));
     smap_add_format(args, "driver_name", "%s", dev_info.driver_name);
@@ -2774,7 +3044,7 @@ netdev_dpdk_get_status(const struct netdev *netdev, struct smap *args)
                                                dev_info.driver_name);
 
     if (dev_info.pci_dev) {
-        smap_add_format(args, "pci-vendor_id", "0x%u",
+        smap_add_format(args, "pci-vendor_id", "0x%x",
                         dev_info.pci_dev->id.vendor_id);
         smap_add_format(args, "pci-device_id", "0x%x",
                         dev_info.pci_dev->id.device_id);
@@ -2919,7 +3189,7 @@ netdev_dpdk_get_mempool_info(struct unixctl_conn *conn,
         ovs_mutex_lock(&dev->mutex);
         ovs_mutex_lock(&dpdk_mp_mutex);
 
-        rte_mempool_dump(stream, dev->mp);
+        rte_mempool_dump(stream, dev->dpdk_mp->mp);
 
         ovs_mutex_unlock(&dpdk_mp_mutex);
         ovs_mutex_unlock(&dev->mutex);
@@ -3164,6 +3434,19 @@ vring_state_changed(int vid, uint16_t queue_id, int enable)
     return 0;
 }
 
+/*
+ * Retrieve the DPDK virtio device ID (vid) associated with a vhostuser
+ * or vhostuserclient netdev.
+ *
+ * Returns a value greater or equal to zero for a valid vid or '-1' if
+ * there is no valid vid associated. A vid of '-1' must not be used in
+ * rte_vhost_ APi calls.
+ *
+ * Once obtained and validated, a vid can be used by a PMD for multiple
+ * subsequent rte_vhost API calls until the PMD quiesces. A PMD should
+ * not fetch the vid again for each of a series of API calls.
+ */
+
 int
 netdev_dpdk_get_vid(const struct netdev_dpdk *dev)
 {
@@ -3514,14 +3797,14 @@ egress_policer_qos_is_equal(const struct qos_conf *conf,
 
 static int
 egress_policer_run(struct qos_conf *conf, struct rte_mbuf **pkts, int pkt_cnt,
-                   bool may_steal)
+                   bool should_steal)
 {
     int cnt = 0;
     struct egress_policer *policer =
         CONTAINER_OF(conf, struct egress_policer, qos_conf);
 
     cnt = netdev_dpdk_policer_run(&policer->egress_meter, pkts,
-                                  pkt_cnt, may_steal);
+                                  pkt_cnt, should_steal);
 
     return cnt;
 }
@@ -3546,21 +3829,26 @@ netdev_dpdk_reconfigure(struct netdev *netdev)
     if (netdev->n_txq == dev->requested_n_txq
         && netdev->n_rxq == dev->requested_n_rxq
         && dev->mtu == dev->requested_mtu
+        && dev->lsc_interrupt_mode == dev->requested_lsc_interrupt_mode
         && dev->rxq_size == dev->requested_rxq_size
         && dev->txq_size == dev->requested_txq_size
-        && dev->socket_id == dev->requested_socket_id) {
+        && dev->socket_id == dev->requested_socket_id
+        && dev->started) {
         /* Reconfiguration is unnecessary */
 
         goto out;
     }
 
     rte_eth_dev_stop(dev->port_id);
+    dev->started = false;
 
     err = netdev_dpdk_mempool_configure(dev);
     if (err && err != EEXIST) {
         goto out;
     }
 
+    dev->lsc_interrupt_mode = dev->requested_lsc_interrupt_mode;
+
     netdev->n_txq = dev->requested_n_txq;
     netdev->n_rxq = dev->requested_n_rxq;
 
@@ -3598,7 +3886,7 @@ dpdk_vhost_reconfigure_helper(struct netdev_dpdk *dev)
 
     err = netdev_dpdk_mempool_configure(dev);
     if (!err) {
-        /* A new mempool was created. */
+        /* A new mempool was created or re-used. */
         netdev_change_seq_changed(&dev->up);
     } else if (err != EEXIST){
         return err;
@@ -3709,149 +3997,784 @@ unlock:
     return err;
 }
 
-#define NETDEV_DPDK_CLASS(NAME, INIT, CONSTRUCT, DESTRUCT,    \
-                          SET_CONFIG, SET_TX_MULTIQ, SEND,    \
-                          GET_CARRIER, GET_STATS,                        \
-                          GET_CUSTOM_STATS,                                      \
-                          GET_FEATURES, GET_STATUS,           \
-                          RECONFIGURE, RXQ_RECV)              \
-{                                                             \
-    NAME,                                                     \
-    true,                       /* is_pmd */                  \
-    INIT,                       /* init */                    \
-    NULL,                       /* netdev_dpdk_run */         \
-    NULL,                       /* netdev_dpdk_wait */        \
-                                                              \
-    netdev_dpdk_alloc,                                        \
-    CONSTRUCT,                                                \
-    DESTRUCT,                                                 \
-    netdev_dpdk_dealloc,                                      \
-    netdev_dpdk_get_config,                                   \
-    SET_CONFIG,                                               \
-    NULL,                       /* get_tunnel_config */       \
-    NULL,                       /* build header */            \
-    NULL,                       /* push header */             \
-    NULL,                       /* pop header */              \
-    netdev_dpdk_get_numa_id,    /* get_numa_id */             \
-    SET_TX_MULTIQ,                                            \
-                                                              \
-    SEND,                       /* send */                    \
-    NULL,                       /* send_wait */               \
-                                                              \
-    netdev_dpdk_set_etheraddr,                                \
-    netdev_dpdk_get_etheraddr,                                \
-    netdev_dpdk_get_mtu,                                      \
-    netdev_dpdk_set_mtu,                                      \
-    netdev_dpdk_get_ifindex,                                  \
-    GET_CARRIER,                                              \
-    netdev_dpdk_get_carrier_resets,                           \
-    netdev_dpdk_set_miimon,                                   \
-    GET_STATS,                                                \
-    GET_CUSTOM_STATS,                                                                            \
-    GET_FEATURES,                                             \
-    NULL,                       /* set_advertisements */      \
-    NULL,                       /* get_pt_mode */             \
-                                                              \
-    netdev_dpdk_set_policing,                                 \
-    netdev_dpdk_get_qos_types,                                \
-    NULL,                       /* get_qos_capabilities */    \
-    netdev_dpdk_get_qos,                                      \
-    netdev_dpdk_set_qos,                                      \
-    NULL,                       /* get_queue */               \
-    NULL,                       /* set_queue */               \
-    NULL,                       /* delete_queue */            \
-    NULL,                       /* get_queue_stats */         \
-    NULL,                       /* queue_dump_start */        \
-    NULL,                       /* queue_dump_next */         \
-    NULL,                       /* queue_dump_done */         \
-    NULL,                       /* dump_queue_stats */        \
-                                                              \
-    NULL,                       /* set_in4 */                 \
-    NULL,                       /* get_addr_list */           \
-    NULL,                       /* add_router */              \
-    NULL,                       /* get_next_hop */            \
-    GET_STATUS,                                               \
-    NULL,                       /* arp_lookup */              \
-                                                              \
-    netdev_dpdk_update_flags,                                 \
-    RECONFIGURE,                                              \
-                                                              \
-    netdev_dpdk_rxq_alloc,                                    \
-    netdev_dpdk_rxq_construct,                                \
-    netdev_dpdk_rxq_destruct,                                 \
-    netdev_dpdk_rxq_dealloc,                                  \
-    RXQ_RECV,                                                 \
-    NULL,                       /* rx_wait */                 \
-    NULL,                       /* rxq_drain */               \
-    NO_OFFLOAD_API                                            \
-}
-
-static const struct netdev_class dpdk_class =
-    NETDEV_DPDK_CLASS(
-        "dpdk",
-        netdev_dpdk_class_init,
-        netdev_dpdk_construct,
-        netdev_dpdk_destruct,
-        netdev_dpdk_set_config,
-        netdev_dpdk_set_tx_multiq,
-        netdev_dpdk_eth_send,
-        netdev_dpdk_get_carrier,
-        netdev_dpdk_get_stats,
-        netdev_dpdk_get_custom_stats,
-        netdev_dpdk_get_features,
-        netdev_dpdk_get_status,
-        netdev_dpdk_reconfigure,
-        netdev_dpdk_rxq_recv);
-
-static const struct netdev_class dpdk_ring_class =
-    NETDEV_DPDK_CLASS(
-        "dpdkr",
-        netdev_dpdk_class_init,
-        netdev_dpdk_ring_construct,
-        netdev_dpdk_destruct,
-        netdev_dpdk_ring_set_config,
-        netdev_dpdk_set_tx_multiq,
-        netdev_dpdk_ring_send,
-        netdev_dpdk_get_carrier,
-        netdev_dpdk_get_stats,
-        netdev_dpdk_get_custom_stats,
-        netdev_dpdk_get_features,
-        netdev_dpdk_get_status,
-        netdev_dpdk_reconfigure,
-        netdev_dpdk_rxq_recv);
-
-static const struct netdev_class dpdk_vhost_class =
-    NETDEV_DPDK_CLASS(
-        "dpdkvhostuser",
-        NULL,
-        netdev_dpdk_vhost_construct,
-        netdev_dpdk_vhost_destruct,
-        NULL,
-        NULL,
-        netdev_dpdk_vhost_send,
-        netdev_dpdk_vhost_get_carrier,
-        netdev_dpdk_vhost_get_stats,
-        NULL,
-        NULL,
-        netdev_dpdk_vhost_user_get_status,
-        netdev_dpdk_vhost_reconfigure,
-        netdev_dpdk_vhost_rxq_recv);
-static const struct netdev_class dpdk_vhost_client_class =
-    NETDEV_DPDK_CLASS(
-        "dpdkvhostuserclient",
-        NULL,
-        netdev_dpdk_vhost_client_construct,
-        netdev_dpdk_vhost_destruct,
-        netdev_dpdk_vhost_client_set_config,
-        NULL,
-        netdev_dpdk_vhost_send,
-        netdev_dpdk_vhost_get_carrier,
-        netdev_dpdk_vhost_get_stats,
-        NULL,
-        NULL,
-        netdev_dpdk_vhost_user_get_status,
-        netdev_dpdk_vhost_client_reconfigure,
-        netdev_dpdk_vhost_rxq_recv);
+
+/* Find rte_flow with @ufid */
+static struct rte_flow *
+ufid_to_rte_flow_find(const ovs_u128 *ufid) {
+    size_t hash = hash_bytes(ufid, sizeof(*ufid), 0);
+    struct ufid_to_rte_flow_data *data;
+
+    CMAP_FOR_EACH_WITH_HASH (data, node, hash, &ufid_to_rte_flow) {
+        if (ovs_u128_equals(*ufid, data->ufid)) {
+            return data->rte_flow;
+        }
+    }
+
+    return NULL;
+}
+
+static inline void
+ufid_to_rte_flow_associate(const ovs_u128 *ufid,
+                           struct rte_flow *rte_flow) {
+    size_t hash = hash_bytes(ufid, sizeof(*ufid), 0);
+    struct ufid_to_rte_flow_data *data = xzalloc(sizeof(*data));
+
+    /*
+     * We should not simply overwrite an existing rte flow.
+     * We should have deleted it first before re-adding it.
+     * Thus, if following assert triggers, something is wrong:
+     * the rte_flow is not destroyed.
+     */
+    ovs_assert(ufid_to_rte_flow_find(ufid) == NULL);
+
+    data->ufid = *ufid;
+    data->rte_flow = rte_flow;
+
+    cmap_insert(&ufid_to_rte_flow,
+                CONST_CAST(struct cmap_node *, &data->node), hash);
+}
+
+static inline void
+ufid_to_rte_flow_disassociate(const ovs_u128 *ufid) {
+    size_t hash = hash_bytes(ufid, sizeof(*ufid), 0);
+    struct ufid_to_rte_flow_data *data;
+
+    CMAP_FOR_EACH_WITH_HASH (data, node, hash, &ufid_to_rte_flow) {
+        if (ovs_u128_equals(*ufid, data->ufid)) {
+            cmap_remove(&ufid_to_rte_flow,
+                        CONST_CAST(struct cmap_node *, &data->node), hash);
+            free(data);
+            return;
+        }
+    }
+
+    VLOG_WARN("ufid "UUID_FMT" is not associated with an rte flow\n",
+              UUID_ARGS((struct uuid *)ufid));
+}
+
+/*
+ * To avoid individual xrealloc calls for each new element, a 'curent_max'
+ * is used to keep track of current allocated number of elements. Starts
+ * by 8 and doubles on each xrealloc call
+ */
+struct flow_patterns {
+    struct rte_flow_item *items;
+    int cnt;
+    int current_max;
+};
+
+struct flow_actions {
+    struct rte_flow_action *actions;
+    int cnt;
+    int current_max;
+};
+
+static void
+dump_flow_pattern(struct rte_flow_item *item)
+{
+    if (item->type == RTE_FLOW_ITEM_TYPE_ETH) {
+        const struct rte_flow_item_eth *eth_spec = item->spec;
+        const struct rte_flow_item_eth *eth_mask = item->mask;
+
+        VLOG_DBG("rte flow eth pattern:\n");
+        if (eth_spec) {
+            VLOG_DBG("  Spec: src="ETH_ADDR_FMT", dst="ETH_ADDR_FMT", "
+                     "type=0x%04" PRIx16"\n",
+                     ETH_ADDR_BYTES_ARGS(eth_spec->src.addr_bytes),
+                     ETH_ADDR_BYTES_ARGS(eth_spec->dst.addr_bytes),
+                     ntohs(eth_spec->type));
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+        if (eth_mask) {
+            VLOG_DBG("  Mask: src="ETH_ADDR_FMT", dst="ETH_ADDR_FMT", "
+                     "type=0x%04"PRIx16"\n",
+                     ETH_ADDR_BYTES_ARGS(eth_mask->src.addr_bytes),
+                     ETH_ADDR_BYTES_ARGS(eth_mask->dst.addr_bytes),
+                     eth_mask->type);
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+
+    if (item->type == RTE_FLOW_ITEM_TYPE_VLAN) {
+        const struct rte_flow_item_vlan *vlan_spec = item->spec;
+        const struct rte_flow_item_vlan *vlan_mask = item->mask;
+
+        VLOG_DBG("rte flow vlan pattern:\n");
+        if (vlan_spec) {
+            VLOG_DBG("  Spec: tpid=0x%"PRIx16", tci=0x%"PRIx16"\n",
+                     ntohs(vlan_spec->tpid), ntohs(vlan_spec->tci));
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+
+        if (vlan_mask) {
+            VLOG_DBG("  Mask: tpid=0x%"PRIx16", tci=0x%"PRIx16"\n",
+                     vlan_mask->tpid, vlan_mask->tci);
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+
+    if (item->type == RTE_FLOW_ITEM_TYPE_IPV4) {
+        const struct rte_flow_item_ipv4 *ipv4_spec = item->spec;
+        const struct rte_flow_item_ipv4 *ipv4_mask = item->mask;
+
+        VLOG_DBG("rte flow ipv4 pattern:\n");
+        if (ipv4_spec) {
+            VLOG_DBG("  Spec: tos=0x%"PRIx8", ttl=%"PRIx8", proto=0x%"PRIx8
+                     ", src="IP_FMT", dst="IP_FMT"\n",
+                     ipv4_spec->hdr.type_of_service,
+                     ipv4_spec->hdr.time_to_live,
+                     ipv4_spec->hdr.next_proto_id,
+                     IP_ARGS(ipv4_spec->hdr.src_addr),
+                     IP_ARGS(ipv4_spec->hdr.dst_addr));
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+        if (ipv4_mask) {
+            VLOG_DBG("  Mask: tos=0x%"PRIx8", ttl=%"PRIx8", proto=0x%"PRIx8
+                     ", src="IP_FMT", dst="IP_FMT"\n",
+                     ipv4_mask->hdr.type_of_service,
+                     ipv4_mask->hdr.time_to_live,
+                     ipv4_mask->hdr.next_proto_id,
+                     IP_ARGS(ipv4_mask->hdr.src_addr),
+                     IP_ARGS(ipv4_mask->hdr.dst_addr));
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+
+    if (item->type == RTE_FLOW_ITEM_TYPE_UDP) {
+        const struct rte_flow_item_udp *udp_spec = item->spec;
+        const struct rte_flow_item_udp *udp_mask = item->mask;
+
+        VLOG_DBG("rte flow udp pattern:\n");
+        if (udp_spec) {
+            VLOG_DBG("  Spec: src_port=%"PRIu16", dst_port=%"PRIu16"\n",
+                     ntohs(udp_spec->hdr.src_port),
+                     ntohs(udp_spec->hdr.dst_port));
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+        if (udp_mask) {
+            VLOG_DBG("  Mask: src_port=0x%"PRIx16", dst_port=0x%"PRIx16"\n",
+                     udp_mask->hdr.src_port,
+                     udp_mask->hdr.dst_port);
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+
+    if (item->type == RTE_FLOW_ITEM_TYPE_SCTP) {
+        const struct rte_flow_item_sctp *sctp_spec = item->spec;
+        const struct rte_flow_item_sctp *sctp_mask = item->mask;
+
+        VLOG_DBG("rte flow sctp pattern:\n");
+        if (sctp_spec) {
+            VLOG_DBG("  Spec: src_port=%"PRIu16", dst_port=%"PRIu16"\n",
+                     ntohs(sctp_spec->hdr.src_port),
+                     ntohs(sctp_spec->hdr.dst_port));
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+        if (sctp_mask) {
+            VLOG_DBG("  Mask: src_port=0x%"PRIx16", dst_port=0x%"PRIx16"\n",
+                     sctp_mask->hdr.src_port,
+                     sctp_mask->hdr.dst_port);
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+
+    if (item->type == RTE_FLOW_ITEM_TYPE_ICMP) {
+        const struct rte_flow_item_icmp *icmp_spec = item->spec;
+        const struct rte_flow_item_icmp *icmp_mask = item->mask;
+
+        VLOG_DBG("rte flow icmp pattern:\n");
+        if (icmp_spec) {
+            VLOG_DBG("  Spec: icmp_type=%"PRIu8", icmp_code=%"PRIu8"\n",
+                     icmp_spec->hdr.icmp_type,
+                     icmp_spec->hdr.icmp_code);
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+        if (icmp_mask) {
+            VLOG_DBG("  Mask: icmp_type=0x%"PRIx8", icmp_code=0x%"PRIx8"\n",
+                     icmp_spec->hdr.icmp_type,
+                     icmp_spec->hdr.icmp_code);
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+
+    if (item->type == RTE_FLOW_ITEM_TYPE_TCP) {
+        const struct rte_flow_item_tcp *tcp_spec = item->spec;
+        const struct rte_flow_item_tcp *tcp_mask = item->mask;
+
+        VLOG_DBG("rte flow tcp pattern:\n");
+        if (tcp_spec) {
+            VLOG_DBG("  Spec: src_port=%"PRIu16", dst_port=%"PRIu16
+                     ", data_off=0x%"PRIx8", tcp_flags=0x%"PRIx8"\n",
+                     ntohs(tcp_spec->hdr.src_port),
+                     ntohs(tcp_spec->hdr.dst_port),
+                     tcp_spec->hdr.data_off,
+                     tcp_spec->hdr.tcp_flags);
+        } else {
+            VLOG_DBG("  Spec = null\n");
+        }
+        if (tcp_mask) {
+            VLOG_DBG("  Mask: src_port=%"PRIx16", dst_port=%"PRIx16
+                     ", data_off=0x%"PRIx8", tcp_flags=0x%"PRIx8"\n",
+                     tcp_mask->hdr.src_port,
+                     tcp_mask->hdr.dst_port,
+                     tcp_mask->hdr.data_off,
+                     tcp_mask->hdr.tcp_flags);
+        } else {
+            VLOG_DBG("  Mask = null\n");
+        }
+    }
+}
+
+static void
+add_flow_pattern(struct flow_patterns *patterns, enum rte_flow_item_type type,
+                 const void *spec, const void *mask) {
+    int cnt = patterns->cnt;
+
+    if (cnt == 0) {
+        patterns->current_max = 8;
+        patterns->items = xcalloc(patterns->current_max,
+                                  sizeof(struct rte_flow_item));
+    } else if (cnt == patterns->current_max) {
+        patterns->current_max *= 2;
+        patterns->items = xrealloc(patterns->items, patterns->current_max *
+                                   sizeof(struct rte_flow_item));
+    }
+
+    patterns->items[cnt].type = type;
+    patterns->items[cnt].spec = spec;
+    patterns->items[cnt].mask = mask;
+    patterns->items[cnt].last = NULL;
+    dump_flow_pattern(&patterns->items[cnt]);
+    patterns->cnt++;
+}
+
+static void
+add_flow_action(struct flow_actions *actions, enum rte_flow_action_type type,
+                const void *conf)
+{
+    int cnt = actions->cnt;
+
+    if (cnt == 0) {
+        actions->current_max = 8;
+        actions->actions = xcalloc(actions->current_max,
+                                   sizeof(struct rte_flow_action));
+    } else if (cnt == actions->current_max) {
+        actions->current_max *= 2;
+        actions->actions = xrealloc(actions->actions, actions->current_max *
+                                    sizeof(struct rte_flow_action));
+    }
+
+    actions->actions[cnt].type = type;
+    actions->actions[cnt].conf = conf;
+    actions->cnt++;
+}
+
+static struct rte_flow_action_rss *
+add_flow_rss_action(struct flow_actions *actions,
+                    struct netdev *netdev) {
+    int i;
+    struct rte_flow_action_rss *rss;
+
+    rss = xmalloc(sizeof(*rss) + sizeof(uint16_t) * netdev->n_rxq);
+    /*
+     * Setting it to NULL will let the driver use the default RSS
+     * configuration we have set: &port_conf.rx_adv_conf.rss_conf.
+     */
+    rss->rss_conf = NULL;
+    rss->num = netdev->n_rxq;
+
+    for (i = 0; i < rss->num; i++) {
+        rss->queue[i] = i;
+    }
+
+    add_flow_action(actions, RTE_FLOW_ACTION_TYPE_RSS, rss);
+
+    return rss;
+}
+
+static int
+netdev_dpdk_add_rte_flow_offload(struct netdev *netdev,
+                                 const struct match *match,
+                                 struct nlattr *nl_actions OVS_UNUSED,
+                                 size_t actions_len OVS_UNUSED,
+                                 const ovs_u128 *ufid,
+                                 struct offload_info *info) {
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    const struct rte_flow_attr flow_attr = {
+        .group = 0,
+        .priority = 0,
+        .ingress = 1,
+        .egress = 0
+    };
+    struct flow_patterns patterns = { .items = NULL, .cnt = 0 };
+    struct flow_actions actions = { .actions = NULL, .cnt = 0 };
+    struct rte_flow *flow;
+    struct rte_flow_error error;
+    uint8_t *ipv4_next_proto_mask = NULL;
+    int ret = 0;
+
+    /* Eth */
+    struct rte_flow_item_eth eth_spec;
+    struct rte_flow_item_eth eth_mask;
+    memset(&eth_spec, 0, sizeof(eth_spec));
+    memset(&eth_mask, 0, sizeof(eth_mask));
+    if (!eth_addr_is_zero(match->wc.masks.dl_src) ||
+        !eth_addr_is_zero(match->wc.masks.dl_dst)) {
+        rte_memcpy(&eth_spec.dst, &match->flow.dl_dst, sizeof(eth_spec.dst));
+        rte_memcpy(&eth_spec.src, &match->flow.dl_src, sizeof(eth_spec.src));
+        eth_spec.type = match->flow.dl_type;
+
+        rte_memcpy(&eth_mask.dst, &match->wc.masks.dl_dst,
+                   sizeof(eth_mask.dst));
+        rte_memcpy(&eth_mask.src, &match->wc.masks.dl_src,
+                   sizeof(eth_mask.src));
+        eth_mask.type = match->wc.masks.dl_type;
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_ETH,
+                         &eth_spec, &eth_mask);
+    } else {
+        /*
+         * If user specifies a flow (like UDP flow) without L2 patterns,
+         * OVS will at least set the dl_type. Normally, it's enough to
+         * create an eth pattern just with it. Unluckily, some Intel's
+         * NIC (such as XL710) doesn't support that. Below is a workaround,
+         * which simply matches any L2 pkts.
+         */
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_ETH, NULL, NULL);
+    }
+
+    /* VLAN */
+    struct rte_flow_item_vlan vlan_spec;
+    struct rte_flow_item_vlan vlan_mask;
+    memset(&vlan_spec, 0, sizeof(vlan_spec));
+    memset(&vlan_mask, 0, sizeof(vlan_mask));
+    if (match->wc.masks.vlans[0].tci && match->flow.vlans[0].tci) {
+        vlan_spec.tci  = match->flow.vlans[0].tci & ~htons(VLAN_CFI);
+        vlan_mask.tci  = match->wc.masks.vlans[0].tci & ~htons(VLAN_CFI);
+
+        /* match any protocols */
+        vlan_mask.tpid = 0;
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_VLAN,
+                         &vlan_spec, &vlan_mask);
+    }
+
+    /* IP v4 */
+    uint8_t proto = 0;
+    struct rte_flow_item_ipv4 ipv4_spec;
+    struct rte_flow_item_ipv4 ipv4_mask;
+    memset(&ipv4_spec, 0, sizeof(ipv4_spec));
+    memset(&ipv4_mask, 0, sizeof(ipv4_mask));
+    if (match->flow.dl_type == htons(ETH_TYPE_IP)) {
+
+        ipv4_spec.hdr.type_of_service = match->flow.nw_tos;
+        ipv4_spec.hdr.time_to_live    = match->flow.nw_ttl;
+        ipv4_spec.hdr.next_proto_id   = match->flow.nw_proto;
+        ipv4_spec.hdr.src_addr        = match->flow.nw_src;
+        ipv4_spec.hdr.dst_addr        = match->flow.nw_dst;
+
+        ipv4_mask.hdr.type_of_service = match->wc.masks.nw_tos;
+        ipv4_mask.hdr.time_to_live    = match->wc.masks.nw_ttl;
+        ipv4_mask.hdr.next_proto_id   = match->wc.masks.nw_proto;
+        ipv4_mask.hdr.src_addr        = match->wc.masks.nw_src;
+        ipv4_mask.hdr.dst_addr        = match->wc.masks.nw_dst;
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_IPV4,
+                         &ipv4_spec, &ipv4_mask);
+
+        /* Save proto for L4 protocol setup */
+        proto = ipv4_spec.hdr.next_proto_id &
+                ipv4_mask.hdr.next_proto_id;
+
+        /* Remember proto mask address for later modification */
+        ipv4_next_proto_mask = &ipv4_mask.hdr.next_proto_id;
+    }
+
+    if (proto != IPPROTO_ICMP && proto != IPPROTO_UDP  &&
+        proto != IPPROTO_SCTP && proto != IPPROTO_TCP  &&
+        (match->wc.masks.tp_src ||
+         match->wc.masks.tp_dst ||
+         match->wc.masks.tcp_flags)) {
+        VLOG_DBG("L4 Protocol (%u) not supported", proto);
+        ret = -1;
+        goto out;
+    }
+
+    if ((match->wc.masks.tp_src && match->wc.masks.tp_src != OVS_BE16_MAX) ||
+        (match->wc.masks.tp_dst && match->wc.masks.tp_dst != OVS_BE16_MAX)) {
+        ret = -1;
+        goto out;
+    }
+
+    struct rte_flow_item_tcp tcp_spec;
+    struct rte_flow_item_tcp tcp_mask;
+    memset(&tcp_spec, 0, sizeof(tcp_spec));
+    memset(&tcp_mask, 0, sizeof(tcp_mask));
+    if (proto == IPPROTO_TCP) {
+        tcp_spec.hdr.src_port  = match->flow.tp_src;
+        tcp_spec.hdr.dst_port  = match->flow.tp_dst;
+        tcp_spec.hdr.data_off  = ntohs(match->flow.tcp_flags) >> 8;
+        tcp_spec.hdr.tcp_flags = ntohs(match->flow.tcp_flags) & 0xff;
+
+        tcp_mask.hdr.src_port  = match->wc.masks.tp_src;
+        tcp_mask.hdr.dst_port  = match->wc.masks.tp_dst;
+        tcp_mask.hdr.data_off  = ntohs(match->wc.masks.tcp_flags) >> 8;
+        tcp_mask.hdr.tcp_flags = ntohs(match->wc.masks.tcp_flags) & 0xff;
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_TCP,
+                         &tcp_spec, &tcp_mask);
+
+        /* proto == TCP and ITEM_TYPE_TCP, thus no need for proto match */
+        if (ipv4_next_proto_mask) {
+            *ipv4_next_proto_mask = 0;
+        }
+        goto end_proto_check;
+    }
+
+    struct rte_flow_item_udp udp_spec;
+    struct rte_flow_item_udp udp_mask;
+    memset(&udp_spec, 0, sizeof(udp_spec));
+    memset(&udp_mask, 0, sizeof(udp_mask));
+    if (proto == IPPROTO_UDP) {
+        udp_spec.hdr.src_port = match->flow.tp_src;
+        udp_spec.hdr.dst_port = match->flow.tp_dst;
+
+        udp_mask.hdr.src_port = match->wc.masks.tp_src;
+        udp_mask.hdr.dst_port = match->wc.masks.tp_dst;
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_UDP,
+                         &udp_spec, &udp_mask);
+
+        /* proto == UDP and ITEM_TYPE_UDP, thus no need for proto match */
+        if (ipv4_next_proto_mask) {
+            *ipv4_next_proto_mask = 0;
+        }
+        goto end_proto_check;
+    }
+
+    struct rte_flow_item_sctp sctp_spec;
+    struct rte_flow_item_sctp sctp_mask;
+    memset(&sctp_spec, 0, sizeof(sctp_spec));
+    memset(&sctp_mask, 0, sizeof(sctp_mask));
+    if (proto == IPPROTO_SCTP) {
+        sctp_spec.hdr.src_port = match->flow.tp_src;
+        sctp_spec.hdr.dst_port = match->flow.tp_dst;
+
+        sctp_mask.hdr.src_port = match->wc.masks.tp_src;
+        sctp_mask.hdr.dst_port = match->wc.masks.tp_dst;
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_SCTP,
+                         &sctp_spec, &sctp_mask);
+
+        /* proto == SCTP and ITEM_TYPE_SCTP, thus no need for proto match */
+        if (ipv4_next_proto_mask) {
+            *ipv4_next_proto_mask = 0;
+        }
+        goto end_proto_check;
+    }
+
+    struct rte_flow_item_icmp icmp_spec;
+    struct rte_flow_item_icmp icmp_mask;
+    memset(&icmp_spec, 0, sizeof(icmp_spec));
+    memset(&icmp_mask, 0, sizeof(icmp_mask));
+    if (proto == IPPROTO_ICMP) {
+        icmp_spec.hdr.icmp_type = (uint8_t)ntohs(match->flow.tp_src);
+        icmp_spec.hdr.icmp_code = (uint8_t)ntohs(match->flow.tp_dst);
+
+        icmp_mask.hdr.icmp_type = (uint8_t)ntohs(match->wc.masks.tp_src);
+        icmp_mask.hdr.icmp_code = (uint8_t)ntohs(match->wc.masks.tp_dst);
+
+        add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_ICMP,
+                         &icmp_spec, &icmp_mask);
+
+        /* proto == ICMP and ITEM_TYPE_ICMP, thus no need for proto match */
+        if (ipv4_next_proto_mask) {
+            *ipv4_next_proto_mask = 0;
+        }
+        goto end_proto_check;
+    }
+
+end_proto_check:
+
+    add_flow_pattern(&patterns, RTE_FLOW_ITEM_TYPE_END, NULL, NULL);
+
+    struct rte_flow_action_mark mark;
+    mark.id = info->flow_mark;
+    add_flow_action(&actions, RTE_FLOW_ACTION_TYPE_MARK, &mark);
+
+    struct rte_flow_action_rss *rss;
+    rss = add_flow_rss_action(&actions, netdev);
+    add_flow_action(&actions, RTE_FLOW_ACTION_TYPE_END, NULL);
+
+    flow = rte_flow_create(dev->port_id, &flow_attr, patterns.items,
+                           actions.actions, &error);
+    free(rss);
+    if (!flow) {
+        VLOG_ERR("rte flow creat error: %u : message : %s\n",
+                 error.type, error.message);
+        ret = -1;
+        goto out;
+    }
+    ufid_to_rte_flow_associate(ufid, flow);
+    VLOG_DBG("installed flow %p by ufid "UUID_FMT"\n",
+             flow, UUID_ARGS((struct uuid *)ufid));
+
+out:
+    free(patterns.items);
+    free(actions.actions);
+    return ret;
+}
+
+static bool
+is_all_zero(const void *addr, size_t n) {
+    size_t i = 0;
+    const uint8_t *p = (uint8_t *)addr;
+
+    for (i = 0; i < n; i++) {
+        if (p[i] != 0) {
+            return false;
+        }
+    }
+
+    return true;
+}
+
+/*
+ * Check if any unsupported flow patterns are specified.
+ */
+static int
+netdev_dpdk_validate_flow(const struct match *match) {
+    struct match match_zero_wc;
+
+    /* Create a wc-zeroed version of flow */
+    match_init(&match_zero_wc, &match->flow, &match->wc);
+
+    if (!is_all_zero(&match_zero_wc.flow.tunnel,
+                     sizeof(match_zero_wc.flow.tunnel))) {
+        goto err;
+    }
+
+    if (match->wc.masks.metadata ||
+        match->wc.masks.skb_priority ||
+        match->wc.masks.pkt_mark ||
+        match->wc.masks.dp_hash) {
+        goto err;
+    }
+
+    /* recirc id must be zero */
+    if (match_zero_wc.flow.recirc_id) {
+        goto err;
+    }
+
+    if (match->wc.masks.ct_state ||
+        match->wc.masks.ct_nw_proto ||
+        match->wc.masks.ct_zone ||
+        match->wc.masks.ct_mark ||
+        match->wc.masks.ct_label.u64.hi ||
+        match->wc.masks.ct_label.u64.lo) {
+        goto err;
+    }
+
+    if (match->wc.masks.conj_id ||
+        match->wc.masks.actset_output) {
+        goto err;
+    }
+
+    /* unsupported L2 */
+    if (!is_all_zero(&match->wc.masks.mpls_lse,
+                     sizeof(match_zero_wc.flow.mpls_lse))) {
+        goto err;
+    }
+
+    /* unsupported L3 */
+    if (match->wc.masks.ipv6_label ||
+        match->wc.masks.ct_nw_src ||
+        match->wc.masks.ct_nw_dst ||
+        !is_all_zero(&match->wc.masks.ipv6_src, sizeof(struct in6_addr)) ||
+        !is_all_zero(&match->wc.masks.ipv6_dst, sizeof(struct in6_addr)) ||
+        !is_all_zero(&match->wc.masks.ct_ipv6_src, sizeof(struct in6_addr)) ||
+        !is_all_zero(&match->wc.masks.ct_ipv6_dst, sizeof(struct in6_addr)) ||
+        !is_all_zero(&match->wc.masks.nd_target, sizeof(struct in6_addr)) ||
+        !is_all_zero(&match->wc.masks.nsh, sizeof(struct ovs_key_nsh)) ||
+        !is_all_zero(&match->wc.masks.arp_sha, sizeof(struct eth_addr)) ||
+        !is_all_zero(&match->wc.masks.arp_tha, sizeof(struct eth_addr))) {
+        goto err;
+    }
+
+    /* If fragmented, then don't HW accelerate - for now */
+    if (match_zero_wc.flow.nw_frag) {
+        goto err;
+    }
+
+    /* unsupported L4 */
+    if (match->wc.masks.igmp_group_ip4 ||
+        match->wc.masks.ct_tp_src ||
+        match->wc.masks.ct_tp_dst) {
+        goto err;
+    }
+
+    return 0;
+
+err:
+    VLOG_ERR("cannot HW accelerate this flow due to unsupported protocols");
+    return -1;
+}
+
+static int
+netdev_dpdk_destroy_rte_flow(struct netdev_dpdk *dev,
+                             const ovs_u128 *ufid,
+                             struct rte_flow *rte_flow) {
+    struct rte_flow_error error;
+    int ret;
+
+    ret = rte_flow_destroy(dev->port_id, rte_flow, &error);
+    if (ret == 0) {
+        ufid_to_rte_flow_disassociate(ufid);
+        VLOG_DBG("removed rte flow %p associated with ufid " UUID_FMT "\n",
+                 rte_flow, UUID_ARGS((struct uuid *)ufid));
+    } else {
+        VLOG_ERR("rte flow destroy error: %u : message : %s\n",
+                 error.type, error.message);
+    }
+
+    return ret;
+}
+
+static int
+netdev_dpdk_flow_put(struct netdev *netdev, struct match *match,
+                     struct nlattr *actions, size_t actions_len,
+                     const ovs_u128 *ufid, struct offload_info *info,
+                     struct dpif_flow_stats *stats OVS_UNUSED) {
+    struct rte_flow *rte_flow;
+    int ret;
+
+    /*
+     * If an old rte_flow exists, it means it's a flow modification.
+     * Here destroy the old rte flow first before adding a new one.
+     */
+    rte_flow = ufid_to_rte_flow_find(ufid);
+    if (rte_flow) {
+        ret = netdev_dpdk_destroy_rte_flow(netdev_dpdk_cast(netdev),
+                                           ufid, rte_flow);
+        if (ret < 0) {
+            return ret;
+        }
+    }
+
+    ret = netdev_dpdk_validate_flow(match);
+    if (ret < 0) {
+        return ret;
+    }
+
+    return netdev_dpdk_add_rte_flow_offload(netdev, match, actions,
+                                            actions_len, ufid, info);
+}
+
+static int
+netdev_dpdk_flow_del(struct netdev *netdev, const ovs_u128 *ufid,
+                     struct dpif_flow_stats *stats OVS_UNUSED) {
+
+    struct rte_flow *rte_flow = ufid_to_rte_flow_find(ufid);
+
+    if (!rte_flow) {
+        return -1;
+    }
+
+    return netdev_dpdk_destroy_rte_flow(netdev_dpdk_cast(netdev),
+                                        ufid, rte_flow);
+}
+
+#define NETDEV_DPDK_CLASS_COMMON                            \
+    .is_pmd = true,                                         \
+    .alloc = netdev_dpdk_alloc,                             \
+    .dealloc = netdev_dpdk_dealloc,                         \
+    .get_config = netdev_dpdk_get_config,                   \
+    .get_numa_id = netdev_dpdk_get_numa_id,                 \
+    .set_etheraddr = netdev_dpdk_set_etheraddr,             \
+    .get_etheraddr = netdev_dpdk_get_etheraddr,             \
+    .get_mtu = netdev_dpdk_get_mtu,                         \
+    .set_mtu = netdev_dpdk_set_mtu,                         \
+    .get_ifindex = netdev_dpdk_get_ifindex,                 \
+    .get_carrier_resets = netdev_dpdk_get_carrier_resets,   \
+    .set_miimon_interval = netdev_dpdk_set_miimon,          \
+    .set_policing = netdev_dpdk_set_policing,               \
+    .get_qos_types = netdev_dpdk_get_qos_types,             \
+    .get_qos = netdev_dpdk_get_qos,                         \
+    .set_qos = netdev_dpdk_set_qos,                         \
+    .update_flags = netdev_dpdk_update_flags,               \
+    .rxq_alloc = netdev_dpdk_rxq_alloc,                     \
+    .rxq_construct = netdev_dpdk_rxq_construct,             \
+    .rxq_destruct = netdev_dpdk_rxq_destruct,               \
+    .rxq_dealloc = netdev_dpdk_rxq_dealloc,                 \
+    DPDK_FLOW_OFFLOAD_API
+
+#define NETDEV_DPDK_CLASS_BASE                          \
+    NETDEV_DPDK_CLASS_COMMON,                           \
+    .init = netdev_dpdk_class_init,                     \
+    .destruct = netdev_dpdk_destruct,                   \
+    .set_tx_multiq = netdev_dpdk_set_tx_multiq,         \
+    .get_carrier = netdev_dpdk_get_carrier,             \
+    .get_stats = netdev_dpdk_get_stats,                 \
+    .get_custom_stats = netdev_dpdk_get_custom_stats,   \
+    .get_features = netdev_dpdk_get_features,           \
+    .get_status = netdev_dpdk_get_status,               \
+    .reconfigure = netdev_dpdk_reconfigure,             \
+    .rxq_recv = netdev_dpdk_rxq_recv
+
+static const struct netdev_class dpdk_class = {
+    .type = "dpdk",
+    NETDEV_DPDK_CLASS_BASE,
+    .construct = netdev_dpdk_construct,
+    .set_config = netdev_dpdk_set_config,
+    .send = netdev_dpdk_eth_send,
+};
+
+static const struct netdev_class dpdk_ring_class = {
+    .type = "dpdkr",
+    NETDEV_DPDK_CLASS_BASE,
+    .construct = netdev_dpdk_ring_construct,
+    .set_config = netdev_dpdk_ring_set_config,
+    .send = netdev_dpdk_ring_send,
+};
+
+static const struct netdev_class dpdk_vhost_class = {
+    .type = "dpdkvhostuser",
+    NETDEV_DPDK_CLASS_COMMON,
+    .construct = netdev_dpdk_vhost_construct,
+    .destruct = netdev_dpdk_vhost_destruct,
+    .send = netdev_dpdk_vhost_send,
+    .get_carrier = netdev_dpdk_vhost_get_carrier,
+    .get_stats = netdev_dpdk_vhost_get_stats,
+    .get_status = netdev_dpdk_vhost_user_get_status,
+    .reconfigure = netdev_dpdk_vhost_reconfigure,
+    .rxq_recv = netdev_dpdk_vhost_rxq_recv
+};
+
+static const struct netdev_class dpdk_vhost_client_class = {
+    .type = "dpdkvhostuserclient",
+    NETDEV_DPDK_CLASS_COMMON,
+    .construct = netdev_dpdk_vhost_client_construct,
+    .destruct = netdev_dpdk_vhost_destruct,
+    .set_config = netdev_dpdk_vhost_client_set_config,
+    .send = netdev_dpdk_vhost_send,
+    .get_carrier = netdev_dpdk_vhost_get_carrier,
+    .get_stats = netdev_dpdk_vhost_get_stats,
+    .get_status = netdev_dpdk_vhost_user_get_status,
+    .reconfigure = netdev_dpdk_vhost_client_reconfigure,
+    .rxq_recv = netdev_dpdk_vhost_rxq_recv
+};
 
 void
 netdev_dpdk_register(void)