]> 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 1aaf6f7e24703c62cc815f2fa6c285c7f19f6a67..f91aa27cd350852a528232679898ca30f8fe95bd 100644 (file)
@@ -26,6 +26,7 @@
 #include <sys/socket.h>
 #include <linux/if.h>
 
+#include <rte_bus_pci.h>
 #include <rte_config.h>
 #include <rte_cycles.h>
 #include <rte_errno.h>
 #include <rte_meter.h>
 #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"
@@ -49,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"
@@ -58,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};
@@ -89,7 +95,7 @@ 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
 
-/* Max and min number of packets in the mempool.  OVS tries to allocate a
+/* 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 */
@@ -99,11 +105,12 @@ static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
 #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);
+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))
+BUILD_ASSERT_DECL((MAX_NB_MBUF / ROUND_DOWN_POW2(MAX_NB_MBUF / MIN_NB_MBUF))
                   % MP_CACHE_SZ == 0);
 
 /*
@@ -150,8 +157,9 @@ BUILD_ASSERT_DECL((MAX_NB_MBUF / ROUND_DOWN_POW2(MAX_NB_MBUF/MIN_NB_MBUF))
 
 #define DPDK_ETH_PORT_ID_INVALID    RTE_MAX_ETHPORTS
 
-/* DPDK library uses uint8_t for port_id. */
-typedef uint8_t dpdk_port_t;
+/* 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)
@@ -177,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.
@@ -279,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);
+                   int pkt_cnt, bool should_steal);
 };
 
 /* dpdk_qos_ops for each type of user space QoS implementation */
@@ -303,16 +322,17 @@ 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);
-};
+     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. */
@@ -349,83 +369,126 @@ struct ingress_policer {
 
 enum dpdk_hw_ol_features {
     NETDEV_RX_CHECKSUM_OFFLOAD = 1 << 0,
+    NETDEV_RX_HW_CRC_STRIP = 1 << 1,
 };
 
-struct netdev_dpdk {
-    struct netdev up;
-    dpdk_port_t port_id;
-    int max_packet_len;
-    enum dpdk_dev_type type;
-
-    struct dpdk_tx_queue *tx_q;
-
-    struct ovs_mutex mutex OVS_ACQ_AFTER(dpdk_mutex);
-
-    struct dpdk_mp *dpdk_mp;
-    int mtu;
-    int socket_id;
-    int buf_size;
-    struct netdev_stats stats;
-    /* Protects stats */
-    rte_spinlock_t stats_lock;
-
-    struct eth_addr hwaddr;
-    enum netdev_flags flags;
-
-    struct rte_eth_link link;
-    int link_reset_cnt;
-
-    /* virtio identifier for vhost devices */
-    ovsrcu_index vid;
-
-    /* True if vHost device is 'up' and has been reconfigured at least once */
-    bool vhost_reconfigured;
-
-    /* Identifier used to distinguish vhost devices from each other. */
-    char vhost_id[PATH_MAX];
-
-    /* Device arguments for dpdk ports */
-    char *devargs;
-
-    /* If true, device was attached by rte_eth_dev_attach(). */
-    bool attached;
-
-    /* In dpdk_list. */
-    struct ovs_list list_node OVS_GUARDED_BY(dpdk_mutex);
-
-    /* QoS configuration and lock for the device */
-    OVSRCU_TYPE(struct qos_conf *) qos_conf;
-
-    /* The following properties cannot be changed when a device is running,
-     * so we remember the request and update them next time
-     * netdev_dpdk*_reconfigure() is called */
-    int requested_mtu;
-    int requested_n_txq;
-    int requested_n_rxq;
-    int requested_rxq_size;
-    int requested_txq_size;
-
-    /* Number of rx/tx descriptors for physical devices */
-    int rxq_size;
-    int txq_size;
-
-    /* Socket ID detected when vHost device is brought up */
-    int requested_socket_id;
-
-    /* Denotes whether vHost port is client/server mode */
-    uint64_t vhost_driver_flags;
-
-    /* Ingress Policer */
-    OVSRCU_TYPE(struct ingress_policer *) ingress_policer;
-    uint32_t policer_rate;
-    uint32_t policer_burst;
-
-    /* DPDK-ETH Flow control */
-    struct rte_eth_fc_conf fc_conf;
+/*
+ * In order to avoid confusion in variables names, following naming convention
+ * should be used, if possible:
+ *
+ *     'struct netdev'          : 'netdev'
+ *     'struct netdev_dpdk'     : 'dev'
+ *     'struct netdev_rxq'      : 'rxq'
+ *     'struct netdev_rxq_dpdk' : 'rx'
+ *
+ * Example:
+ *     struct netdev *netdev = netdev_from_name(name);
+ *     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+ *
+ *  Also, 'netdev' should be used instead of 'dev->up', where 'netdev' was
+ *  already defined.
+ */
 
-    /* DPDK-ETH hardware offload features,
-     * from the enum set 'dpdk_hw_ol_features' */
-    uint32_t hw_ol_features;
+struct netdev_dpdk {
+    PADDED_MEMBERS_CACHELINE_MARKER(CACHE_LINE_SIZE, cacheline0,
+        dpdk_port_t port_id;
+
+        /* 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;
+        int buf_size;
+        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;
+    );
+
+    PADDED_MEMBERS_CACHELINE_MARKER(CACHE_LINE_SIZE, cacheline1,
+        struct ovs_mutex mutex OVS_ACQ_AFTER(dpdk_mutex);
+        struct dpdk_mp *dpdk_mp;
+
+        /* virtio identifier for vhost devices */
+        ovsrcu_index vid;
+
+        /* True if vHost device is 'up' and has been reconfigured at least once */
+        bool vhost_reconfigured;
+        /* 3 pad bytes here. */
+    );
+
+    PADDED_MEMBERS(CACHE_LINE_SIZE,
+        /* Identifier used to distinguish vhost devices from each other. */
+        char vhost_id[PATH_MAX];
+    );
+
+    PADDED_MEMBERS(CACHE_LINE_SIZE,
+        struct netdev up;
+        /* In dpdk_list. */
+        struct ovs_list list_node OVS_GUARDED_BY(dpdk_mutex);
+
+        /* QoS configuration and lock for the device */
+        OVSRCU_TYPE(struct qos_conf *) qos_conf;
+
+        /* Ingress Policer */
+        OVSRCU_TYPE(struct ingress_policer *) ingress_policer;
+        uint32_t policer_rate;
+        uint32_t policer_burst;
+    );
+
+    PADDED_MEMBERS(CACHE_LINE_SIZE,
+        struct netdev_stats stats;
+        /* Protects stats */
+        rte_spinlock_t stats_lock;
+        /* 44 pad bytes here. */
+    );
+
+    PADDED_MEMBERS(CACHE_LINE_SIZE,
+        /* The following properties cannot be changed when a device is running,
+         * so we remember the request and update them next time
+         * netdev_dpdk*_reconfigure() is called */
+        int requested_mtu;
+        int requested_n_txq;
+        int requested_n_rxq;
+        int requested_rxq_size;
+        int requested_txq_size;
+
+        /* Number of rx/tx descriptors for physical devices */
+        int rxq_size;
+        int txq_size;
+
+        /* Socket ID detected when vHost device is brought up */
+        int requested_socket_id;
+
+        /* Denotes whether vHost port is client/server mode */
+        uint64_t vhost_driver_flags;
+
+        /* DPDK-ETH Flow control */
+        struct rte_eth_fc_conf fc_conf;
+
+        /* 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,
+        /* Names of all XSTATS counters */
+        struct rte_eth_xstat_name *rte_xstats_names;
+        int rte_xstats_names_size;
+        int rte_xstats_ids_size;
+        uint64_t *rte_xstats_ids;
+    );
 };
 
 struct netdev_rxq_dpdk {
@@ -436,6 +499,8 @@ struct netdev_rxq_dpdk {
 static void netdev_dpdk_destruct(struct netdev *netdev);
 static void netdev_dpdk_vhost_destruct(struct netdev *netdev);
 
+static void netdev_dpdk_clear_xstats(struct netdev_dpdk *dev);
+
 int netdev_dpdk_get_vid(const struct netdev_dpdk *dev);
 
 struct ingress_policer *
@@ -492,12 +557,90 @@ ovs_rte_pktmbuf_init(struct rte_mempool *mp OVS_UNUSED,
     dp_packet_init_dpdk((struct dp_packet *) pkt, pkt->buf_len);
 }
 
+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(int socket_id, int mtu)
+dpdk_mp_create(struct netdev_dpdk *dev, int mtu, bool per_port_mp)
 {
-    struct dpdk_mp *dmp;
-    unsigned mp_size;
-    char *mp_name;
+    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 dpdk_mp *dmp = NULL;
+    int ret;
 
     dmp = dpdk_rte_mzalloc(sizeof *dmp);
     if (!dmp) {
@@ -506,71 +649,129 @@ dpdk_mp_create(int socket_id, int mtu)
     dmp->socket_id = socket_id;
     dmp->mtu = mtu;
     dmp->refcount = 1;
-    /* 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) {
-        mp_size = MAX_NB_MBUF;
-    } else {
-        mp_size = MIN_NB_MBUF;
-    }
+
+    n_mbufs = dpdk_calculate_mbufs(dev, mtu, per_port_mp);
 
     do {
-        mp_name = xasprintf("ovs_mp_%d_%d_%u", dmp->mtu, dmp->socket_id,
-                            mp_size);
+        /* Full DPDK memory pool name must be unique and cannot be
+         * 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) {
+            VLOG_DBG("snprintf returned %d. "
+                     "Failed to generate a mempool name for \"%s\". "
+                     "Hash:0x%x, socket_id: %d, mtu:%d, mbufs:%u.",
+                     ret, netdev_name, hash, socket_id, mtu, n_mbufs);
+            break;
+        }
 
-        dmp->mp = rte_pktmbuf_pool_create(mp_name, mp_size,
+        VLOG_DBG("Port %s: Requesting a mempool of %u mbufs "
+                  "on socket %d for %d Rx and %d Tx queues.",
+                  netdev_name, n_mbufs, socket_id,
+                  dev->requested_n_rxq, dev->requested_n_txq);
+
+        dmp->mp = rte_pktmbuf_pool_create(mp_name, n_mbufs,
                                           MP_CACHE_SZ,
                                           sizeof (struct dp_packet)
-                                                 - sizeof (struct rte_mbuf),
+                                          - sizeof (struct rte_mbuf),
                                           MBUF_SIZE(mtu)
-                                                 - sizeof(struct dp_packet),
+                                          - sizeof(struct dp_packet),
                                           socket_id);
+
         if (dmp->mp) {
             VLOG_DBG("Allocated \"%s\" mempool with %u mbufs",
-                     mp_name, mp_size);
-        }
-        free(mp_name);
-        if (dmp->mp) {
+                     mp_name, n_mbufs);
             /* rte_pktmbuf_pool_create has done some initialization of the
              * 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. */
+            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, dmp->mp);
+            return dmp;
+        } else {
+            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 (rte_errno == ENOMEM && (mp_size /= 2) >= MIN_NB_MBUF);
+    } 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(int socket_id, int mtu)
+dpdk_mp_get(struct netdev_dpdk *dev, int mtu, bool per_port_mp)
 {
-    struct dpdk_mp *dmp;
+    struct dpdk_mp *dmp, *next;
+    bool reuse = false;
 
     ovs_mutex_lock(&dpdk_mp_mutex);
-    LIST_FOR_EACH (dmp, list_node, &dpdk_mp_list) {
-        if (dmp->socket_id == socket_id && dmp->mtu == mtu) {
-            dmp->refcount++;
-            goto out;
+    /* 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;
+            }
         }
     }
-
-    dmp = dpdk_mp_create(socket_id, mtu);
-    if (dmp) {
-        ovs_list_push_back(&dpdk_mp_list, &dmp->list_node);
+    /* 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);
+            }
+        }
     }
 
-out:
+
     ovs_mutex_unlock(&dpdk_mp_mutex);
 
     return dmp;
 }
 
+/* Decrement reference to a mempool. */
 static void
 dpdk_mp_put(struct dpdk_mp *dmp)
 {
@@ -580,42 +781,60 @@ dpdk_mp_put(struct dpdk_mp *dmp)
 
     ovs_mutex_lock(&dpdk_mp_mutex);
     ovs_assert(dmp->refcount);
-
-    if (!--dmp->refcount) {
-        ovs_list_remove(&dmp->list_node);
-        rte_mempool_free(dmp->mp);
-        rte_free(dmp);
-    }
+    dmp->refcount--;
     ovs_mutex_unlock(&dpdk_mp_mutex);
 }
 
-/* Tries to allocate new mempool on requested_socket_id with
- * mbuf size corresponding to requested_mtu.
- * On success 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 dpdk_mp *mp;
+    struct dpdk_mp *dmp;
+    int ret = 0;
+    bool per_port_mp = dpdk_per_port_memory();
+
+    /* 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;
+    }
 
-    mp = dpdk_mp_get(dev->requested_socket_id, FRAME_LEN_TO_MTU(buf_size));
-    if (!mp) {
+    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));
-        return rte_errno;
+        ret = rte_errno;
     } else {
-        dpdk_mp_put(dev->dpdk_mp);
-        dev->dpdk_mp = mp;
+        /* 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->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);
     }
 
-    return 0;
+    return ret;
 }
 
 static void
@@ -631,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);
         }
     }
 }
@@ -665,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
@@ -698,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;
             }
@@ -724,7 +970,7 @@ dpdk_eth_dev_queue_setup(struct netdev_dpdk *dev, int n_rxq, int n_txq)
                                           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;
             }
@@ -749,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);
     }
 }
@@ -769,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;
@@ -781,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;
     }
 
@@ -794,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->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;
 }
 
@@ -877,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;
@@ -906,6 +1156,12 @@ common_construct(struct netdev *netdev, dpdk_port_t port_no,
 
     netdev_request_reconfigure(netdev);
 
+    dev->rte_xstats_names = NULL;
+    dev->rte_xstats_names_size = 0;
+
+    dev->rte_xstats_ids = NULL;
+    dev->rte_xstats_ids_size = 0;
+
     return 0;
 }
 
@@ -1070,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);
@@ -1080,6 +1337,7 @@ netdev_dpdk_destruct(struct netdev *netdev)
         }
     }
 
+    netdev_dpdk_clear_xstats(dev);
     free(dev->devargs);
     common_destruct(dev);
 
@@ -1144,6 +1402,129 @@ netdev_dpdk_dealloc(struct netdev *netdev)
     rte_free(dev);
 }
 
+static void
+netdev_dpdk_clear_xstats(struct netdev_dpdk *dev)
+{
+    /* If statistics are already allocated, we have to
+     * reconfigure, as port_id could have been changed. */
+    if (dev->rte_xstats_names) {
+        free(dev->rte_xstats_names);
+        dev->rte_xstats_names = NULL;
+        dev->rte_xstats_names_size = 0;
+    }
+    if (dev->rte_xstats_ids) {
+        free(dev->rte_xstats_ids);
+        dev->rte_xstats_ids = NULL;
+        dev->rte_xstats_ids_size = 0;
+    }
+}
+
+static const char*
+netdev_dpdk_get_xstat_name(struct netdev_dpdk *dev, uint64_t id)
+{
+    if (id >= dev->rte_xstats_names_size) {
+        return "UNKNOWN";
+    }
+    return dev->rte_xstats_names[id].name;
+}
+
+static bool
+netdev_dpdk_configure_xstats(struct netdev_dpdk *dev)
+    OVS_REQUIRES(dev->mutex)
+{
+    int rte_xstats_len;
+    bool ret;
+    struct rte_eth_xstat *rte_xstats;
+    uint64_t id;
+    int xstats_no;
+    const char *name;
+
+    /* Retrieving all XSTATS names. If something will go wrong
+     * or amount of counters will be equal 0, rte_xstats_names
+     * buffer will be marked as NULL, and any further xstats
+     * query won't be performed (e.g. during netdev_dpdk_get_stats
+     * execution). */
+
+    ret = false;
+    rte_xstats = NULL;
+
+    if (dev->rte_xstats_names == NULL || dev->rte_xstats_ids == NULL) {
+        dev->rte_xstats_names_size =
+                rte_eth_xstats_get_names(dev->port_id, NULL, 0);
+
+        if (dev->rte_xstats_names_size < 0) {
+            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 */
+            dev->rte_xstats_names = xcalloc(dev->rte_xstats_names_size,
+                                            sizeof *dev->rte_xstats_names);
+
+            if (dev->rte_xstats_names) {
+                /* Retreive xstats names */
+                rte_xstats_len =
+                        rte_eth_xstats_get_names(dev->port_id,
+                                                 dev->rte_xstats_names,
+                                                 dev->rte_xstats_names_size);
+
+                if (rte_xstats_len < 0) {
+                    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: "
+                              DPDK_PORT_ID_FMT, dev->port_id);
+                    goto out;
+                }
+
+                dev->rte_xstats_ids = xcalloc(dev->rte_xstats_names_size,
+                                              sizeof(uint64_t));
+
+                /* We have to calculate number of counters */
+                rte_xstats = xmalloc(rte_xstats_len * sizeof *rte_xstats);
+                memset(rte_xstats, 0xff, sizeof *rte_xstats * rte_xstats_len);
+
+                /* Retreive xstats values */
+                if (rte_eth_xstats_get(dev->port_id, rte_xstats,
+                                       rte_xstats_len) > 0) {
+                    dev->rte_xstats_ids_size = 0;
+                    xstats_no = 0;
+                    for (uint32_t i = 0; i < rte_xstats_len; i++) {
+                        id = rte_xstats[i].id;
+                        name = netdev_dpdk_get_xstat_name(dev, id);
+                        /* We need to filter out everything except
+                         * dropped, error and management counters */
+                        if (string_ends_with(name, "_errors") ||
+                            strstr(name, "_management_") ||
+                            string_ends_with(name, "_dropped")) {
+
+                            dev->rte_xstats_ids[xstats_no] = id;
+                            xstats_no++;
+                        }
+                    }
+                    dev->rte_xstats_ids_size = xstats_no;
+                    ret = true;
+                } else {
+                    VLOG_WARN("Can't get XSTATS IDs for port: "
+                              DPDK_PORT_ID_FMT, dev->port_id);
+                }
+
+                free(rte_xstats);
+            }
+        }
+    } else {
+        /* Already configured */
+        ret = true;
+    }
+
+out:
+    if (!ret) {
+        netdev_dpdk_clear_xstats(dev);
+    }
+    return ret;
+}
+
 static int
 netdev_dpdk_get_config(const struct netdev *netdev, struct smap *args)
 {
@@ -1171,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);
 
@@ -1192,31 +1575,70 @@ netdev_dpdk_lookup_by_port_id(dpdk_port_t port_id)
     return NULL;
 }
 
+static dpdk_port_t
+netdev_dpdk_get_port_by_mac(const char *mac_str)
+{
+    dpdk_port_t port_id;
+    struct eth_addr mac, port_mac;
+
+    if (!eth_addr_from_string(mac_str, &mac)) {
+        VLOG_ERR("invalid mac: %s", mac_str);
+        return DPDK_ETH_PORT_ID_INVALID;
+    }
+
+    RTE_ETH_FOREACH_DEV (port_id) {
+        struct ether_addr ea;
+
+        rte_eth_macaddr_get(port_id, &ea);
+        memcpy(port_mac.ea, ea.addr_bytes, ETH_ADDR_LEN);
+        if (eth_addr_equals(mac, port_mac)) {
+            return port_id;
+        }
+    }
+
+    return DPDK_ETH_PORT_ID_INVALID;
+}
+
+/*
+ * Normally, a PCI id is enough for identifying a specific DPDK port.
+ * However, for some NICs having multiple ports sharing the same PCI
+ * id, using PCI id won't work then.
+ *
+ * To fix that, here one more method is introduced: "class=eth,mac=$MAC".
+ *
+ * Note that the compatibility is fully kept: user can still use the
+ * PCI id for adding ports (when it's enough for them).
+ */
 static dpdk_port_t
 netdev_dpdk_process_devargs(struct netdev_dpdk *dev,
                             const char *devargs, char **errp)
 {
-    /* Get the name up to the first comma. */
-    char *name = xmemdup0(devargs, strcspn(devargs, ","));
+    char *name;
     dpdk_port_t new_port_id = DPDK_ETH_PORT_ID_INVALID;
 
-    if (!rte_eth_dev_count()
-            || rte_eth_dev_get_port_by_name(name, &new_port_id)
-            || !rte_eth_dev_is_valid_port(new_port_id)) {
-        /* Device not found in DPDK, attempt to attach it */
-        if (!rte_eth_dev_attach(devargs, &new_port_id)) {
-            /* Attach successful */
-            dev->attached = true;
-            VLOG_INFO("Device '%s' attached to DPDK", devargs);
-        } else {
-            /* Attach unsuccessful */
-            new_port_id = DPDK_ETH_PORT_ID_INVALID;
-            VLOG_WARN_BUF(errp, "Error attaching device '%s' to DPDK",
-                          devargs);
+    if (strncmp(devargs, "class=eth,mac=", 14) == 0) {
+        new_port_id = netdev_dpdk_get_port_by_mac(&devargs[14]);
+    } else {
+        name = xmemdup0(devargs, strcspn(devargs, ","));
+        if (rte_eth_dev_get_port_by_name(name, &new_port_id)
+                || !rte_eth_dev_is_valid_port(new_port_id)) {
+            /* Device not found in DPDK, attempt to attach it */
+            if (!rte_eth_dev_attach(devargs, &new_port_id)) {
+                /* Attach successful */
+                dev->attached = true;
+                VLOG_INFO("Device '%s' attached to DPDK", devargs);
+            } else {
+                /* Attach unsuccessful */
+                new_port_id = DPDK_ETH_PORT_ID_INVALID;
+            }
         }
+        free(name);
+    }
+
+    if (new_port_id == DPDK_ETH_PORT_ID_INVALID) {
+        VLOG_WARN_BUF(errp, "Error attaching device '%s' to DPDK", devargs);
     }
 
-    free(name);
     return new_port_id;
 }
 
@@ -1255,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},
@@ -1316,6 +1738,7 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
                     dev->devargs = xstrdup(new_devargs);
                     dev->port_id = new_port_id;
                     netdev_request_reconfigure(&dev->up);
+                    netdev_dpdk_clear_xstats(dev);
                     err = 0;
                 }
             }
@@ -1331,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);
@@ -1339,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);
     }
 
@@ -1375,6 +1810,12 @@ netdev_dpdk_vhost_client_set_config(struct netdev *netdev,
         path = smap_get(args, "vhost-server-path");
         if (path && strcmp(path, dev->vhost_id)) {
             strcpy(dev->vhost_id, path);
+            /* check zero copy configuration */
+            if (smap_get_bool(args, "dq-zero-copy", false)) {
+                dev->vhost_driver_flags |= RTE_VHOST_USER_DEQUEUE_ZERO_COPY;
+            } else {
+                dev->vhost_driver_flags &= ~RTE_VHOST_USER_DEQUEUE_ZERO_COPY;
+            }
             netdev_request_reconfigure(netdev);
         }
     }
@@ -1501,7 +1942,8 @@ 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)
+                        struct rte_mbuf **pkts, int pkt_cnt,
+                        bool should_steal)
 {
     int i = 0;
     int cnt = 0;
@@ -1517,7 +1959,9 @@ netdev_dpdk_policer_run(struct rte_meter_srtcm *meter,
             }
             cnt++;
         } else {
-            rte_pktmbuf_free(pkt);
+            if (should_steal) {
+                rte_pktmbuf_free(pkt);
+            }
         }
     }
 
@@ -1526,12 +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)
+                    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);
+    cnt = netdev_dpdk_policer_run(&policer->in_policer, pkts,
+                                  pkt_cnt, should_steal);
     rte_spinlock_unlock(&policer->policer_lock);
 
     return cnt;
@@ -1609,33 +2054,42 @@ 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);
-    int qid = rxq->queue_id;
     struct ingress_policer *policer = netdev_dpdk_get_ingress_policer(dev);
     uint16_t nb_rx = 0;
     uint16_t dropped = 0;
+    int qid = rxq->queue_id * VIRTIO_QNUM + VIRTIO_TXQ;
+    int vid = netdev_dpdk_get_vid(dev);
 
-    if (OVS_UNLIKELY(!is_vhost_running(dev)
+    if (OVS_UNLIKELY(vid < 0 || !dev->vhost_reconfigured
                      || !(dev->flags & NETDEV_UP))) {
         return EAGAIN;
     }
 
-    nb_rx = rte_vhost_dequeue_burst(netdev_dpdk_get_vid(dev),
-                                    qid * VIRTIO_QNUM + VIRTIO_TXQ,
-                                    dev->dpdk_mp->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,
                                     (struct rte_mbuf **) batch->packets,
-                                    nb_rx);
+                                    nb_rx, true);
         dropped -= nb_rx;
     }
 
@@ -1644,12 +2098,15 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
                                          nb_rx, dropped);
     rte_spinlock_unlock(&dev->stats_lock);
 
-    batch->count = (int) nb_rx;
+    batch->count = nb_rx;
+    dp_packet_batch_init_packet_fields(batch);
+
     return 0;
 }
 
 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);
@@ -1672,7 +2129,7 @@ netdev_dpdk_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch)
         dropped = nb_rx;
         nb_rx = ingress_policer_run(policer,
                                     (struct rte_mbuf **) batch->packets,
-                                    nb_rx);
+                                    nb_rx, true);
         dropped -= nb_rx;
     }
 
@@ -1684,19 +2141,28 @@ 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)
+                    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);
+        cnt = qos_conf->ops->qos_run(qos_conf, pkts, cnt, should_steal);
         rte_spinlock_unlock(&qos_conf->lock);
     }
 
@@ -1755,10 +2221,11 @@ __netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
     unsigned int total_pkts = cnt;
     unsigned int dropped = 0;
     int i, retries = 0;
+    int vid = netdev_dpdk_get_vid(dev);
 
     qid = dev->tx_q[qid % netdev->n_txq].map;
 
-    if (OVS_UNLIKELY(!is_vhost_running(dev) || qid < 0
+    if (OVS_UNLIKELY(vid < 0 || !dev->vhost_reconfigured || qid < 0
                      || !(dev->flags & NETDEV_UP))) {
         rte_spinlock_lock(&dev->stats_lock);
         dev->stats.tx_dropped+= cnt;
@@ -1770,15 +2237,14 @@ __netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
 
     cnt = netdev_dpdk_filter_packet_len(dev, cur_pkts, cnt);
     /* Check has QoS has been configured for the netdev */
-    cnt = netdev_dpdk_qos_run(dev, cur_pkts, cnt);
+    cnt = netdev_dpdk_qos_run(dev, cur_pkts, cnt, true);
     dropped = total_pkts - cnt;
 
     do {
         int vhost_qid = qid * VIRTIO_QNUM + VIRTIO_RXQ;
         unsigned int tx_pkts;
 
-        tx_pkts = rte_vhost_enqueue_burst(netdev_dpdk_get_vid(dev),
-                                          vhost_qid, cur_pkts, cnt);
+        tx_pkts = rte_vhost_enqueue_burst(vid, vhost_qid, cur_pkts, cnt);
         if (OVS_LIKELY(tx_pkts)) {
             /* Packets have been sent.*/
             cnt -= tx_pkts;
@@ -1808,59 +2274,60 @@ static void
 dpdk_do_tx_copy(struct netdev *netdev, int qid, struct dp_packet_batch *batch)
     OVS_NO_THREAD_SAFETY_ANALYSIS
 {
+    const size_t batch_cnt = dp_packet_batch_size(batch);
 #if !defined(__CHECKER__) && !defined(_WIN32)
-    const size_t PKT_ARRAY_SIZE = batch->count;
+    const size_t PKT_ARRAY_SIZE = batch_cnt;
 #else
     /* Sparse or MSVC doesn't like variable length array. */
     enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST };
 #endif
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
     struct rte_mbuf *pkts[PKT_ARRAY_SIZE];
-    int dropped = 0;
-    int newcnt = 0;
-    int i;
+    uint32_t cnt = batch_cnt;
+    uint32_t dropped = 0;
+
+    if (dev->type != DPDK_DEV_VHOST) {
+        /* Check if QoS has been configured for this netdev. */
+        cnt = netdev_dpdk_qos_run(dev, (struct rte_mbuf **) batch->packets,
+                                  batch_cnt, false);
+        dropped += batch_cnt - cnt;
+    }
 
-    dp_packet_batch_apply_cutlen(batch);
+    uint32_t txcnt = 0;
 
-    for (i = 0; i < batch->count; i++) {
-        int size = dp_packet_size(batch->packets[i]);
+    for (uint32_t i = 0; i < cnt; i++) {
+        struct dp_packet *packet = batch->packets[i];
+        uint32_t size = dp_packet_size(packet);
 
         if (OVS_UNLIKELY(size > dev->max_packet_len)) {
-            VLOG_WARN_RL(&rl, "Too big size %d max_packet_len %d",
-                         (int) size, dev->max_packet_len);
+            VLOG_WARN_RL(&rl, "Too big size %u max_packet_len %d",
+                         size, dev->max_packet_len);
 
             dropped++;
             continue;
         }
 
-        pkts[newcnt] = rte_pktmbuf_alloc(dev->dpdk_mp->mp);
-
-        if (!pkts[newcnt]) {
-            dropped += batch->count - i;
+        pkts[txcnt] = rte_pktmbuf_alloc(dev->dpdk_mp->mp);
+        if (OVS_UNLIKELY(!pkts[txcnt])) {
+            dropped += cnt - i;
             break;
         }
 
         /* We have to do a copy for now */
-        memcpy(rte_pktmbuf_mtod(pkts[newcnt], void *),
-               dp_packet_data(batch->packets[i]), size);
+        memcpy(rte_pktmbuf_mtod(pkts[txcnt], void *),
+               dp_packet_data(packet), size);
+        dp_packet_set_size((struct dp_packet *)pkts[txcnt], size);
 
-        rte_pktmbuf_data_len(pkts[newcnt]) = size;
-        rte_pktmbuf_pkt_len(pkts[newcnt]) = size;
-
-        newcnt++;
+        txcnt++;
     }
 
-    if (dev->type == DPDK_DEV_VHOST) {
-        __netdev_dpdk_vhost_send(netdev, qid, (struct dp_packet **) pkts,
-                                 newcnt);
-    } else {
-        unsigned int qos_pkts = newcnt;
-
-        /* Check if QoS has been configured for this netdev. */
-        newcnt = netdev_dpdk_qos_run(dev, pkts, newcnt);
-
-        dropped += qos_pkts - newcnt;
-        dropped += netdev_dpdk_eth_tx_burst(dev, qid, pkts, newcnt);
+    if (OVS_LIKELY(txcnt)) {
+        if (dev->type == DPDK_DEV_VHOST) {
+            __netdev_dpdk_vhost_send(netdev, qid, (struct dp_packet **) pkts,
+                                     txcnt);
+        } else {
+            dropped += netdev_dpdk_eth_tx_burst(dev, qid, pkts, txcnt);
+        }
     }
 
     if (OVS_UNLIKELY(dropped)) {
@@ -1873,14 +2340,13 @@ dpdk_do_tx_copy(struct netdev *netdev, int qid, struct dp_packet_batch *batch)
 static int
 netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
                        struct dp_packet_batch *batch,
-                       bool may_steal, bool concurrent_txq OVS_UNUSED)
+                       bool concurrent_txq OVS_UNUSED)
 {
 
-    if (OVS_UNLIKELY(!may_steal || batch->packets[0]->source != DPBUF_DPDK)) {
+    if (OVS_UNLIKELY(batch->packets[0]->source != DPBUF_DPDK)) {
         dpdk_do_tx_copy(netdev, qid, batch);
-        dp_packet_delete_batch(batch, may_steal);
+        dp_packet_delete_batch(batch, true);
     } else {
-        dp_packet_batch_apply_cutlen(batch);
         __netdev_dpdk_vhost_send(netdev, qid, batch->packets, batch->count);
     }
     return 0;
@@ -1888,11 +2354,11 @@ netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
 
 static inline void
 netdev_dpdk_send__(struct netdev_dpdk *dev, int qid,
-                   struct dp_packet_batch *batch, bool may_steal,
+                   struct dp_packet_batch *batch,
                    bool concurrent_txq)
 {
     if (OVS_UNLIKELY(!(dev->flags & NETDEV_UP))) {
-        dp_packet_delete_batch(batch, may_steal);
+        dp_packet_delete_batch(batch, true);
         return;
     }
 
@@ -1901,24 +2367,21 @@ netdev_dpdk_send__(struct netdev_dpdk *dev, int qid,
         rte_spinlock_lock(&dev->tx_q[qid].tx_lock);
     }
 
-    if (OVS_UNLIKELY(!may_steal ||
-                     batch->packets[0]->source != DPBUF_DPDK)) {
+    if (OVS_UNLIKELY(batch->packets[0]->source != DPBUF_DPDK)) {
         struct netdev *netdev = &dev->up;
 
         dpdk_do_tx_copy(netdev, qid, batch);
-        dp_packet_delete_batch(batch, may_steal);
+        dp_packet_delete_batch(batch, true);
     } else {
-        int dropped;
-        int cnt = batch->count;
+        int tx_cnt, dropped;
+        int batch_cnt = dp_packet_batch_size(batch);
         struct rte_mbuf **pkts = (struct rte_mbuf **) batch->packets;
 
-        dp_packet_batch_apply_cutlen(batch);
-
-        cnt = netdev_dpdk_filter_packet_len(dev, pkts, cnt);
-        cnt = netdev_dpdk_qos_run(dev, pkts, cnt);
-        dropped = batch->count - cnt;
+        tx_cnt = netdev_dpdk_filter_packet_len(dev, pkts, batch_cnt);
+        tx_cnt = netdev_dpdk_qos_run(dev, pkts, tx_cnt, true);
+        dropped = batch_cnt - tx_cnt;
 
-        dropped += netdev_dpdk_eth_tx_burst(dev, qid, pkts, cnt);
+        dropped += netdev_dpdk_eth_tx_burst(dev, qid, pkts, tx_cnt);
 
         if (OVS_UNLIKELY(dropped)) {
             rte_spinlock_lock(&dev->stats_lock);
@@ -1934,12 +2397,11 @@ netdev_dpdk_send__(struct netdev_dpdk *dev, int qid,
 
 static int
 netdev_dpdk_eth_send(struct netdev *netdev, int qid,
-                     struct dp_packet_batch *batch, bool may_steal,
-                     bool concurrent_txq)
+                     struct dp_packet_batch *batch, bool concurrent_txq)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
 
-    netdev_dpdk_send__(dev, qid, batch, may_steal, concurrent_txq);
+    netdev_dpdk_send__(dev, qid, batch, concurrent_txq);
     return 0;
 }
 
@@ -1987,7 +2449,18 @@ netdev_dpdk_set_mtu(struct netdev *netdev, int mtu)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
 
-    if (MTU_TO_FRAME_LEN(mtu) > NETDEV_DPDK_MAX_PKT_LEN
+    /* XXX: Ensure that the overall frame length of the requested MTU does not
+     * surpass the NETDEV_DPDK_MAX_PKT_LEN. DPDK device drivers differ in how
+     * the L2 frame length is calculated for a given MTU when
+     * rte_eth_dev_set_mtu(mtu) is called e.g. i40e driver includes 2 x vlan
+     * headers, the em driver includes 1 x vlan header, the ixgbe driver does
+     * not include vlan headers. As such we should use
+     * MTU_TO_MAX_FRAME_LEN(mtu) which includes an additional 2 x vlan headers
+     * (8 bytes) for comparison. This avoids a failure later with
+     * rte_eth_dev_set_mtu(). This approach should be used until DPDK provides
+     * a method to retrieve the upper bound MTU for a given device.
+     */
+    if (MTU_TO_MAX_FRAME_LEN(mtu) > NETDEV_DPDK_MAX_PKT_LEN
         || mtu < ETHER_MIN_MTU) {
         VLOG_WARN("%s: unsupported MTU %d\n", dev->up.name, mtu);
         return EINVAL;
@@ -2016,10 +2489,10 @@ netdev_dpdk_vhost_get_stats(const struct netdev *netdev,
 
     rte_spinlock_lock(&dev->stats_lock);
     /* Supported Stats */
-    stats->rx_packets += dev->stats.rx_packets;
-    stats->tx_packets += dev->stats.tx_packets;
+    stats->rx_packets = dev->stats.rx_packets;
+    stats->tx_packets = dev->stats.tx_packets;
     stats->rx_dropped = dev->stats.rx_dropped;
-    stats->tx_dropped += dev->stats.tx_dropped;
+    stats->tx_dropped = dev->stats.tx_dropped;
     stats->multicast = dev->stats.multicast;
     stats->rx_bytes = dev->stats.rx_bytes;
     stats->tx_bytes = dev->stats.tx_bytes;
@@ -2109,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;
     }
@@ -2117,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 */
@@ -2129,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 */
@@ -2140,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:
@@ -2169,6 +2646,58 @@ out:
     return 0;
 }
 
+static int
+netdev_dpdk_get_custom_stats(const struct netdev *netdev,
+                             struct netdev_custom_stats *custom_stats)
+{
+
+    uint32_t i;
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    int rte_xstats_ret;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    if (netdev_dpdk_configure_xstats(dev)) {
+        uint64_t *values = xcalloc(dev->rte_xstats_ids_size,
+                                   sizeof(uint64_t));
+
+        rte_xstats_ret =
+                rte_eth_xstats_get_by_id(dev->port_id, dev->rte_xstats_ids,
+                                         values, dev->rte_xstats_ids_size);
+
+        if (rte_xstats_ret > 0 &&
+            rte_xstats_ret <= dev->rte_xstats_ids_size) {
+
+            custom_stats->size = rte_xstats_ret;
+            custom_stats->counters =
+                    (struct netdev_custom_counter *) xcalloc(rte_xstats_ret,
+                            sizeof(struct netdev_custom_counter));
+
+            for (i = 0; i < rte_xstats_ret; i++) {
+                ovs_strlcpy(custom_stats->counters[i].name,
+                            netdev_dpdk_get_xstat_name(dev,
+                                                       dev->rte_xstats_ids[i]),
+                            NETDEV_CUSTOM_STATS_NAME_SIZE);
+                custom_stats->counters[i].value = values[i];
+            }
+        } else {
+            VLOG_WARN("Cannot get XSTATS values for port: "DPDK_PORT_ID_FMT,
+                      dev->port_id);
+            custom_stats->counters = NULL;
+            custom_stats->size = 0;
+            /* Let's clear statistics cache, so it will be
+             * reconfigured */
+            netdev_dpdk_clear_xstats(dev);
+        }
+
+        free(values);
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return 0;
+}
+
 static int
 netdev_dpdk_get_features(const struct netdev *netdev,
                          enum netdev_features *current,
@@ -2206,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) {
@@ -2229,8 +2761,8 @@ netdev_dpdk_policer_construct(uint32_t rate, uint32_t burst)
     rte_spinlock_init(&policer->policer_lock);
 
     /* rte_meter requires bytes so convert kbits rate and burst to bytes. */
-    rate_bytes = rate * 1000/8;
-    burst_bytes = burst * 1000/8;
+    rate_bytes = rate * 1000ULL / 8;
+    burst_bytes = burst * 1000ULL / 8;
 
     policer->app_srtcm_params.cir = rate_bytes;
     policer->app_srtcm_params.cbs = burst_bytes;
@@ -2239,6 +2771,7 @@ netdev_dpdk_policer_construct(uint32_t rate, uint32_t burst)
                                     &policer->app_srtcm_params);
     if (err) {
         VLOG_ERR("Could not create rte meter for ingress policer");
+        free(policer);
         return NULL;
     }
 
@@ -2418,38 +2951,103 @@ netdev_dpdk_update_flags(struct netdev *netdev,
 }
 
 static int
-netdev_dpdk_get_status(const struct netdev *netdev, struct smap *args)
+netdev_dpdk_vhost_user_get_status(const struct netdev *netdev,
+                                  struct smap *args)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-    struct rte_eth_dev_info dev_info;
-
-    if (!rte_eth_dev_is_valid_port(dev->port_id)) {
-        return ENODEV;
-    }
 
     ovs_mutex_lock(&dev->mutex);
-    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, "numa_id", "%d",
-                           rte_eth_dev_socket_id(dev->port_id));
-    smap_add_format(args, "driver_name", "%s", dev_info.driver_name);
-    smap_add_format(args, "min_rx_bufsize", "%u", dev_info.min_rx_bufsize);
-    smap_add_format(args, "max_rx_pktlen", "%u", dev->max_packet_len);
-    smap_add_format(args, "max_rx_queues", "%u", dev_info.max_rx_queues);
-    smap_add_format(args, "max_tx_queues", "%u", dev_info.max_tx_queues);
-    smap_add_format(args, "max_mac_addrs", "%u", dev_info.max_mac_addrs);
-    smap_add_format(args, "max_hash_mac_addrs", "%u",
-                           dev_info.max_hash_mac_addrs);
-    smap_add_format(args, "max_vfs", "%u", dev_info.max_vfs);
-    smap_add_format(args, "max_vmdq_pools", "%u", dev_info.max_vmdq_pools);
+    bool client_mode = dev->vhost_driver_flags & RTE_VHOST_USER_CLIENT;
+    smap_add_format(args, "mode", "%s", client_mode ? "client" : "server");
 
-    if (dev_info.pci_dev) {
-        smap_add_format(args, "pci-vendor_id", "0x%u",
-                        dev_info.pci_dev->id.vendor_id);
-        smap_add_format(args, "pci-device_id", "0x%x",
-                        dev_info.pci_dev->id.device_id);
+    int vid = netdev_dpdk_get_vid(dev);
+    if (vid < 0) {
+        smap_add_format(args, "status", "disconnected");
+        ovs_mutex_unlock(&dev->mutex);
+        return 0;
+    } else {
+        smap_add_format(args, "status", "connected");
+    }
+
+    char socket_name[PATH_MAX];
+    if (!rte_vhost_get_ifname(vid, socket_name, PATH_MAX)) {
+        smap_add_format(args, "socket", "%s", socket_name);
+    }
+
+    uint64_t features;
+    if (!rte_vhost_get_negotiated_features(vid, &features)) {
+        smap_add_format(args, "features", "0x%016"PRIx64, features);
+    }
+
+    uint16_t mtu;
+    if (!rte_vhost_get_mtu(vid, &mtu)) {
+        smap_add_format(args, "mtu", "%d", mtu);
+    }
+
+    int numa = rte_vhost_get_numa_node(vid);
+    if (numa >= 0) {
+        smap_add_format(args, "numa", "%d", numa);
+    }
+
+    uint16_t vring_num = rte_vhost_get_vring_num(vid);
+    if (vring_num) {
+        smap_add_format(args, "num_of_vrings", "%d", vring_num);
+    }
+
+    for (int i = 0; i < vring_num; i++) {
+        struct rte_vhost_vring vring;
+
+        rte_vhost_get_vhost_vring(vid, i, &vring);
+        smap_add_nocopy(args, xasprintf("vring_%d_size", i),
+                        xasprintf("%d", vring.size));
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+    return 0;
+}
+
+static int
+netdev_dpdk_get_status(const struct netdev *netdev, struct smap *args)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    struct rte_eth_dev_info dev_info;
+
+    if (!rte_eth_dev_is_valid_port(dev->port_id)) {
+        return ENODEV;
+    }
+
+    ovs_mutex_lock(&dev->mutex);
+    rte_eth_dev_info_get(dev->port_id, &dev_info);
+    ovs_mutex_unlock(&dev->mutex);
+
+    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);
+    smap_add_format(args, "min_rx_bufsize", "%u", dev_info.min_rx_bufsize);
+    smap_add_format(args, "max_rx_pktlen", "%u", dev->max_packet_len);
+    smap_add_format(args, "max_rx_queues", "%u", dev_info.max_rx_queues);
+    smap_add_format(args, "max_tx_queues", "%u", dev_info.max_tx_queues);
+    smap_add_format(args, "max_mac_addrs", "%u", dev_info.max_mac_addrs);
+    smap_add_format(args, "max_hash_mac_addrs", "%u",
+                           dev_info.max_hash_mac_addrs);
+    smap_add_format(args, "max_vfs", "%u", dev_info.max_vfs);
+    smap_add_format(args, "max_vmdq_pools", "%u", dev_info.max_vmdq_pools);
+
+    /* Querying the DPDK library for iftype may be done in future, pending
+     * support; cf. RFC 3635 Section 3.2.4. */
+    enum { IF_TYPE_ETHERNETCSMACD = 6 };
+
+    smap_add_format(args, "if_type", "%"PRIu32, IF_TYPE_ETHERNETCSMACD);
+    smap_add_format(args, "if_descr", "%s %s", rte_version(),
+                                               dev_info.driver_name);
+
+    if (dev_info.pci_dev) {
+        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);
     }
 
     return 0;
@@ -2485,12 +3083,13 @@ netdev_dpdk_set_admin_state(struct unixctl_conn *conn, int argc,
 
     if (argc > 2) {
         struct netdev *netdev = netdev_from_name(argv[1]);
+
         if (netdev && is_dpdk_class(netdev->netdev_class)) {
-            struct netdev_dpdk *dpdk_dev = netdev_dpdk_cast(netdev);
+            struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
 
-            ovs_mutex_lock(&dpdk_dev->mutex);
-            netdev_dpdk_set_admin_state__(dpdk_dev, up);
-            ovs_mutex_unlock(&dpdk_dev->mutex);
+            ovs_mutex_lock(&dev->mutex);
+            netdev_dpdk_set_admin_state__(dev, up);
+            ovs_mutex_unlock(&dev->mutex);
 
             netdev_close(netdev);
         } else {
@@ -2499,13 +3098,13 @@ netdev_dpdk_set_admin_state(struct unixctl_conn *conn, int argc,
             return;
         }
     } else {
-        struct netdev_dpdk *netdev;
+        struct netdev_dpdk *dev;
 
         ovs_mutex_lock(&dpdk_mutex);
-        LIST_FOR_EACH (netdev, list_node, &dpdk_list) {
-            ovs_mutex_lock(&netdev->mutex);
-            netdev_dpdk_set_admin_state__(netdev, up);
-            ovs_mutex_unlock(&netdev->mutex);
+        LIST_FOR_EACH (dev, list_node, &dpdk_list) {
+            ovs_mutex_lock(&dev->mutex);
+            netdev_dpdk_set_admin_state__(dev, up);
+            ovs_mutex_unlock(&dev->mutex);
         }
         ovs_mutex_unlock(&dpdk_mutex);
     }
@@ -2518,14 +3117,13 @@ netdev_dpdk_detach(struct unixctl_conn *conn, int argc OVS_UNUSED,
 {
     int ret;
     char *response;
-    uint8_t port_id;
+    dpdk_port_t port_id;
     char devname[RTE_ETH_NAME_MAX_LEN];
     struct netdev_dpdk *dev;
 
     ovs_mutex_lock(&dpdk_mutex);
 
-    if (!rte_eth_dev_count() || rte_eth_dev_get_port_by_name(argv[1],
-                                                             &port_id)) {
+    if (rte_eth_dev_get_port_by_name(argv[1], &port_id)) {
         response = xasprintf("Device '%s' not found in DPDK", argv[1]);
         goto error;
     }
@@ -2559,6 +3157,56 @@ error:
     free(response);
 }
 
+static void
+netdev_dpdk_get_mempool_info(struct unixctl_conn *conn,
+                             int argc, const char *argv[],
+                             void *aux OVS_UNUSED)
+{
+    size_t size;
+    FILE *stream;
+    char *response = NULL;
+    struct netdev *netdev = NULL;
+
+    if (argc == 2) {
+        netdev = netdev_from_name(argv[1]);
+        if (!netdev || !is_dpdk_class(netdev->netdev_class)) {
+            unixctl_command_reply_error(conn, "Not a DPDK Interface");
+            goto out;
+        }
+    }
+
+    stream = open_memstream(&response, &size);
+    if (!stream) {
+        response = xasprintf("Unable to open memstream: %s.",
+                             ovs_strerror(errno));
+        unixctl_command_reply_error(conn, response);
+        goto out;
+    }
+
+    if (netdev) {
+        struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+
+        ovs_mutex_lock(&dev->mutex);
+        ovs_mutex_lock(&dpdk_mp_mutex);
+
+        rte_mempool_dump(stream, dev->dpdk_mp->mp);
+
+        ovs_mutex_unlock(&dpdk_mp_mutex);
+        ovs_mutex_unlock(&dev->mutex);
+    } else {
+        ovs_mutex_lock(&dpdk_mp_mutex);
+        rte_mempool_list_dump(stream);
+        ovs_mutex_unlock(&dpdk_mp_mutex);
+    }
+
+    fclose(stream);
+
+    unixctl_command_reply(conn, response);
+out:
+    free(response);
+    netdev_close(netdev);
+}
+
 /*
  * Set virtqueue flags so that we do not receive interrupts.
  */
@@ -2786,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)
 {
@@ -2815,6 +3476,10 @@ netdev_dpdk_class_init(void)
                                  "pci address of device", 1, 1,
                                  netdev_dpdk_detach, NULL);
 
+        unixctl_command_register("netdev-dpdk/get-mempool-info",
+                                 "[netdev]", 0, 1,
+                                 netdev_dpdk_get_mempool_info, NULL);
+
         ovsthread_once_done(&once);
     }
 
@@ -2906,21 +3571,20 @@ dpdk_ring_open(const char dev_name[], dpdk_port_t *eth_port_id)
 
 static int
 netdev_dpdk_ring_send(struct netdev *netdev, int qid,
-                      struct dp_packet_batch *batch, bool may_steal,
-                      bool concurrent_txq)
+                      struct dp_packet_batch *batch, bool concurrent_txq)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-    unsigned i;
+    struct dp_packet *packet;
 
     /* When using 'dpdkr' and sending to a DPDK ring, we want to ensure that
      * the rss hash field is clear. This is because the same mbuf may be
      * modified by the consumer of the ring and return into the datapath
      * without recalculating the RSS hash. */
-    for (i = 0; i < batch->count; i++) {
-        dp_packet_mbuf_rss_flag_reset(batch->packets[i]);
+    DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
+        dp_packet_mbuf_rss_flag_reset(packet);
     }
 
-    netdev_dpdk_send__(dev, qid, batch, may_steal, concurrent_txq);
+    netdev_dpdk_send__(dev, qid, batch, concurrent_txq);
     return 0;
 }
 
@@ -3132,13 +3796,15 @@ 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)
+egress_policer_run(struct qos_conf *conf, struct rte_mbuf **pkts, int pkt_cnt,
+                   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);
+    cnt = netdev_dpdk_policer_run(&policer->egress_meter, pkts,
+                                  pkt_cnt, should_steal);
 
     return cnt;
 }
@@ -3163,24 +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;
 
-    if (dev->mtu != dev->requested_mtu
-        || dev->socket_id != dev->requested_socket_id) {
-        err = netdev_dpdk_mempool_configure(dev);
-        if (err) {
-            goto out;
-        }
+    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;
 
@@ -3216,18 +3884,19 @@ dpdk_vhost_reconfigure_helper(struct netdev_dpdk *dev)
 
     netdev_dpdk_remap_txqs(dev);
 
-    if (dev->requested_socket_id != dev->socket_id
-        || dev->requested_mtu != dev->mtu) {
-        err = netdev_dpdk_mempool_configure(dev);
-        if (err) {
-            return err;
-        } else {
-            netdev_change_seq_changed(&dev->up);
-        }
+    err = netdev_dpdk_mempool_configure(dev);
+    if (!err) {
+        /* A new mempool was created or re-used. */
+        netdev_change_seq_changed(&dev->up);
+    } else if (err != EEXIST){
+        return err;
     }
-
     if (netdev_dpdk_get_vid(dev) >= 0) {
-        dev->vhost_reconfigured = true;
+        if (dev->vhost_reconfigured == false) {
+            dev->vhost_reconfigured = true;
+            /* Carrier status may need updating. */
+            netdev_change_seq_changed(&dev->up);
+        }
     }
 
     return 0;
@@ -3251,6 +3920,8 @@ netdev_dpdk_vhost_client_reconfigure(struct netdev *netdev)
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
     int err;
+    uint64_t vhost_flags = 0;
+    bool zc_enabled;
 
     ovs_mutex_lock(&dev->mutex);
 
@@ -3261,19 +3932,35 @@ netdev_dpdk_vhost_client_reconfigure(struct netdev *netdev)
      */
     if (!(dev->vhost_driver_flags & RTE_VHOST_USER_CLIENT)
             && strlen(dev->vhost_id)) {
-        /* Register client-mode device */
-        err = rte_vhost_driver_register(dev->vhost_id,
-                                        RTE_VHOST_USER_CLIENT);
+        /* Register client-mode device. */
+        vhost_flags |= RTE_VHOST_USER_CLIENT;
+
+        /* Enable IOMMU support, if explicitly requested. */
+        if (dpdk_vhost_iommu_enabled()) {
+            vhost_flags |= RTE_VHOST_USER_IOMMU_SUPPORT;
+        }
+
+        zc_enabled = dev->vhost_driver_flags
+                     & RTE_VHOST_USER_DEQUEUE_ZERO_COPY;
+        /* Enable zero copy flag, if requested */
+        if (zc_enabled) {
+            vhost_flags |= RTE_VHOST_USER_DEQUEUE_ZERO_COPY;
+        }
+
+        err = rte_vhost_driver_register(dev->vhost_id, vhost_flags);
         if (err) {
             VLOG_ERR("vhost-user device setup failure for device %s\n",
                      dev->vhost_id);
             goto unlock;
         } else {
             /* Configuration successful */
-            dev->vhost_driver_flags |= RTE_VHOST_USER_CLIENT;
+            dev->vhost_driver_flags |= vhost_flags;
             VLOG_INFO("vHost User device '%s' created in 'client' mode, "
                       "using client socket '%s'",
                       dev->up.name, dev->vhost_id);
+            if (zc_enabled) {
+                VLOG_INFO("Zero copy enabled for vHost port %s", dev->up.name);
+            }
         }
 
         err = rte_vhost_driver_callback_register(dev->vhost_id,
@@ -3310,143 +3997,784 @@ unlock:
     return err;
 }
 
-#define NETDEV_DPDK_CLASS(NAME, INIT, CONSTRUCT, DESTRUCT,    \
-                          SET_CONFIG, SET_TX_MULTIQ, SEND,    \
-                          GET_CARRIER, GET_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_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_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_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_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_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)