]> git.proxmox.com Git - mirror_ovs.git/blobdiff - lib/netdev-dpdk.c
lldp: increase statsTLVsUnrecognizedTotal on unknown TLV
[mirror_ovs.git] / lib / netdev-dpdk.c
index c06f4693147c6e9ade4e8f19c1925f1a8c77eb25..0b830be784ea10f804829e88bb9d4cab88b8b677 100644 (file)
 #include <sys/socket.h>
 #include <linux/if.h>
 
+/* Include rte_compat.h first to allow experimental API's needed for the
+ * rte_meter.h rfc4115 functions. Once they are no longer marked as
+ * experimental the #define and rte_compat.h include can be removed.
+ */
+#define ALLOW_EXPERIMENTAL_API
+#include <rte_compat.h>
 #include <rte_bus_pci.h>
 #include <rte_config.h>
 #include <rte_cycles.h>
 #include <rte_errno.h>
-#include <rte_eth_ring.h>
 #include <rte_ethdev.h>
 #include <rte_flow.h>
 #include <rte_malloc.h>
 #include <rte_vhost.h>
 
 #include "cmap.h"
+#include "coverage.h"
 #include "dirs.h"
 #include "dp-packet.h"
 #include "dpdk.h"
 #include "dpif-netdev.h"
 #include "fatal-signal.h"
+#include "if-notifier.h"
 #include "netdev-provider.h"
-#include "netdev-rte-offloads.h"
 #include "netdev-vport.h"
 #include "odp-util.h"
 #include "openvswitch/dynamic-string.h"
@@ -65,6 +71,7 @@
 #include "timeval.h"
 #include "unaligned.h"
 #include "unixctl.h"
+#include "userspace-tso.h"
 #include "util.h"
 #include "uuid.h"
 
@@ -73,6 +80,9 @@ enum {VIRTIO_RXQ, VIRTIO_TXQ, VIRTIO_QNUM};
 VLOG_DEFINE_THIS_MODULE(netdev_dpdk);
 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
 
+COVERAGE_DEFINE(vhost_tx_contention);
+COVERAGE_DEFINE(vhost_notification);
+
 #define DPDK_PORT_WATCHDOG_INTERVAL 5
 
 #define OVS_CACHE_LINE_SIZE CACHE_LINE_SIZE
@@ -84,12 +94,13 @@ static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
  * The minimum mbuf size is limited to avoid scatter behaviour and drop in
  * performance for standard Ethernet MTU.
  */
-#define ETHER_HDR_MAX_LEN           (ETHER_HDR_LEN + ETHER_CRC_LEN \
+#define ETHER_HDR_MAX_LEN           (RTE_ETHER_HDR_LEN + RTE_ETHER_CRC_LEN \
                                      + (2 * VLAN_HEADER_LEN))
-#define MTU_TO_FRAME_LEN(mtu)       ((mtu) + ETHER_HDR_LEN + ETHER_CRC_LEN)
+#define MTU_TO_FRAME_LEN(mtu)       ((mtu) + RTE_ETHER_HDR_LEN + \
+                                     RTE_ETHER_CRC_LEN)
 #define MTU_TO_MAX_FRAME_LEN(mtu)   ((mtu) + ETHER_HDR_MAX_LEN)
 #define FRAME_LEN_TO_MTU(frame_len) ((frame_len)                    \
-                                     - ETHER_HDR_LEN - ETHER_CRC_LEN)
+                                     - RTE_ETHER_HDR_LEN - RTE_ETHER_CRC_LEN)
 #define NETDEV_DPDK_MBUF_ALIGN      1024
 #define NETDEV_DPDK_MAX_PKT_LEN     9728
 
@@ -111,34 +122,6 @@ 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);
 
-/*
- * DPDK XSTATS Counter names definition
- */
-#define XSTAT_RX_64_PACKETS              "rx_size_64_packets"
-#define XSTAT_RX_65_TO_127_PACKETS       "rx_size_65_to_127_packets"
-#define XSTAT_RX_128_TO_255_PACKETS      "rx_size_128_to_255_packets"
-#define XSTAT_RX_256_TO_511_PACKETS      "rx_size_256_to_511_packets"
-#define XSTAT_RX_512_TO_1023_PACKETS     "rx_size_512_to_1023_packets"
-#define XSTAT_RX_1024_TO_1522_PACKETS    "rx_size_1024_to_1522_packets"
-#define XSTAT_RX_1523_TO_MAX_PACKETS     "rx_size_1523_to_max_packets"
-
-#define XSTAT_TX_64_PACKETS              "tx_size_64_packets"
-#define XSTAT_TX_65_TO_127_PACKETS       "tx_size_65_to_127_packets"
-#define XSTAT_TX_128_TO_255_PACKETS      "tx_size_128_to_255_packets"
-#define XSTAT_TX_256_TO_511_PACKETS      "tx_size_256_to_511_packets"
-#define XSTAT_TX_512_TO_1023_PACKETS     "tx_size_512_to_1023_packets"
-#define XSTAT_TX_1024_TO_1522_PACKETS    "tx_size_1024_to_1522_packets"
-#define XSTAT_TX_1523_TO_MAX_PACKETS     "tx_size_1523_to_max_packets"
-
-#define XSTAT_RX_MULTICAST_PACKETS       "rx_multicast_packets"
-#define XSTAT_TX_MULTICAST_PACKETS       "tx_multicast_packets"
-#define XSTAT_RX_BROADCAST_PACKETS       "rx_broadcast_packets"
-#define XSTAT_TX_BROADCAST_PACKETS       "tx_broadcast_packets"
-#define XSTAT_RX_UNDERSIZED_ERRORS       "rx_undersized_errors"
-#define XSTAT_RX_OVERSIZE_ERRORS         "rx_oversize_errors"
-#define XSTAT_RX_FRAGMENTED_ERRORS       "rx_fragmented_errors"
-#define XSTAT_RX_JABBER_ERRORS           "rx_jabber_errors"
-
 #define SOCKET0              0
 
 /* Default size of Physical NIC RXQ */
@@ -159,12 +142,27 @@ BUILD_ASSERT_DECL((MAX_NB_MBUF / ROUND_DOWN_POW2(MAX_NB_MBUF / MIN_NB_MBUF))
 typedef uint16_t dpdk_port_t;
 #define DPDK_PORT_ID_FMT "%"PRIu16
 
-#define VHOST_ENQ_RETRY_NUM 8
+/* Minimum amount of vhost tx retries, effectively a disable. */
+#define VHOST_ENQ_RETRY_MIN 0
+/* Maximum amount of vhost tx retries. */
+#define VHOST_ENQ_RETRY_MAX 32
+/* Legacy default value for vhost tx retries. */
+#define VHOST_ENQ_RETRY_DEF 8
+
 #define IF_NAME_SZ (PATH_MAX > IFNAMSIZ ? PATH_MAX : IFNAMSIZ)
 
+/* List of required flags advertised by the hardware that will be used
+ * if TSO is enabled. Ideally this should include DEV_TX_OFFLOAD_SCTP_CKSUM.
+ * However, very few drivers supports that the moment and SCTP is not a
+ * widely used protocol as TCP and UDP, so it's optional. */
+#define DPDK_TX_TSO_OFFLOAD_FLAGS (DEV_TX_OFFLOAD_TCP_TSO        \
+                                   | DEV_TX_OFFLOAD_TCP_CKSUM    \
+                                   | DEV_TX_OFFLOAD_UDP_CKSUM    \
+                                   | DEV_TX_OFFLOAD_IPV4_CKSUM)
+
+
 static const struct rte_eth_conf port_conf = {
     .rxmode = {
-        .mq_mode = ETH_MQ_RX_RSS,
         .split_hdr_size = 0,
         .offloads = 0,
     },
@@ -186,17 +184,35 @@ static const struct rte_eth_conf port_conf = {
 static int new_device(int vid);
 static void destroy_device(int vid);
 static int vring_state_changed(int vid, uint16_t queue_id, int enable);
+static void destroy_connection(int vid);
+static void vhost_guest_notified(int vid);
+
 static const struct vhost_device_ops virtio_net_device_ops =
 {
     .new_device =  new_device,
     .destroy_device = destroy_device,
     .vring_state_changed = vring_state_changed,
-    .features_changed = NULL
+    .features_changed = NULL,
+    .new_connection = NULL,
+    .destroy_connection = destroy_connection,
+    .guest_notified = vhost_guest_notified,
 };
 
-enum { DPDK_RING_SIZE = 256 };
-BUILD_ASSERT_DECL(IS_POW2(DPDK_RING_SIZE));
-enum { DRAIN_TSC = 200000ULL };
+/* Custom software stats for dpdk ports */
+struct netdev_dpdk_sw_stats {
+    /* No. of retries when unable to transmit. */
+    uint64_t tx_retries;
+    /* Packet drops when unable to transmit; Probably Tx queue is full. */
+    uint64_t tx_failure_drops;
+    /* Packet length greater than device MTU. */
+    uint64_t tx_mtu_exceeded_drops;
+    /* Packet drops in egress policer processing. */
+    uint64_t tx_qos_drops;
+    /* Packet drops in ingress policer processing. */
+    uint64_t rx_qos_drops;
+    /* Packet drops in HWOL processing. */
+    uint64_t tx_invalid_hwol_drops;
+};
 
 enum dpdk_dev_type {
     DPDK_DEV_ETH = 0,
@@ -216,6 +232,13 @@ struct qos_conf {
     rte_spinlock_t lock;
 };
 
+/* QoS queue information used by the netdev queue dump functions. */
+struct netdev_dpdk_queue_state {
+    uint32_t *queues;
+    size_t cur_queue;
+    size_t n_queues;
+};
+
 /* A particular implementation of dpdk QoS operations.
  *
  * The functions below return 0 if successful or a positive errno value on
@@ -282,10 +305,46 @@ struct dpdk_qos_ops {
      */
     int (*qos_run)(struct qos_conf *qos_conf, struct rte_mbuf **pkts,
                    int pkt_cnt, bool should_steal);
+
+    /* Called to construct a QoS Queue. The implementation should make
+     * the appropriate calls to configure QoS Queue according to 'details'.
+     *
+     * The contents of 'details' should be documented as valid for 'ovs_name'
+     * in the "other_config" column in the "QoS" table in vswitchd/vswitch.xml
+     * (which is built as ovs-vswitchd.conf.db(8)).
+     *
+     * This function must return 0 if and only if it constructs
+     * QoS queue successfully.
+     */
+    int (*qos_queue_construct)(const struct smap *details,
+                               uint32_t queue_id, struct qos_conf *conf);
+
+    /* Destroys the QoS Queue. */
+    void (*qos_queue_destruct)(struct qos_conf *conf, uint32_t queue_id);
+
+    /* Retrieves details of QoS Queue configuration into 'details'.
+     *
+     * The contents of 'details' should be documented as valid for 'ovs_name'
+     * in the "other_config" column in the "QoS" table in vswitchd/vswitch.xml
+     * (which is built as ovs-vswitchd.conf.db(8)).
+     */
+    int (*qos_queue_get)(struct smap *details, uint32_t queue_id,
+                         const struct qos_conf *conf);
+
+    /* Retrieves statistics of QoS Queue configuration into 'stats'. */
+    int (*qos_queue_get_stats)(const struct qos_conf *conf, uint32_t queue_id,
+                               struct netdev_queue_stats *stats);
+
+    /* Setup the 'netdev_dpdk_queue_state' structure used by the dpdk queue
+     * dump functions.
+     */
+    int (*qos_queue_dump_state_init)(const struct qos_conf *conf,
+                                     struct netdev_dpdk_queue_state *state);
 };
 
-/* dpdk_qos_ops for each type of user space QoS implementation */
+/* dpdk_qos_ops for each type of user space QoS implementation. */
 static const struct dpdk_qos_ops egress_policer_ops;
+static const struct dpdk_qos_ops trtcm_policer_ops;
 
 /*
  * Array of dpdk_qos_ops, contains pointer to all supported QoS
@@ -293,6 +352,7 @@ static const struct dpdk_qos_ops egress_policer_ops;
  */
 static const struct dpdk_qos_ops *const qos_confs[] = {
     &egress_policer_ops,
+    &trtcm_policer_ops,
     NULL
 };
 
@@ -318,30 +378,17 @@ struct dpdk_mp {
  };
 
 /* There should be one 'struct dpdk_tx_queue' created for
- * each cpu core. */
+ * each netdev tx queue. */
 struct dpdk_tx_queue {
-    rte_spinlock_t tx_lock;        /* Protects the members and the NIC queue
-                                    * from concurrent access.  It is used only
-                                    * if the queue is shared among different
-                                    * pmd threads (see 'concurrent_txq'). */
-    int map;                       /* Mapping of configured vhost-user queues
-                                    * to enabled by guest. */
-};
-
-/* dpdk has no way to remove dpdk ring ethernet devices
-   so we have to keep them around once they've been created
-*/
-
-static struct ovs_list dpdk_ring_list OVS_GUARDED_BY(dpdk_mutex)
-    = OVS_LIST_INITIALIZER(&dpdk_ring_list);
-
-struct dpdk_ring {
-    /* For the client rings */
-    struct rte_ring *cring_tx;
-    struct rte_ring *cring_rx;
-    unsigned int user_port_id; /* User given port no, parsed from port name */
-    dpdk_port_t eth_port_id; /* ethernet device port id */
-    struct ovs_list list_node OVS_GUARDED_BY(dpdk_mutex);
+    /* Padding to make dpdk_tx_queue exactly one cache line long. */
+    PADDED_MEMBERS(CACHE_LINE_SIZE,
+        /* Protects the members and the NIC queue from concurrent access.
+         * It is used only if the queue is shared among different pmd threads
+         * (see 'concurrent_txq'). */
+        rte_spinlock_t tx_lock;
+        /* Mapping of configured vhost-user queue to enabled by guest. */
+        int map;
+    );
 };
 
 struct ingress_policer {
@@ -354,7 +401,9 @@ struct ingress_policer {
 enum dpdk_hw_ol_features {
     NETDEV_RX_CHECKSUM_OFFLOAD = 1 << 0,
     NETDEV_RX_HW_CRC_STRIP = 1 << 1,
-    NETDEV_RX_HW_SCATTER = 1 << 2
+    NETDEV_RX_HW_SCATTER = 1 << 2,
+    NETDEV_TX_TSO_OFFLOAD = 1 << 3,
+    NETDEV_TX_SCTP_CHECKSUM_OFFLOAD = 1 << 4,
 };
 
 /*
@@ -382,6 +431,8 @@ struct netdev_dpdk {
         bool attached;
         /* If true, rte_eth_dev_start() was successfully called */
         bool started;
+        bool reset_needed;
+        /* 1 pad byte here. */
         struct eth_addr hwaddr;
         int mtu;
         int socket_id;
@@ -409,7 +460,9 @@ struct netdev_dpdk {
 
         /* True if vHost device is 'up' and has been reconfigured at least once */
         bool vhost_reconfigured;
-        /* 3 pad bytes here. */
+
+        atomic_uint8_t vhost_tx_retries_max;
+        /* 2 pad bytes here. */
     );
 
     PADDED_MEMBERS(CACHE_LINE_SIZE,
@@ -424,13 +477,17 @@ struct netdev_dpdk {
         OVSRCU_TYPE(struct ingress_policer *) ingress_policer;
         uint32_t policer_rate;
         uint32_t policer_burst;
+
+        /* Array of vhost rxq states, see vring_state_changed. */
+        bool *vhost_rxq_enabled;
     );
 
     PADDED_MEMBERS(CACHE_LINE_SIZE,
         struct netdev_stats stats;
+        struct netdev_dpdk_sw_stats *sw_stats;
         /* Protects stats */
         rte_spinlock_t stats_lock;
-        /* 44 pad bytes here. */
+        /* 36 pad bytes here. */
     );
 
     PADDED_MEMBERS(CACHE_LINE_SIZE,
@@ -484,6 +541,8 @@ struct netdev_rxq_dpdk {
 static void netdev_dpdk_destruct(struct netdev *netdev);
 static void netdev_dpdk_vhost_destruct(struct netdev *netdev);
 
+static int netdev_dpdk_get_sw_custom_stats(const struct netdev *,
+                                           struct netdev_custom_stats *);
 static void netdev_dpdk_clear_xstats(struct netdev_dpdk *dev);
 
 int netdev_dpdk_get_vid(const struct netdev_dpdk *dev);
@@ -594,7 +653,7 @@ dpdk_calculate_mbufs(struct netdev_dpdk *dev, int mtu, bool per_port_mp)
          * can change dynamically at runtime. For now, use this rough
          * heurisitic.
          */
-        if (mtu >= ETHER_MTU) {
+        if (mtu >= RTE_ETHER_MTU) {
             n_mbufs = MAX_NB_MBUF;
         } else {
             n_mbufs = MIN_NB_MBUF;
@@ -905,11 +964,19 @@ dpdk_eth_dev_port_config(struct netdev_dpdk *dev, int n_rxq, int n_txq)
 
     rte_eth_dev_info_get(dev->port_id, &info);
 
+    /* As of DPDK 19.11, it is not allowed to set a mq_mode for
+     * virtio PMD driver. */
+    if (!strcmp(info.driver_name, "net_virtio")) {
+        conf.rxmode.mq_mode = ETH_MQ_RX_NONE;
+    } else {
+        conf.rxmode.mq_mode = ETH_MQ_RX_RSS;
+    }
+
     /* As of DPDK 17.11.1 a few PMDs require to explicitly enable
      * scatter to support jumbo RX.
      * Setting scatter for the device is done after checking for
      * scatter support in the device capabilites. */
-    if (dev->mtu > ETHER_MTU) {
+    if (dev->mtu > RTE_ETHER_MTU) {
         if (dev->hw_ol_features & NETDEV_RX_HW_SCATTER) {
             conf.rxmode.offloads |= DEV_RX_OFFLOAD_SCATTER;
         }
@@ -926,6 +993,13 @@ dpdk_eth_dev_port_config(struct netdev_dpdk *dev, int n_rxq, int n_txq)
         conf.rxmode.offloads |= DEV_RX_OFFLOAD_KEEP_CRC;
     }
 
+    if (dev->hw_ol_features & NETDEV_TX_TSO_OFFLOAD) {
+        conf.txmode.offloads |= DPDK_TX_TSO_OFFLOAD_FLAGS;
+        if (dev->hw_ol_features & NETDEV_TX_SCTP_CHECKSUM_OFFLOAD) {
+            conf.txmode.offloads |= DEV_TX_OFFLOAD_SCTP_CKSUM;
+        }
+    }
+
     /* Limit configured rss hash functions to only those supported
      * by the eth device. */
     conf.rx_adv_conf.rss_conf.rss_hf &= info.flow_type_rss_offloads;
@@ -1021,9 +1095,10 @@ dpdk_eth_dev_init(struct netdev_dpdk *dev)
 {
     struct rte_pktmbuf_pool_private *mbp_priv;
     struct rte_eth_dev_info info;
-    struct ether_addr eth_addr;
+    struct rte_ether_addr eth_addr;
     int diag;
     int n_rxq, n_txq;
+    uint32_t tx_tso_offload_capa = DPDK_TX_TSO_OFFLOAD_FLAGS;
     uint32_t rx_chksm_offload_capa = DEV_RX_OFFLOAD_UDP_CKSUM |
                                      DEV_RX_OFFLOAD_TCP_CKSUM |
                                      DEV_RX_OFFLOAD_IPV4_CKSUM;
@@ -1053,6 +1128,24 @@ dpdk_eth_dev_init(struct netdev_dpdk *dev)
         dev->hw_ol_features &= ~NETDEV_RX_HW_SCATTER;
     }
 
+    dev->hw_ol_features &= ~NETDEV_TX_TSO_OFFLOAD;
+    if (userspace_tso_enabled()) {
+        if ((info.tx_offload_capa & tx_tso_offload_capa)
+            == tx_tso_offload_capa) {
+            dev->hw_ol_features |= NETDEV_TX_TSO_OFFLOAD;
+            if (info.tx_offload_capa & DEV_TX_OFFLOAD_SCTP_CKSUM) {
+                dev->hw_ol_features |= NETDEV_TX_SCTP_CHECKSUM_OFFLOAD;
+            } else {
+                VLOG_WARN("%s: Tx SCTP checksum offload is not supported, "
+                          "SCTP packets sent to this device will be dropped",
+                          netdev_get_name(&dev->up));
+            }
+        } else {
+            VLOG_WARN("%s: Tx TSO offload is not supported.",
+                      netdev_get_name(&dev->up));
+        }
+    }
+
     n_rxq = MIN(info.max_rx_queues, dev->up.n_rxq);
     n_txq = MIN(info.max_tx_queues, dev->up.n_txq);
 
@@ -1146,12 +1239,14 @@ common_construct(struct netdev *netdev, dpdk_port_t port_no,
     dev->port_id = port_no;
     dev->type = type;
     dev->flags = 0;
-    dev->requested_mtu = ETHER_MTU;
+    dev->requested_mtu = RTE_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;
+    dev->started = false;
+    dev->reset_needed = false;
 
     ovsrcu_init(&dev->qos_conf, NULL);
 
@@ -1184,28 +1279,10 @@ common_construct(struct netdev *netdev, dpdk_port_t port_no,
     dev->rte_xstats_ids = NULL;
     dev->rte_xstats_ids_size = 0;
 
-    return 0;
-}
-
-/* dev_name must be the prefix followed by a positive decimal number.
- * (no leading + or - signs are allowed) */
-static int
-dpdk_dev_parse_name(const char dev_name[], const char prefix[],
-                    unsigned int *port_no)
-{
-    const char *cport;
-
-    if (strncmp(dev_name, prefix, strlen(prefix))) {
-        return ENODEV;
-    }
-
-    cport = dev_name + strlen(prefix);
+    dev->sw_stats = xzalloc(sizeof *dev->sw_stats);
+    dev->sw_stats->tx_retries = (dev->type == DPDK_DEV_VHOST) ? 0 : UINT64_MAX;
 
-    if (str_to_uint(cport, 10, port_no)) {
-        return 0;
-    } else {
-        return ENODEV;
-    }
+    return 0;
 }
 
 /* Get the number of OVS interfaces which have the same DPDK
@@ -1235,11 +1312,19 @@ vhost_common_construct(struct netdev *netdev)
     int socket_id = rte_lcore_to_socket_id(rte_get_master_lcore());
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
 
+    dev->vhost_rxq_enabled = dpdk_rte_mzalloc(OVS_VHOST_MAX_QUEUE_NUM *
+                                              sizeof *dev->vhost_rxq_enabled);
+    if (!dev->vhost_rxq_enabled) {
+        return ENOMEM;
+    }
     dev->tx_q = netdev_dpdk_alloc_txq(OVS_VHOST_MAX_QUEUE_NUM);
     if (!dev->tx_q) {
+        rte_free(dev->vhost_rxq_enabled);
         return ENOMEM;
     }
 
+    atomic_init(&dev->vhost_tx_retries_max, VHOST_ENQ_RETRY_DEF);
+
     return common_construct(netdev, DPDK_ETH_PORT_ID_INVALID,
                             DPDK_DEV_VHOST, socket_id);
 }
@@ -1268,6 +1353,9 @@ netdev_dpdk_vhost_construct(struct netdev *netdev)
     dev->vhost_id = xasprintf("%s/%s", dpdk_get_vhost_sock_dir(), name);
 
     dev->vhost_driver_flags &= ~RTE_VHOST_USER_CLIENT;
+
+    /* There is no support for multi-segments buffers. */
+    dev->vhost_driver_flags |= RTE_VHOST_USER_LINEARBUF_SUPPORT;
     err = rte_vhost_driver_register(dev->vhost_id, dev->vhost_driver_flags);
     if (err) {
         VLOG_ERR("vhost-user socket device setup failure for socket %s\n",
@@ -1287,14 +1375,16 @@ netdev_dpdk_vhost_construct(struct netdev *netdev)
         goto out;
     }
 
-    err = rte_vhost_driver_disable_features(dev->vhost_id,
-                                1ULL << VIRTIO_NET_F_HOST_TSO4
-                                | 1ULL << VIRTIO_NET_F_HOST_TSO6
-                                | 1ULL << VIRTIO_NET_F_CSUM);
-    if (err) {
-        VLOG_ERR("rte_vhost_driver_disable_features failed for vhost user "
-                 "port: %s\n", name);
-        goto out;
+    if (!userspace_tso_enabled()) {
+        err = rte_vhost_driver_disable_features(dev->vhost_id,
+                                    1ULL << VIRTIO_NET_F_HOST_TSO4
+                                    | 1ULL << VIRTIO_NET_F_HOST_TSO6
+                                    | 1ULL << VIRTIO_NET_F_CSUM);
+        if (err) {
+            VLOG_ERR("rte_vhost_driver_disable_features failed for vhost user "
+                     "port: %s\n", name);
+            goto out;
+        }
     }
 
     err = rte_vhost_driver_start(dev->vhost_id);
@@ -1360,6 +1450,7 @@ common_destruct(struct netdev_dpdk *dev)
     ovs_list_remove(&dev->list_node);
     free(ovsrcu_get_protected(struct ingress_policer *,
                               &dev->ingress_policer));
+    free(dev->sw_stats);
     ovs_mutex_destroy(&dev->mutex);
 }
 
@@ -1395,8 +1486,11 @@ netdev_dpdk_destruct(struct netdev *netdev)
          * device are closed.
          */
         if (!remove_on_close || !netdev_dpdk_get_num_ports(rte_dev)) {
-            if (rte_dev_remove(rte_dev) < 0) {
-                VLOG_ERR("Device '%s' can not be detached", dev->devargs);
+            int ret = rte_dev_remove(rte_dev);
+
+            if (ret < 0) {
+                VLOG_ERR("Device '%s' can not be detached: %s.",
+                         dev->devargs, rte_strerror(-ret));
             } else {
                 /* Device was closed and detached. */
                 VLOG_INFO("Device '%s' has been removed and detached",
@@ -1446,6 +1540,7 @@ netdev_dpdk_vhost_destruct(struct netdev *netdev)
 
     vhost_id = dev->vhost_id;
     dev->vhost_id = NULL;
+    rte_free(dev->vhost_rxq_enabled);
 
     common_destruct(dev);
 
@@ -1624,6 +1719,11 @@ netdev_dpdk_get_config(const struct netdev *netdev, struct smap *args)
         } else {
             smap_add(args, "rx_csum_offload", "false");
         }
+        if (dev->hw_ol_features & NETDEV_TX_TSO_OFFLOAD) {
+            smap_add(args, "tx_tso_offload", "true");
+        } else {
+            smap_add(args, "tx_tso_offload", "false");
+        }
         smap_add(args, "lsc_interrupt_mode",
                  dev->lsc_interrupt_mode ? "true" : "false");
     }
@@ -1659,7 +1759,7 @@ netdev_dpdk_get_port_by_mac(const char *mac_str)
     }
 
     RTE_ETH_FOREACH_DEV (port_id) {
-        struct ether_addr ea;
+        struct rte_ether_addr ea;
 
         rte_eth_macaddr_get(port_id, &ea);
         memcpy(port_mac.ea, ea.addr_bytes, ETH_ADDR_LEN);
@@ -1734,6 +1834,34 @@ netdev_dpdk_process_devargs(struct netdev_dpdk *dev,
     return new_port_id;
 }
 
+static int
+dpdk_eth_event_callback(dpdk_port_t port_id, enum rte_eth_event_type type,
+                        void *param OVS_UNUSED, void *ret_param OVS_UNUSED)
+{
+    struct netdev_dpdk *dev;
+
+    switch ((int) type) {
+    case RTE_ETH_EVENT_INTR_RESET:
+        ovs_mutex_lock(&dpdk_mutex);
+        dev = netdev_dpdk_lookup_by_port_id(port_id);
+        if (dev) {
+            ovs_mutex_lock(&dev->mutex);
+            dev->reset_needed = true;
+            netdev_request_reconfigure(&dev->up);
+            VLOG_DBG_RL(&rl, "%s: Device reset requested.",
+                        netdev_get_name(&dev->up));
+            ovs_mutex_unlock(&dev->mutex);
+        }
+        ovs_mutex_unlock(&dpdk_mutex);
+        break;
+
+    default:
+        /* Ignore all other types. */
+        break;
+   }
+   return 0;
+}
+
 static void
 dpdk_set_rxq_config(struct netdev_dpdk *dev, const struct smap *args)
     OVS_REQUIRES(dev->mutex)
@@ -1770,6 +1898,7 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
     bool rx_fc_en, tx_fc_en, autoneg, lsc_interrupt_mode;
+    bool flow_control_requested = true;
     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},
@@ -1792,7 +1921,7 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
 
     new_devargs = smap_get(args, "dpdk-devargs");
 
-    if (dev->devargs && strcmp(new_devargs, dev->devargs)) {
+    if (dev->devargs && new_devargs && strcmp(new_devargs, dev->devargs)) {
         /* The user requested a new device.  If we return error, the caller
          * will delete this netdev and try to recreate it. */
         err = EAGAIN;
@@ -1857,15 +1986,34 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args,
     autoneg = smap_get_bool(args, "flow-ctrl-autoneg", false);
 
     fc_mode = fc_mode_set[tx_fc_en][rx_fc_en];
+
+    if (!smap_get(args, "rx-flow-ctrl") && !smap_get(args, "tx-flow-ctrl")
+        && !smap_get(args, "flow-ctrl-autoneg")) {
+        /* FIXME: User didn't ask for flow control configuration.
+         *        For now we'll not print a warning if flow control is not
+         *        supported by the DPDK port. */
+        flow_control_requested = false;
+    }
+
+    /* Get the Flow control configuration. */
+    err = -rte_eth_dev_flow_ctrl_get(dev->port_id, &dev->fc_conf);
+    if (err) {
+        if (err == ENOTSUP) {
+            if (flow_control_requested) {
+                VLOG_WARN("%s: Flow control is not supported.",
+                          netdev_get_name(netdev));
+            }
+            err = 0; /* Not fatal. */
+        } else {
+            VLOG_WARN("%s: Cannot get flow control parameters: %s",
+                      netdev_get_name(netdev), rte_strerror(err));
+        }
+        goto out;
+    }
+
     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);
     }
 
@@ -1876,19 +2024,6 @@ out:
     return err;
 }
 
-static int
-netdev_dpdk_ring_set_config(struct netdev *netdev, const struct smap *args,
-                            char **errp OVS_UNUSED)
-{
-    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-
-    ovs_mutex_lock(&dev->mutex);
-    dpdk_set_rxq_config(dev, args);
-    ovs_mutex_unlock(&dev->mutex);
-
-    return 0;
-}
-
 static int
 netdev_dpdk_vhost_client_set_config(struct netdev *netdev,
                                     const struct smap *args,
@@ -1896,6 +2031,7 @@ netdev_dpdk_vhost_client_set_config(struct netdev *netdev,
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
     const char *path;
+    int max_tx_retries, cur_max_tx_retries;
 
     ovs_mutex_lock(&dev->mutex);
     if (!(dev->vhost_driver_flags & RTE_VHOST_USER_CLIENT)) {
@@ -1903,15 +2039,22 @@ netdev_dpdk_vhost_client_set_config(struct netdev *netdev,
         if (!nullable_string_is_equal(path, dev->vhost_id)) {
             free(dev->vhost_id);
             dev->vhost_id = nullable_xstrdup(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);
         }
     }
+
+    max_tx_retries = smap_get_int(args, "tx-retries-max",
+                                  VHOST_ENQ_RETRY_DEF);
+    if (max_tx_retries < VHOST_ENQ_RETRY_MIN
+        || max_tx_retries > VHOST_ENQ_RETRY_MAX) {
+        max_tx_retries = VHOST_ENQ_RETRY_DEF;
+    }
+    atomic_read_relaxed(&dev->vhost_tx_retries_max, &cur_max_tx_retries);
+    if (max_tx_retries != cur_max_tx_retries) {
+        atomic_store_relaxed(&dev->vhost_tx_retries_max, max_tx_retries);
+        VLOG_INFO("Max Tx retries for vhost device '%s' set to %d",
+                  netdev_get_name(netdev), max_tx_retries);
+    }
     ovs_mutex_unlock(&dev->mutex);
 
     return 0;
@@ -1989,6 +2132,67 @@ netdev_dpdk_rxq_dealloc(struct netdev_rxq *rxq)
     rte_free(rx);
 }
 
+/* Prepare the packet for HWOL.
+ * Return True if the packet is OK to continue. */
+static bool
+netdev_dpdk_prep_hwol_packet(struct netdev_dpdk *dev, struct rte_mbuf *mbuf)
+{
+    struct dp_packet *pkt = CONTAINER_OF(mbuf, struct dp_packet, mbuf);
+
+    if (mbuf->ol_flags & PKT_TX_L4_MASK) {
+        mbuf->l2_len = (char *)dp_packet_l3(pkt) - (char *)dp_packet_eth(pkt);
+        mbuf->l3_len = (char *)dp_packet_l4(pkt) - (char *)dp_packet_l3(pkt);
+        mbuf->outer_l2_len = 0;
+        mbuf->outer_l3_len = 0;
+    }
+
+    if (mbuf->ol_flags & PKT_TX_TCP_SEG) {
+        struct tcp_header *th = dp_packet_l4(pkt);
+
+        if (!th) {
+            VLOG_WARN_RL(&rl, "%s: TCP Segmentation without L4 header"
+                         " pkt len: %"PRIu32"", dev->up.name, mbuf->pkt_len);
+            return false;
+        }
+
+        mbuf->l4_len = TCP_OFFSET(th->tcp_ctl) * 4;
+        mbuf->ol_flags |= PKT_TX_TCP_CKSUM;
+        mbuf->tso_segsz = dev->mtu - mbuf->l3_len - mbuf->l4_len;
+
+        if (mbuf->ol_flags & PKT_TX_IPV4) {
+            mbuf->ol_flags |= PKT_TX_IP_CKSUM;
+        }
+    }
+    return true;
+}
+
+/* Prepare a batch for HWOL.
+ * Return the number of good packets in the batch. */
+static int
+netdev_dpdk_prep_hwol_batch(struct netdev_dpdk *dev, struct rte_mbuf **pkts,
+                            int pkt_cnt)
+{
+    int i = 0;
+    int cnt = 0;
+    struct rte_mbuf *pkt;
+
+    /* Prepare and filter bad HWOL packets. */
+    for (i = 0; i < pkt_cnt; i++) {
+        pkt = pkts[i];
+        if (!netdev_dpdk_prep_hwol_packet(dev, pkt)) {
+            rte_pktmbuf_free(pkt);
+            continue;
+        }
+
+        if (OVS_UNLIKELY(i != cnt)) {
+            pkts[cnt] = pkt;
+        }
+        cnt++;
+    }
+
+    return cnt;
+}
+
 /* Tries to transmit 'pkts' to txq 'qid' of device 'dev'.  Takes ownership of
  * 'pkts', even in case of failure.
  *
@@ -1998,11 +2202,22 @@ netdev_dpdk_eth_tx_burst(struct netdev_dpdk *dev, int qid,
                          struct rte_mbuf **pkts, int cnt)
 {
     uint32_t nb_tx = 0;
+    uint16_t nb_tx_prep = cnt;
+
+    if (userspace_tso_enabled()) {
+        nb_tx_prep = rte_eth_tx_prepare(dev->port_id, qid, pkts, cnt);
+        if (nb_tx_prep != cnt) {
+            VLOG_WARN_RL(&rl, "%s: Output batch contains invalid packets. "
+                         "Only %u/%u are valid: %s", dev->up.name, nb_tx_prep,
+                         cnt, rte_strerror(rte_errno));
+        }
+    }
 
-    while (nb_tx != cnt) {
+    while (nb_tx != nb_tx_prep) {
         uint32_t ret;
 
-        ret = rte_eth_tx_burst(dev->port_id, qid, pkts + nb_tx, cnt - nb_tx);
+        ret = rte_eth_tx_burst(dev->port_id, qid, pkts + nb_tx,
+                               nb_tx_prep - nb_tx);
         if (!ret) {
             break;
         }
@@ -2024,21 +2239,21 @@ netdev_dpdk_eth_tx_burst(struct netdev_dpdk *dev, int qid,
 }
 
 static inline bool
-netdev_dpdk_policer_pkt_handle(struct rte_meter_srtcm *meter,
-                               struct rte_meter_srtcm_profile *profile,
-                               struct rte_mbuf *pkt, uint64_t time)
+netdev_dpdk_srtcm_policer_pkt_handle(struct rte_meter_srtcm *meter,
+                                     struct rte_meter_srtcm_profile *profile,
+                                     struct rte_mbuf *pkt, uint64_t time)
 {
-    uint32_t pkt_len = rte_pktmbuf_pkt_len(pkt) - sizeof(struct ether_hdr);
+    uint32_t pkt_len = rte_pktmbuf_pkt_len(pkt) - sizeof(struct rte_ether_hdr);
 
     return rte_meter_srtcm_color_blind_check(meter, profile, time, pkt_len) ==
-                                             e_RTE_METER_GREEN;
+                                             RTE_COLOR_GREEN;
 }
 
 static int
-netdev_dpdk_policer_run(struct rte_meter_srtcm *meter,
-                        struct rte_meter_srtcm_profile *profile,
-                        struct rte_mbuf **pkts, int pkt_cnt,
-                        bool should_steal)
+srtcm_policer_run_single_packet(struct rte_meter_srtcm *meter,
+                                struct rte_meter_srtcm_profile *profile,
+                                struct rte_mbuf **pkts, int pkt_cnt,
+                                bool should_steal)
 {
     int i = 0;
     int cnt = 0;
@@ -2048,8 +2263,8 @@ netdev_dpdk_policer_run(struct rte_meter_srtcm *meter,
     for (i = 0; i < pkt_cnt; i++) {
         pkt = pkts[i];
         /* Handle current packet */
-        if (netdev_dpdk_policer_pkt_handle(meter, profile,
-                                           pkt, current_time)) {
+        if (netdev_dpdk_srtcm_policer_pkt_handle(meter, profile,
+                                                 pkt, current_time)) {
             if (cnt != i) {
                 pkts[cnt] = pkt;
             }
@@ -2071,8 +2286,9 @@ ingress_policer_run(struct ingress_policer *policer, struct rte_mbuf **pkts,
     int cnt = 0;
 
     rte_spinlock_lock(&policer->policer_lock);
-    cnt = netdev_dpdk_policer_run(&policer->in_policer, &policer->in_prof,
-                                  pkts, pkt_cnt, should_steal);
+    cnt = srtcm_policer_run_single_packet(&policer->in_policer,
+                                          &policer->in_prof,
+                                          pkts, pkt_cnt, should_steal);
     rte_spinlock_unlock(&policer->policer_lock);
 
     return cnt;
@@ -2111,16 +2327,17 @@ netdev_dpdk_vhost_update_rx_size_counters(struct netdev_stats *stats,
 }
 
 static inline void
-netdev_dpdk_vhost_update_rx_counters(struct netdev_stats *stats,
+netdev_dpdk_vhost_update_rx_counters(struct netdev_dpdk *dev,
                                      struct dp_packet **packets, int count,
-                                     int dropped)
+                                     int qos_drops)
 {
-    int i;
-    unsigned int packet_size;
+    struct netdev_stats *stats = &dev->stats;
     struct dp_packet *packet;
+    unsigned int packet_size;
+    int i;
 
     stats->rx_packets += count;
-    stats->rx_dropped += dropped;
+    stats->rx_dropped += qos_drops;
     for (i = 0; i < count; i++) {
         packet = packets[i];
         packet_size = dp_packet_size(packet);
@@ -2143,6 +2360,10 @@ netdev_dpdk_vhost_update_rx_counters(struct netdev_stats *stats,
 
         stats->rx_bytes += packet_size;
     }
+
+    if (OVS_UNLIKELY(qos_drops)) {
+        dev->sw_stats->rx_qos_drops += qos_drops;
+    }
 }
 
 /*
@@ -2155,7 +2376,7 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
     struct netdev_dpdk *dev = netdev_dpdk_cast(rxq->netdev);
     struct ingress_policer *policer = netdev_dpdk_get_ingress_policer(dev);
     uint16_t nb_rx = 0;
-    uint16_t dropped = 0;
+    uint16_t qos_drops = 0;
     int qid = rxq->queue_id * VIRTIO_QNUM + VIRTIO_TXQ;
     int vid = netdev_dpdk_get_vid(dev);
 
@@ -2182,16 +2403,16 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
     }
 
     if (policer) {
-        dropped = nb_rx;
+        qos_drops = nb_rx;
         nb_rx = ingress_policer_run(policer,
                                     (struct rte_mbuf **) batch->packets,
                                     nb_rx, true);
-        dropped -= nb_rx;
+        qos_drops -= nb_rx;
     }
 
     rte_spinlock_lock(&dev->stats_lock);
-    netdev_dpdk_vhost_update_rx_counters(&dev->stats, batch->packets,
-                                         nb_rx, dropped);
+    netdev_dpdk_vhost_update_rx_counters(dev, batch->packets,
+                                         nb_rx, qos_drops);
     rte_spinlock_unlock(&dev->stats_lock);
 
     batch->count = nb_rx;
@@ -2200,6 +2421,14 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
     return 0;
 }
 
+static bool
+netdev_dpdk_vhost_rxq_enabled(struct netdev_rxq *rxq)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(rxq->netdev);
+
+    return dev->vhost_rxq_enabled[rxq->queue_id];
+}
+
 static int
 netdev_dpdk_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch,
                      int *qfill)
@@ -2233,6 +2462,7 @@ netdev_dpdk_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch,
     if (OVS_UNLIKELY(dropped)) {
         rte_spinlock_lock(&dev->stats_lock);
         dev->stats.rx_dropped += dropped;
+        dev->sw_stats->rx_qos_drops += dropped;
         rte_spinlock_unlock(&dev->stats_lock);
     }
 
@@ -2273,11 +2503,14 @@ netdev_dpdk_filter_packet_len(struct netdev_dpdk *dev, struct rte_mbuf **pkts,
     int cnt = 0;
     struct rte_mbuf *pkt;
 
+    /* Filter oversized packets, unless are marked for TSO. */
     for (i = 0; i < pkt_cnt; i++) {
         pkt = pkts[i];
-        if (OVS_UNLIKELY(pkt->pkt_len > dev->max_packet_len)) {
-            VLOG_WARN_RL(&rl, "%s: Too big size %" PRIu32 " max_packet_len %d",
-                         dev->up.name, pkt->pkt_len, dev->max_packet_len);
+        if (OVS_UNLIKELY((pkt->pkt_len > dev->max_packet_len)
+            && !(pkt->ol_flags & PKT_TX_TCP_SEG))) {
+            VLOG_WARN_RL(&rl, "%s: Too big size %" PRIu32 " "
+                         "max_packet_len %d", dev->up.name, pkt->pkt_len,
+                         dev->max_packet_len);
             rte_pktmbuf_free(pkt);
             continue;
         }
@@ -2292,13 +2525,18 @@ netdev_dpdk_filter_packet_len(struct netdev_dpdk *dev, struct rte_mbuf **pkts,
 }
 
 static inline void
-netdev_dpdk_vhost_update_tx_counters(struct netdev_stats *stats,
+netdev_dpdk_vhost_update_tx_counters(struct netdev_dpdk *dev,
                                      struct dp_packet **packets,
                                      int attempted,
-                                     int dropped)
+                                     struct netdev_dpdk_sw_stats *sw_stats_add)
 {
-    int i;
+    int dropped = sw_stats_add->tx_mtu_exceeded_drops +
+                  sw_stats_add->tx_qos_drops +
+                  sw_stats_add->tx_failure_drops +
+                  sw_stats_add->tx_invalid_hwol_drops;
+    struct netdev_stats *stats = &dev->stats;
     int sent = attempted - dropped;
+    int i;
 
     stats->tx_packets += sent;
     stats->tx_dropped += dropped;
@@ -2306,6 +2544,16 @@ netdev_dpdk_vhost_update_tx_counters(struct netdev_stats *stats,
     for (i = 0; i < sent; i++) {
         stats->tx_bytes += dp_packet_size(packets[i]);
     }
+
+    if (OVS_UNLIKELY(dropped || sw_stats_add->tx_retries)) {
+        struct netdev_dpdk_sw_stats *sw_stats = dev->sw_stats;
+
+        sw_stats->tx_retries            += sw_stats_add->tx_retries;
+        sw_stats->tx_failure_drops      += sw_stats_add->tx_failure_drops;
+        sw_stats->tx_mtu_exceeded_drops += sw_stats_add->tx_mtu_exceeded_drops;
+        sw_stats->tx_qos_drops          += sw_stats_add->tx_qos_drops;
+        sw_stats->tx_invalid_hwol_drops += sw_stats_add->tx_invalid_hwol_drops;
+    }
 }
 
 static void
@@ -2314,9 +2562,11 @@ __netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
 {
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
     struct rte_mbuf **cur_pkts = (struct rte_mbuf **) pkts;
-    unsigned int total_pkts = cnt;
-    unsigned int dropped = 0;
+    struct netdev_dpdk_sw_stats sw_stats_add;
+    unsigned int n_packets_to_free = cnt;
+    unsigned int total_packets = cnt;
     int i, retries = 0;
+    int max_retries = VHOST_ENQ_RETRY_MIN;
     int vid = netdev_dpdk_get_vid(dev);
 
     qid = dev->tx_q[qid % netdev->n_txq].map;
@@ -2329,12 +2579,27 @@ __netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
         goto out;
     }
 
-    rte_spinlock_lock(&dev->tx_q[qid].tx_lock);
+    if (OVS_UNLIKELY(!rte_spinlock_trylock(&dev->tx_q[qid].tx_lock))) {
+        COVERAGE_INC(vhost_tx_contention);
+        rte_spinlock_lock(&dev->tx_q[qid].tx_lock);
+    }
+
+    sw_stats_add.tx_invalid_hwol_drops = cnt;
+    if (userspace_tso_enabled()) {
+        cnt = netdev_dpdk_prep_hwol_batch(dev, cur_pkts, cnt);
+    }
 
+    sw_stats_add.tx_invalid_hwol_drops -= cnt;
+    sw_stats_add.tx_mtu_exceeded_drops = cnt;
     cnt = netdev_dpdk_filter_packet_len(dev, cur_pkts, cnt);
+    sw_stats_add.tx_mtu_exceeded_drops -= cnt;
+
     /* Check has QoS has been configured for the netdev */
+    sw_stats_add.tx_qos_drops = cnt;
     cnt = netdev_dpdk_qos_run(dev, cur_pkts, cnt, true);
-    dropped = total_pkts - cnt;
+    sw_stats_add.tx_qos_drops -= cnt;
+
+    n_packets_to_free = cnt;
 
     do {
         int vhost_qid = qid * VIRTIO_QNUM + VIRTIO_RXQ;
@@ -2346,25 +2611,149 @@ __netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
             cnt -= tx_pkts;
             /* Prepare for possible retry.*/
             cur_pkts = &cur_pkts[tx_pkts];
+            if (OVS_UNLIKELY(cnt && !retries)) {
+                /*
+                 * Read max retries as there are packets not sent
+                 * and no retries have already occurred.
+                 */
+                atomic_read_relaxed(&dev->vhost_tx_retries_max, &max_retries);
+            }
         } else {
             /* No packets sent - do not retry.*/
             break;
         }
-    } while (cnt && (retries++ <= VHOST_ENQ_RETRY_NUM));
+    } while (cnt && (retries++ < max_retries));
 
     rte_spinlock_unlock(&dev->tx_q[qid].tx_lock);
 
+    sw_stats_add.tx_failure_drops = cnt;
+    sw_stats_add.tx_retries = MIN(retries, max_retries);
+
     rte_spinlock_lock(&dev->stats_lock);
-    netdev_dpdk_vhost_update_tx_counters(&dev->stats, pkts, total_pkts,
-                                         cnt + dropped);
+    netdev_dpdk_vhost_update_tx_counters(dev, pkts, total_packets,
+                                         &sw_stats_add);
     rte_spinlock_unlock(&dev->stats_lock);
 
 out:
-    for (i = 0; i < total_pkts - dropped; i++) {
+    for (i = 0; i < n_packets_to_free; i++) {
         dp_packet_delete(pkts[i]);
     }
 }
 
+static void
+netdev_dpdk_extbuf_free(void *addr OVS_UNUSED, void *opaque)
+{
+    rte_free(opaque);
+}
+
+static struct rte_mbuf *
+dpdk_pktmbuf_attach_extbuf(struct rte_mbuf *pkt, uint32_t data_len)
+{
+    uint32_t total_len = RTE_PKTMBUF_HEADROOM + data_len;
+    struct rte_mbuf_ext_shared_info *shinfo = NULL;
+    uint16_t buf_len;
+    void *buf;
+
+    if (rte_pktmbuf_tailroom(pkt) >= sizeof *shinfo) {
+        shinfo = rte_pktmbuf_mtod(pkt, struct rte_mbuf_ext_shared_info *);
+    } else {
+        total_len += sizeof *shinfo + sizeof(uintptr_t);
+        total_len = RTE_ALIGN_CEIL(total_len, sizeof(uintptr_t));
+    }
+
+    if (OVS_UNLIKELY(total_len > UINT16_MAX)) {
+        VLOG_ERR("Can't copy packet: too big %u", total_len);
+        return NULL;
+    }
+
+    buf_len = total_len;
+    buf = rte_malloc(NULL, buf_len, RTE_CACHE_LINE_SIZE);
+    if (OVS_UNLIKELY(buf == NULL)) {
+        VLOG_ERR("Failed to allocate memory using rte_malloc: %u", buf_len);
+        return NULL;
+    }
+
+    /* Initialize shinfo. */
+    if (shinfo) {
+        shinfo->free_cb = netdev_dpdk_extbuf_free;
+        shinfo->fcb_opaque = buf;
+        rte_mbuf_ext_refcnt_set(shinfo, 1);
+    } else {
+        shinfo = rte_pktmbuf_ext_shinfo_init_helper(buf, &buf_len,
+                                                    netdev_dpdk_extbuf_free,
+                                                    buf);
+        if (OVS_UNLIKELY(shinfo == NULL)) {
+            rte_free(buf);
+            VLOG_ERR("Failed to initialize shared info for mbuf while "
+                     "attempting to attach an external buffer.");
+            return NULL;
+        }
+    }
+
+    rte_pktmbuf_attach_extbuf(pkt, buf, rte_malloc_virt2iova(buf), buf_len,
+                              shinfo);
+    rte_pktmbuf_reset_headroom(pkt);
+
+    return pkt;
+}
+
+static struct rte_mbuf *
+dpdk_pktmbuf_alloc(struct rte_mempool *mp, uint32_t data_len)
+{
+    struct rte_mbuf *pkt = rte_pktmbuf_alloc(mp);
+
+    if (OVS_UNLIKELY(!pkt)) {
+        return NULL;
+    }
+
+    if (rte_pktmbuf_tailroom(pkt) >= data_len) {
+        return pkt;
+    }
+
+    if (dpdk_pktmbuf_attach_extbuf(pkt, data_len)) {
+        return pkt;
+    }
+
+    rte_pktmbuf_free(pkt);
+
+    return NULL;
+}
+
+static struct dp_packet *
+dpdk_copy_dp_packet_to_mbuf(struct rte_mempool *mp, struct dp_packet *pkt_orig)
+{
+    struct rte_mbuf *mbuf_dest;
+    struct dp_packet *pkt_dest;
+    uint32_t pkt_len;
+
+    pkt_len = dp_packet_size(pkt_orig);
+    mbuf_dest = dpdk_pktmbuf_alloc(mp, pkt_len);
+    if (OVS_UNLIKELY(mbuf_dest == NULL)) {
+            return NULL;
+    }
+
+    pkt_dest = CONTAINER_OF(mbuf_dest, struct dp_packet, mbuf);
+    memcpy(dp_packet_data(pkt_dest), dp_packet_data(pkt_orig), pkt_len);
+    dp_packet_set_size(pkt_dest, pkt_len);
+
+    mbuf_dest->tx_offload = pkt_orig->mbuf.tx_offload;
+    mbuf_dest->packet_type = pkt_orig->mbuf.packet_type;
+    mbuf_dest->ol_flags |= (pkt_orig->mbuf.ol_flags &
+                            ~(EXT_ATTACHED_MBUF | IND_ATTACHED_MBUF));
+
+    memcpy(&pkt_dest->l2_pad_size, &pkt_orig->l2_pad_size,
+           sizeof(struct dp_packet) - offsetof(struct dp_packet, l2_pad_size));
+
+    if (mbuf_dest->ol_flags & PKT_TX_L4_MASK) {
+        mbuf_dest->l2_len = (char *)dp_packet_l3(pkt_dest)
+                                - (char *)dp_packet_eth(pkt_dest);
+        mbuf_dest->l3_len = (char *)dp_packet_l4(pkt_dest)
+                                - (char *) dp_packet_l3(pkt_dest);
+    }
+
+    return pkt_dest;
+}
+
 /* Tx function. Transmit packets indefinitely */
 static void
 dpdk_do_tx_copy(struct netdev *netdev, int qid, struct dp_packet_batch *batch)
@@ -2378,15 +2767,19 @@ dpdk_do_tx_copy(struct netdev *netdev, int qid, struct dp_packet_batch *batch)
     enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST };
 #endif
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-    struct rte_mbuf *pkts[PKT_ARRAY_SIZE];
+    struct dp_packet *pkts[PKT_ARRAY_SIZE];
+    struct netdev_dpdk_sw_stats *sw_stats = dev->sw_stats;
     uint32_t cnt = batch_cnt;
     uint32_t dropped = 0;
+    uint32_t tx_failure = 0;
+    uint32_t mtu_drops = 0;
+    uint32_t qos_drops = 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;
+        qos_drops = batch_cnt - cnt;
     }
 
     uint32_t txcnt = 0;
@@ -2395,40 +2788,40 @@ dpdk_do_tx_copy(struct netdev *netdev, int qid, struct dp_packet_batch *batch)
         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 %u max_packet_len %d",
-                         size, dev->max_packet_len);
-
-            dropped++;
+        if (size > dev->max_packet_len
+            && !(packet->mbuf.ol_flags & PKT_TX_TCP_SEG)) {
+            VLOG_WARN_RL(&rl, "Too big size %u max_packet_len %d", size,
+                         dev->max_packet_len);
+            mtu_drops++;
             continue;
         }
 
-        pkts[txcnt] = rte_pktmbuf_alloc(dev->dpdk_mp->mp);
+        pkts[txcnt] = dpdk_copy_dp_packet_to_mbuf(dev->dpdk_mp->mp, packet);
         if (OVS_UNLIKELY(!pkts[txcnt])) {
-            dropped += cnt - i;
+            dropped = cnt - i;
             break;
         }
 
-        /* We have to do a copy for now */
-        memcpy(rte_pktmbuf_mtod(pkts[txcnt], void *),
-               dp_packet_data(packet), size);
-        dp_packet_set_size((struct dp_packet *)pkts[txcnt], size);
-
         txcnt++;
     }
 
     if (OVS_LIKELY(txcnt)) {
         if (dev->type == DPDK_DEV_VHOST) {
-            __netdev_dpdk_vhost_send(netdev, qid, (struct dp_packet **) pkts,
-                                     txcnt);
+            __netdev_dpdk_vhost_send(netdev, qid, pkts, txcnt);
         } else {
-            dropped += netdev_dpdk_eth_tx_burst(dev, qid, pkts, txcnt);
+            tx_failure += netdev_dpdk_eth_tx_burst(dev, qid,
+                                                   (struct rte_mbuf **)pkts,
+                                                   txcnt);
         }
     }
 
+    dropped += qos_drops + mtu_drops + tx_failure;
     if (OVS_UNLIKELY(dropped)) {
         rte_spinlock_lock(&dev->stats_lock);
         dev->stats.tx_dropped += dropped;
+        sw_stats->tx_failure_drops += tx_failure;
+        sw_stats->tx_mtu_exceeded_drops += mtu_drops;
+        sw_stats->tx_qos_drops += qos_drops;
         rte_spinlock_unlock(&dev->stats_lock);
     }
 }
@@ -2443,7 +2836,8 @@ netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
         dpdk_do_tx_copy(netdev, qid, batch);
         dp_packet_delete_batch(batch, true);
     } else {
-        __netdev_dpdk_vhost_send(netdev, qid, batch->packets, batch->count);
+        __netdev_dpdk_vhost_send(netdev, qid, batch->packets,
+                                 dp_packet_batch_size(batch));
     }
     return 0;
 }
@@ -2469,19 +2863,34 @@ netdev_dpdk_send__(struct netdev_dpdk *dev, int qid,
         dpdk_do_tx_copy(netdev, qid, batch);
         dp_packet_delete_batch(batch, true);
     } else {
-        int tx_cnt, dropped;
+        struct netdev_dpdk_sw_stats *sw_stats = dev->sw_stats;
+        int dropped;
+        int tx_failure, mtu_drops, qos_drops, hwol_drops;
         int batch_cnt = dp_packet_batch_size(batch);
         struct rte_mbuf **pkts = (struct rte_mbuf **) batch->packets;
 
-        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;
+        hwol_drops = batch_cnt;
+        if (userspace_tso_enabled()) {
+            batch_cnt = netdev_dpdk_prep_hwol_batch(dev, pkts, batch_cnt);
+        }
+        hwol_drops -= batch_cnt;
+        mtu_drops = batch_cnt;
+        batch_cnt = netdev_dpdk_filter_packet_len(dev, pkts, batch_cnt);
+        mtu_drops -= batch_cnt;
+        qos_drops = batch_cnt;
+        batch_cnt = netdev_dpdk_qos_run(dev, pkts, batch_cnt, true);
+        qos_drops -= batch_cnt;
 
-        dropped += netdev_dpdk_eth_tx_burst(dev, qid, pkts, tx_cnt);
+        tx_failure = netdev_dpdk_eth_tx_burst(dev, qid, pkts, batch_cnt);
 
+        dropped = tx_failure + mtu_drops + qos_drops + hwol_drops;
         if (OVS_UNLIKELY(dropped)) {
             rte_spinlock_lock(&dev->stats_lock);
             dev->stats.tx_dropped += dropped;
+            sw_stats->tx_failure_drops += tx_failure;
+            sw_stats->tx_mtu_exceeded_drops += mtu_drops;
+            sw_stats->tx_qos_drops += qos_drops;
+            sw_stats->tx_invalid_hwol_drops += hwol_drops;
             rte_spinlock_unlock(&dev->stats_lock);
         }
     }
@@ -2557,7 +2966,7 @@ netdev_dpdk_set_mtu(struct netdev *netdev, int mtu)
      * 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) {
+        || mtu < RTE_ETHER_MIN_MTU) {
         VLOG_WARN("%s: unsupported MTU %d\n", dev->up.name, mtu);
         return EINVAL;
     }
@@ -2616,51 +3025,41 @@ netdev_dpdk_convert_xstats(struct netdev_stats *stats,
                            const struct rte_eth_xstat_name *names,
                            const unsigned int size)
 {
+/* DPDK XSTATS Counter names definition. */
+#define DPDK_XSTATS \
+    DPDK_XSTAT(multicast,               "rx_multicast_packets"            ) \
+    DPDK_XSTAT(tx_multicast_packets,    "tx_multicast_packets"            ) \
+    DPDK_XSTAT(rx_broadcast_packets,    "rx_broadcast_packets"            ) \
+    DPDK_XSTAT(tx_broadcast_packets,    "tx_broadcast_packets"            ) \
+    DPDK_XSTAT(rx_undersized_errors,    "rx_undersized_errors"            ) \
+    DPDK_XSTAT(rx_oversize_errors,      "rx_oversize_errors"              ) \
+    DPDK_XSTAT(rx_fragmented_errors,    "rx_fragmented_errors"            ) \
+    DPDK_XSTAT(rx_jabber_errors,        "rx_jabber_errors"                ) \
+    DPDK_XSTAT(rx_1_to_64_packets,      "rx_size_64_packets"              ) \
+    DPDK_XSTAT(rx_65_to_127_packets,    "rx_size_65_to_127_packets"       ) \
+    DPDK_XSTAT(rx_128_to_255_packets,   "rx_size_128_to_255_packets"      ) \
+    DPDK_XSTAT(rx_256_to_511_packets,   "rx_size_256_to_511_packets"      ) \
+    DPDK_XSTAT(rx_512_to_1023_packets,  "rx_size_512_to_1023_packets"     ) \
+    DPDK_XSTAT(rx_1024_to_1522_packets, "rx_size_1024_to_1522_packets"    ) \
+    DPDK_XSTAT(rx_1523_to_max_packets,  "rx_size_1523_to_max_packets"     ) \
+    DPDK_XSTAT(tx_1_to_64_packets,      "tx_size_64_packets"              ) \
+    DPDK_XSTAT(tx_65_to_127_packets,    "tx_size_65_to_127_packets"       ) \
+    DPDK_XSTAT(tx_128_to_255_packets,   "tx_size_128_to_255_packets"      ) \
+    DPDK_XSTAT(tx_256_to_511_packets,   "tx_size_256_to_511_packets"      ) \
+    DPDK_XSTAT(tx_512_to_1023_packets,  "tx_size_512_to_1023_packets"     ) \
+    DPDK_XSTAT(tx_1024_to_1522_packets, "tx_size_1024_to_1522_packets"    ) \
+    DPDK_XSTAT(tx_1523_to_max_packets,  "tx_size_1523_to_max_packets"     )
+
     for (unsigned int i = 0; i < size; i++) {
-        if (strcmp(XSTAT_RX_64_PACKETS, names[i].name) == 0) {
-            stats->rx_1_to_64_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_65_TO_127_PACKETS, names[i].name) == 0) {
-            stats->rx_65_to_127_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_128_TO_255_PACKETS, names[i].name) == 0) {
-            stats->rx_128_to_255_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_256_TO_511_PACKETS, names[i].name) == 0) {
-            stats->rx_256_to_511_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_512_TO_1023_PACKETS, names[i].name) == 0) {
-            stats->rx_512_to_1023_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_1024_TO_1522_PACKETS, names[i].name) == 0) {
-            stats->rx_1024_to_1522_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_1523_TO_MAX_PACKETS, names[i].name) == 0) {
-            stats->rx_1523_to_max_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_64_PACKETS, names[i].name) == 0) {
-            stats->tx_1_to_64_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_65_TO_127_PACKETS, names[i].name) == 0) {
-            stats->tx_65_to_127_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_128_TO_255_PACKETS, names[i].name) == 0) {
-            stats->tx_128_to_255_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_256_TO_511_PACKETS, names[i].name) == 0) {
-            stats->tx_256_to_511_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_512_TO_1023_PACKETS, names[i].name) == 0) {
-            stats->tx_512_to_1023_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_1024_TO_1522_PACKETS, names[i].name) == 0) {
-            stats->tx_1024_to_1522_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_1523_TO_MAX_PACKETS, names[i].name) == 0) {
-            stats->tx_1523_to_max_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_MULTICAST_PACKETS, names[i].name) == 0) {
-            stats->multicast = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_MULTICAST_PACKETS, names[i].name) == 0) {
-            stats->tx_multicast_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_BROADCAST_PACKETS, names[i].name) == 0) {
-            stats->rx_broadcast_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_TX_BROADCAST_PACKETS, names[i].name) == 0) {
-            stats->tx_broadcast_packets = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_UNDERSIZED_ERRORS, names[i].name) == 0) {
-            stats->rx_undersized_errors = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_FRAGMENTED_ERRORS, names[i].name) == 0) {
-            stats->rx_fragmented_errors = xstats[i].value;
-        } else if (strcmp(XSTAT_RX_JABBER_ERRORS, names[i].name) == 0) {
-            stats->rx_jabber_errors = xstats[i].value;
+#define DPDK_XSTAT(MEMBER, NAME) \
+        if (strcmp(NAME, names[i].name) == 0) {   \
+            stats->MEMBER = xstats[i].value;      \
+            continue;                             \
         }
+        DPDK_XSTATS;
+#undef DPDK_XSTAT
     }
+#undef DPDK_XSTATS
 }
 
 static int
@@ -2749,7 +3148,9 @@ netdev_dpdk_get_custom_stats(const struct netdev *netdev,
 
     uint32_t i;
     struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-    int rte_xstats_ret;
+    int rte_xstats_ret, sw_stats_size;
+
+    netdev_dpdk_get_sw_custom_stats(netdev, custom_stats);
 
     ovs_mutex_lock(&dev->mutex);
 
@@ -2764,23 +3165,22 @@ netdev_dpdk_get_custom_stats(const struct netdev *netdev,
         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));
+            sw_stats_size = custom_stats->size;
+            custom_stats->size += rte_xstats_ret;
+            custom_stats->counters = xrealloc(custom_stats->counters,
+                                              custom_stats->size *
+                                              sizeof *custom_stats->counters);
 
             for (i = 0; i < rte_xstats_ret; i++) {
-                ovs_strlcpy(custom_stats->counters[i].name,
+                ovs_strlcpy(custom_stats->counters[sw_stats_size + 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];
+                custom_stats->counters[sw_stats_size + 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);
@@ -2794,6 +3194,56 @@ netdev_dpdk_get_custom_stats(const struct netdev *netdev,
     return 0;
 }
 
+static int
+netdev_dpdk_get_sw_custom_stats(const struct netdev *netdev,
+                                struct netdev_custom_stats *custom_stats)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    int i, n;
+
+#define SW_CSTATS                    \
+    SW_CSTAT(tx_retries)             \
+    SW_CSTAT(tx_failure_drops)       \
+    SW_CSTAT(tx_mtu_exceeded_drops)  \
+    SW_CSTAT(tx_qos_drops)           \
+    SW_CSTAT(rx_qos_drops)           \
+    SW_CSTAT(tx_invalid_hwol_drops)
+
+#define SW_CSTAT(NAME) + 1
+    custom_stats->size = SW_CSTATS;
+#undef SW_CSTAT
+    custom_stats->counters = xcalloc(custom_stats->size,
+                                     sizeof *custom_stats->counters);
+
+    ovs_mutex_lock(&dev->mutex);
+
+    rte_spinlock_lock(&dev->stats_lock);
+    i = 0;
+#define SW_CSTAT(NAME) \
+    custom_stats->counters[i++].value = dev->sw_stats->NAME;
+    SW_CSTATS;
+#undef SW_CSTAT
+    rte_spinlock_unlock(&dev->stats_lock);
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    i = 0;
+    n = 0;
+#define SW_CSTAT(NAME) \
+    if (custom_stats->counters[i].value != UINT64_MAX) {                   \
+        ovs_strlcpy(custom_stats->counters[n].name,                        \
+                    "ovs_"#NAME, NETDEV_CUSTOM_STATS_NAME_SIZE);           \
+        custom_stats->counters[n].value = custom_stats->counters[i].value; \
+        n++;                                                               \
+    }                                                                      \
+    i++;
+    SW_CSTATS;
+#undef SW_CSTAT
+
+    custom_stats->size = n;
+    return 0;
+}
+
 static int
 netdev_dpdk_get_features(const struct netdev *netdev,
                          enum netdev_features *current,
@@ -3492,8 +3942,8 @@ new_device(int vid)
                 newnode = dev->socket_id;
             }
 
-            if (dev->requested_n_txq != qp_num
-                || dev->requested_n_rxq != qp_num
+            if (dev->requested_n_txq < qp_num
+                || dev->requested_n_rxq < qp_num
                 || dev->requested_socket_id != newnode) {
                 dev->requested_socket_id = newnode;
                 dev->requested_n_rxq = qp_num;
@@ -3563,6 +4013,8 @@ destroy_device(int vid)
             ovs_mutex_lock(&dev->mutex);
             dev->vhost_reconfigured = false;
             ovsrcu_index_set(&dev->vid, -1);
+            memset(dev->vhost_rxq_enabled, 0,
+                   dev->up.n_rxq * sizeof *dev->vhost_rxq_enabled);
             netdev_dpdk_txq_map_clear(dev);
 
             netdev_change_seq_changed(&dev->up);
@@ -3597,24 +4049,30 @@ vring_state_changed(int vid, uint16_t queue_id, int enable)
     struct netdev_dpdk *dev;
     bool exists = false;
     int qid = queue_id / VIRTIO_QNUM;
+    bool is_rx = (queue_id % VIRTIO_QNUM) == VIRTIO_TXQ;
     char ifname[IF_NAME_SZ];
 
     rte_vhost_get_ifname(vid, ifname, sizeof ifname);
 
-    if (queue_id % VIRTIO_QNUM == VIRTIO_TXQ) {
-        return 0;
-    }
-
     ovs_mutex_lock(&dpdk_mutex);
     LIST_FOR_EACH (dev, list_node, &dpdk_list) {
         ovs_mutex_lock(&dev->mutex);
         if (nullable_string_is_equal(ifname, dev->vhost_id)) {
-            if (enable) {
-                dev->tx_q[qid].map = qid;
+            if (is_rx) {
+                bool old_state = dev->vhost_rxq_enabled[qid];
+
+                dev->vhost_rxq_enabled[qid] = enable != 0;
+                if (old_state != dev->vhost_rxq_enabled[qid]) {
+                    netdev_change_seq_changed(&dev->up);
+                }
             } else {
-                dev->tx_q[qid].map = OVS_VHOST_QUEUE_DISABLED;
+                if (enable) {
+                    dev->tx_q[qid].map = qid;
+                } else {
+                    dev->tx_q[qid].map = OVS_VHOST_QUEUE_DISABLED;
+                }
+                netdev_dpdk_remap_txqs(dev);
             }
-            netdev_dpdk_remap_txqs(dev);
             exists = true;
             ovs_mutex_unlock(&dev->mutex);
             break;
@@ -3624,9 +4082,9 @@ vring_state_changed(int vid, uint16_t queue_id, int enable)
     ovs_mutex_unlock(&dpdk_mutex);
 
     if (exists) {
-        VLOG_INFO("State of queue %d ( tx_qid %d ) of vhost device '%s' "
-                  "changed to \'%s\'", queue_id, qid, ifname,
-                  (enable == 1) ? "enabled" : "disabled");
+        VLOG_INFO("State of queue %d ( %s_qid %d ) of vhost device '%s' "
+                  "changed to \'%s\'", queue_id, is_rx == true ? "rx" : "tx",
+                  qid, ifname, (enable == 1) ? "enabled" : "disabled");
     } else {
         VLOG_INFO("vHost Device '%s' not found", ifname);
         return -1;
@@ -3635,6 +4093,54 @@ vring_state_changed(int vid, uint16_t queue_id, int enable)
     return 0;
 }
 
+static void
+destroy_connection(int vid)
+{
+    struct netdev_dpdk *dev;
+    char ifname[IF_NAME_SZ];
+    bool exists = false;
+
+    rte_vhost_get_ifname(vid, ifname, sizeof ifname);
+
+    ovs_mutex_lock(&dpdk_mutex);
+    LIST_FOR_EACH (dev, list_node, &dpdk_list) {
+        ovs_mutex_lock(&dev->mutex);
+        if (nullable_string_is_equal(ifname, dev->vhost_id)) {
+            uint32_t qp_num = NR_QUEUE;
+
+            if (netdev_dpdk_get_vid(dev) >= 0) {
+                VLOG_ERR("Connection on socket '%s' destroyed while vhost "
+                         "device still attached.", dev->vhost_id);
+            }
+
+            /* Restore the number of queue pairs to default. */
+            if (dev->requested_n_txq != qp_num
+                || dev->requested_n_rxq != qp_num) {
+                dev->requested_n_rxq = qp_num;
+                dev->requested_n_txq = qp_num;
+                netdev_request_reconfigure(&dev->up);
+            }
+            ovs_mutex_unlock(&dev->mutex);
+            exists = true;
+            break;
+        }
+        ovs_mutex_unlock(&dev->mutex);
+    }
+    ovs_mutex_unlock(&dpdk_mutex);
+
+    if (exists) {
+        VLOG_INFO("vHost Device '%s' connection has been destroyed", ifname);
+    } else {
+        VLOG_INFO("vHost Device '%s' not found", ifname);
+    }
+}
+
+static
+void vhost_guest_notified(int vid OVS_UNUSED)
+{
+    COVERAGE_INC(vhost_notification);
+}
+
 /*
  * Retrieve the DPDK virtio device ID (vid) associated with a vhostuser
  * or vhostuserclient netdev.
@@ -3668,6 +4174,8 @@ netdev_dpdk_class_init(void)
     /* This function can be called for different classes.  The initialization
      * needs to be done only once */
     if (ovsthread_once_start(&once)) {
+        int ret;
+
         ovs_thread_create("dpdk_watchdog", dpdk_watchdog, NULL);
         unixctl_command_register("netdev-dpdk/set-admin-state",
                                  "[netdev] up|down", 1, 2,
@@ -3681,133 +4189,20 @@ netdev_dpdk_class_init(void)
                                  "[netdev]", 0, 1,
                                  netdev_dpdk_get_mempool_info, NULL);
 
-        ovsthread_once_done(&once);
-    }
-
-    return 0;
-}
-
-/* Client Rings */
-
-static int
-dpdk_ring_create(const char dev_name[], unsigned int port_no,
-                 dpdk_port_t *eth_port_id)
-{
-    struct dpdk_ring *ring_pair;
-    char *ring_name;
-    int port_id;
-
-    ring_pair = dpdk_rte_mzalloc(sizeof *ring_pair);
-    if (!ring_pair) {
-        return ENOMEM;
-    }
-
-    /* XXX: Add support for multiquque ring. */
-    ring_name = xasprintf("%s_tx", dev_name);
-
-    /* Create single producer tx ring, netdev does explicit locking. */
-    ring_pair->cring_tx = rte_ring_create(ring_name, DPDK_RING_SIZE, SOCKET0,
-                                        RING_F_SP_ENQ);
-    free(ring_name);
-    if (ring_pair->cring_tx == NULL) {
-        rte_free(ring_pair);
-        return ENOMEM;
-    }
-
-    ring_name = xasprintf("%s_rx", dev_name);
-
-    /* Create single consumer rx ring, netdev does explicit locking. */
-    ring_pair->cring_rx = rte_ring_create(ring_name, DPDK_RING_SIZE, SOCKET0,
-                                        RING_F_SC_DEQ);
-    free(ring_name);
-    if (ring_pair->cring_rx == NULL) {
-        rte_free(ring_pair);
-        return ENOMEM;
-    }
-
-    port_id = rte_eth_from_rings(dev_name, &ring_pair->cring_rx, 1,
-                                 &ring_pair->cring_tx, 1, SOCKET0);
-
-    if (port_id < 0) {
-        rte_free(ring_pair);
-        return ENODEV;
-    }
-
-    ring_pair->user_port_id = port_no;
-    ring_pair->eth_port_id = port_id;
-    *eth_port_id = port_id;
-
-    ovs_list_push_back(&dpdk_ring_list, &ring_pair->list_node);
-
-    return 0;
-}
-
-static int
-dpdk_ring_open(const char dev_name[], dpdk_port_t *eth_port_id)
-    OVS_REQUIRES(dpdk_mutex)
-{
-    struct dpdk_ring *ring_pair;
-    unsigned int port_no;
-    int err = 0;
-
-    /* Names always start with "dpdkr" */
-    err = dpdk_dev_parse_name(dev_name, "dpdkr", &port_no);
-    if (err) {
-        return err;
-    }
-
-    /* Look through our list to find the device */
-    LIST_FOR_EACH (ring_pair, list_node, &dpdk_ring_list) {
-         if (ring_pair->user_port_id == port_no) {
-            VLOG_INFO("Found dpdk ring device %s:", dev_name);
-            /* Really all that is needed */
-            *eth_port_id = ring_pair->eth_port_id;
-            return 0;
-         }
-    }
-    /* Need to create the device rings */
-    return dpdk_ring_create(dev_name, port_no, eth_port_id);
-}
-
-static int
-netdev_dpdk_ring_send(struct netdev *netdev, int qid,
-                      struct dp_packet_batch *batch, bool concurrent_txq)
-{
-    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-    struct dp_packet *packet;
+        ret = rte_eth_dev_callback_register(RTE_ETH_ALL,
+                                            RTE_ETH_EVENT_INTR_RESET,
+                                            dpdk_eth_event_callback, NULL);
+        if (ret != 0) {
+            VLOG_ERR("Ethernet device callback register error: %s",
+                     rte_strerror(-ret));
+        }
 
-    /* When using 'dpdkr' and sending to a DPDK ring, we want to ensure that
-     * the offload fields are 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 or revalidating the checksums. */
-    DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
-        dp_packet_reset_offload(packet);
+        ovsthread_once_done(&once);
     }
 
-    netdev_dpdk_send__(dev, qid, batch, concurrent_txq);
     return 0;
 }
 
-static int
-netdev_dpdk_ring_construct(struct netdev *netdev)
-{
-    dpdk_port_t port_no = 0;
-    int err = 0;
-
-    ovs_mutex_lock(&dpdk_mutex);
-
-    err = dpdk_ring_open(netdev->name, &port_no);
-    if (err) {
-        goto unlock_dpdk;
-    }
-
-    err = common_construct(netdev, port_no, DPDK_DEV_ETH,
-                           rte_eth_dev_socket_id(port_no));
-unlock_dpdk:
-    ovs_mutex_unlock(&dpdk_mutex);
-    return err;
-}
-
 /* QoS Functions */
 
 /*
@@ -3921,6 +4316,164 @@ netdev_dpdk_set_qos(struct netdev *netdev, const char *type,
     return error;
 }
 
+static int
+netdev_dpdk_get_queue(const struct netdev *netdev, uint32_t queue_id,
+                      struct smap *details)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    struct qos_conf *qos_conf;
+    int error = 0;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    qos_conf = ovsrcu_get_protected(struct qos_conf *, &dev->qos_conf);
+    if (!qos_conf || !qos_conf->ops || !qos_conf->ops->qos_queue_get) {
+        error = EOPNOTSUPP;
+    } else {
+        error = qos_conf->ops->qos_queue_get(details, queue_id, qos_conf);
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return error;
+}
+
+static int
+netdev_dpdk_set_queue(struct netdev *netdev, uint32_t queue_id,
+                      const struct smap *details)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    struct qos_conf *qos_conf;
+    int error = 0;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    qos_conf = ovsrcu_get_protected(struct qos_conf *, &dev->qos_conf);
+    if (!qos_conf || !qos_conf->ops || !qos_conf->ops->qos_queue_construct) {
+        error = EOPNOTSUPP;
+    } else {
+        error = qos_conf->ops->qos_queue_construct(details, queue_id,
+                                                   qos_conf);
+    }
+
+    if (error && error != EOPNOTSUPP) {
+        VLOG_ERR("Failed to set QoS queue %d on port %s: %s",
+                 queue_id, netdev_get_name(netdev), rte_strerror(error));
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return error;
+}
+
+static int
+netdev_dpdk_delete_queue(struct netdev *netdev, uint32_t queue_id)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    struct qos_conf *qos_conf;
+    int error = 0;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    qos_conf = ovsrcu_get_protected(struct qos_conf *, &dev->qos_conf);
+    if (qos_conf && qos_conf->ops && qos_conf->ops->qos_queue_destruct) {
+        qos_conf->ops->qos_queue_destruct(qos_conf, queue_id);
+    } else {
+        error =  EOPNOTSUPP;
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return error;
+}
+
+static int
+netdev_dpdk_get_queue_stats(const struct netdev *netdev, uint32_t queue_id,
+                            struct netdev_queue_stats *stats)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    struct qos_conf *qos_conf;
+    int error = 0;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    qos_conf = ovsrcu_get_protected(struct qos_conf *, &dev->qos_conf);
+    if (qos_conf && qos_conf->ops && qos_conf->ops->qos_queue_get_stats) {
+        qos_conf->ops->qos_queue_get_stats(qos_conf, queue_id, stats);
+    } else {
+        error = EOPNOTSUPP;
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return error;
+}
+
+static int
+netdev_dpdk_queue_dump_start(const struct netdev *netdev, void **statep)
+{
+    int error = 0;
+    struct qos_conf *qos_conf;
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+
+    ovs_mutex_lock(&dev->mutex);
+
+    qos_conf = ovsrcu_get_protected(struct qos_conf *, &dev->qos_conf);
+    if (qos_conf && qos_conf->ops
+        && qos_conf->ops->qos_queue_dump_state_init) {
+        struct netdev_dpdk_queue_state *state;
+
+        *statep = state = xmalloc(sizeof *state);
+        error = qos_conf->ops->qos_queue_dump_state_init(qos_conf, state);
+    } else {
+        error = EOPNOTSUPP;
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return error;
+}
+
+static int
+netdev_dpdk_queue_dump_next(const struct netdev *netdev, void *state_,
+                            uint32_t *queue_idp, struct smap *details)
+{
+    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
+    struct netdev_dpdk_queue_state *state = state_;
+    struct qos_conf *qos_conf;
+    int error = EOF;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    while (state->cur_queue < state->n_queues) {
+        uint32_t queue_id = state->queues[state->cur_queue++];
+
+        qos_conf = ovsrcu_get_protected(struct qos_conf *, &dev->qos_conf);
+        if (qos_conf && qos_conf->ops && qos_conf->ops->qos_queue_get) {
+            *queue_idp = queue_id;
+            error = qos_conf->ops->qos_queue_get(details, queue_id, qos_conf);
+            break;
+        }
+    }
+
+    ovs_mutex_unlock(&dev->mutex);
+
+    return error;
+}
+
+static int
+netdev_dpdk_queue_dump_done(const struct netdev *netdev OVS_UNUSED,
+                            void *state_)
+{
+    struct netdev_dpdk_queue_state *state = state_;
+
+    free(state->queues);
+    free(state);
+    return 0;
+}
+
+
+
 /* egress-policer details */
 
 struct egress_policer {
@@ -4010,20 +4563,340 @@ egress_policer_run(struct qos_conf *conf, struct rte_mbuf **pkts, int pkt_cnt,
     struct egress_policer *policer =
         CONTAINER_OF(conf, struct egress_policer, qos_conf);
 
-    cnt = netdev_dpdk_policer_run(&policer->egress_meter,
-                                  &policer->egress_prof, pkts,
-                                  pkt_cnt, should_steal);
+    cnt = srtcm_policer_run_single_packet(&policer->egress_meter,
+                                          &policer->egress_prof, pkts,
+                                          pkt_cnt, should_steal);
 
     return cnt;
 }
 
 static const struct dpdk_qos_ops egress_policer_ops = {
-    "egress-policer",    /* qos_name */
-    egress_policer_qos_construct,
-    egress_policer_qos_destruct,
-    egress_policer_qos_get,
-    egress_policer_qos_is_equal,
-    egress_policer_run
+    .qos_name = "egress-policer",    /* qos_name */
+    .qos_construct = egress_policer_qos_construct,
+    .qos_destruct = egress_policer_qos_destruct,
+    .qos_get = egress_policer_qos_get,
+    .qos_is_equal = egress_policer_qos_is_equal,
+    .qos_run = egress_policer_run
+};
+
+/* trtcm-policer details */
+
+struct trtcm_policer {
+    struct qos_conf qos_conf;
+    struct rte_meter_trtcm_rfc4115_params meter_params;
+    struct rte_meter_trtcm_rfc4115_profile meter_profile;
+    struct rte_meter_trtcm_rfc4115 meter;
+    struct netdev_queue_stats stats;
+    struct hmap queues;
+};
+
+struct trtcm_policer_queue {
+    struct hmap_node hmap_node;
+    uint32_t queue_id;
+    struct rte_meter_trtcm_rfc4115_params meter_params;
+    struct rte_meter_trtcm_rfc4115_profile meter_profile;
+    struct rte_meter_trtcm_rfc4115 meter;
+    struct netdev_queue_stats stats;
+};
+
+static void
+trtcm_policer_details_to_param(const struct smap *details,
+                               struct rte_meter_trtcm_rfc4115_params *params)
+{
+    memset(params, 0, sizeof *params);
+    params->cir = smap_get_ullong(details, "cir", 0);
+    params->eir = smap_get_ullong(details, "eir", 0);
+    params->cbs = smap_get_ullong(details, "cbs", 0);
+    params->ebs = smap_get_ullong(details, "ebs", 0);
+}
+
+static void
+trtcm_policer_param_to_detail(
+    const struct rte_meter_trtcm_rfc4115_params *params,
+    struct smap *details)
+{
+    smap_add_format(details, "cir", "%"PRIu64, params->cir);
+    smap_add_format(details, "eir", "%"PRIu64, params->eir);
+    smap_add_format(details, "cbs", "%"PRIu64, params->cbs);
+    smap_add_format(details, "ebs", "%"PRIu64, params->ebs);
+}
+
+
+static int
+trtcm_policer_qos_construct(const struct smap *details,
+                            struct qos_conf **conf)
+{
+    struct trtcm_policer *policer;
+    int err = 0;
+
+    policer = xmalloc(sizeof *policer);
+    qos_conf_init(&policer->qos_conf, &trtcm_policer_ops);
+    trtcm_policer_details_to_param(details, &policer->meter_params);
+    err = rte_meter_trtcm_rfc4115_profile_config(&policer->meter_profile,
+                                                 &policer->meter_params);
+    if (!err) {
+        err = rte_meter_trtcm_rfc4115_config(&policer->meter,
+                                             &policer->meter_profile);
+    }
+
+    if (!err) {
+        *conf = &policer->qos_conf;
+        memset(&policer->stats, 0, sizeof policer->stats);
+        hmap_init(&policer->queues);
+    } else {
+        free(policer);
+        *conf = NULL;
+        err = -err;
+    }
+
+    return err;
+}
+
+static void
+trtcm_policer_qos_destruct(struct qos_conf *conf)
+{
+    struct trtcm_policer_queue *queue, *next_queue;
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    HMAP_FOR_EACH_SAFE (queue, next_queue, hmap_node, &policer->queues) {
+        hmap_remove(&policer->queues, &queue->hmap_node);
+        free(queue);
+    }
+    hmap_destroy(&policer->queues);
+    free(policer);
+}
+
+static int
+trtcm_policer_qos_get(const struct qos_conf *conf, struct smap *details)
+{
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    trtcm_policer_param_to_detail(&policer->meter_params, details);
+    return 0;
+}
+
+static bool
+trtcm_policer_qos_is_equal(const struct qos_conf *conf,
+                           const struct smap *details)
+{
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+    struct rte_meter_trtcm_rfc4115_params params;
+
+    trtcm_policer_details_to_param(details, &params);
+
+    return !memcmp(&params, &policer->meter_params, sizeof params);
+}
+
+static struct trtcm_policer_queue *
+trtcm_policer_qos_find_queue(struct trtcm_policer *policer, uint32_t queue_id)
+{
+    struct trtcm_policer_queue *queue;
+    HMAP_FOR_EACH_WITH_HASH (queue, hmap_node, hash_2words(queue_id, 0),
+                             &policer->queues) {
+        if (queue->queue_id == queue_id) {
+            return queue;
+        }
+    }
+    return NULL;
+}
+
+static inline bool
+trtcm_policer_run_single_packet(struct trtcm_policer *policer,
+                                struct rte_mbuf *pkt, uint64_t time)
+{
+    enum rte_color pkt_color;
+    struct trtcm_policer_queue *queue;
+    uint32_t pkt_len = rte_pktmbuf_pkt_len(pkt) - sizeof(struct rte_ether_hdr);
+    struct dp_packet *dpkt = CONTAINER_OF(pkt, struct dp_packet, mbuf);
+
+    queue = trtcm_policer_qos_find_queue(policer, dpkt->md.skb_priority);
+    if (!queue) {
+        /* If no queue is found, use the default queue, which MUST exist. */
+        queue = trtcm_policer_qos_find_queue(policer, 0);
+        if (!queue) {
+            return false;
+        }
+    }
+
+    pkt_color = rte_meter_trtcm_rfc4115_color_blind_check(&queue->meter,
+                                                          &queue->meter_profile,
+                                                          time,
+                                                          pkt_len);
+
+    if (pkt_color == RTE_COLOR_RED) {
+        queue->stats.tx_errors++;
+    } else {
+        queue->stats.tx_bytes += pkt_len;
+        queue->stats.tx_packets++;
+    }
+
+    pkt_color = rte_meter_trtcm_rfc4115_color_aware_check(&policer->meter,
+                                                     &policer->meter_profile,
+                                                     time, pkt_len,
+                                                     pkt_color);
+
+    if (pkt_color == RTE_COLOR_RED) {
+        policer->stats.tx_errors++;
+        return false;
+    }
+
+    policer->stats.tx_bytes += pkt_len;
+    policer->stats.tx_packets++;
+    return true;
+}
+
+static int
+trtcm_policer_run(struct qos_conf *conf, struct rte_mbuf **pkts, int pkt_cnt,
+                  bool should_steal)
+{
+    int i = 0;
+    int cnt = 0;
+    struct rte_mbuf *pkt = NULL;
+    uint64_t current_time = rte_rdtsc();
+
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    for (i = 0; i < pkt_cnt; i++) {
+        pkt = pkts[i];
+
+        if (trtcm_policer_run_single_packet(policer, pkt, current_time)) {
+            if (cnt != i) {
+                pkts[cnt] = pkt;
+            }
+            cnt++;
+        } else {
+            if (should_steal) {
+                rte_pktmbuf_free(pkt);
+            }
+        }
+    }
+    return cnt;
+}
+
+static int
+trtcm_policer_qos_queue_construct(const struct smap *details,
+                                  uint32_t queue_id, struct qos_conf *conf)
+{
+    int err = 0;
+    struct trtcm_policer_queue *queue;
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    queue = trtcm_policer_qos_find_queue(policer, queue_id);
+    if (!queue) {
+        queue = xmalloc(sizeof *queue);
+        queue->queue_id = queue_id;
+        memset(&queue->stats, 0, sizeof queue->stats);
+        queue->stats.created = time_msec();
+        hmap_insert(&policer->queues, &queue->hmap_node,
+                    hash_2words(queue_id, 0));
+    }
+    if (queue_id == 0 && smap_is_empty(details)) {
+        /* No default queue configured, use port values */
+        memcpy(&queue->meter_params, &policer->meter_params,
+               sizeof queue->meter_params);
+    } else {
+        trtcm_policer_details_to_param(details, &queue->meter_params);
+    }
+
+    err = rte_meter_trtcm_rfc4115_profile_config(&queue->meter_profile,
+                                                 &queue->meter_params);
+
+    if (!err) {
+        err = rte_meter_trtcm_rfc4115_config(&queue->meter,
+                                             &queue->meter_profile);
+    }
+    if (err) {
+        hmap_remove(&policer->queues, &queue->hmap_node);
+        free(queue);
+        err = -err;
+    }
+    return err;
+}
+
+static void
+trtcm_policer_qos_queue_destruct(struct qos_conf *conf, uint32_t queue_id)
+{
+    struct trtcm_policer_queue *queue;
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    queue = trtcm_policer_qos_find_queue(policer, queue_id);
+    if (queue) {
+        hmap_remove(&policer->queues, &queue->hmap_node);
+        free(queue);
+    }
+}
+
+static int
+trtcm_policer_qos_queue_get(struct smap *details, uint32_t queue_id,
+                            const struct qos_conf *conf)
+{
+    struct trtcm_policer_queue *queue;
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    queue = trtcm_policer_qos_find_queue(policer, queue_id);
+    if (!queue) {
+        return EINVAL;
+    }
+
+    trtcm_policer_param_to_detail(&queue->meter_params, details);
+    return 0;
+}
+
+static int
+trtcm_policer_qos_queue_get_stats(const struct qos_conf *conf,
+                                  uint32_t queue_id,
+                                  struct netdev_queue_stats *stats)
+{
+    struct trtcm_policer_queue *queue;
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    queue = trtcm_policer_qos_find_queue(policer, queue_id);
+    if (!queue) {
+        return EINVAL;
+    }
+    memcpy(stats, &queue->stats, sizeof *stats);
+    return 0;
+}
+
+static int
+trtcm_policer_qos_queue_dump_state_init(const struct qos_conf *conf,
+                                        struct netdev_dpdk_queue_state *state)
+{
+    uint32_t i = 0;
+    struct trtcm_policer_queue *queue;
+    struct trtcm_policer *policer = CONTAINER_OF(conf, struct trtcm_policer,
+                                                 qos_conf);
+
+    state->n_queues = hmap_count(&policer->queues);
+    state->cur_queue = 0;
+    state->queues = xmalloc(state->n_queues * sizeof *state->queues);
+
+    HMAP_FOR_EACH (queue, hmap_node, &policer->queues) {
+        state->queues[i++] = queue->queue_id;
+    }
+    return 0;
+}
+
+static const struct dpdk_qos_ops trtcm_policer_ops = {
+    .qos_name = "trtcm-policer",
+    .qos_construct = trtcm_policer_qos_construct,
+    .qos_destruct = trtcm_policer_qos_destruct,
+    .qos_get = trtcm_policer_qos_get,
+    .qos_is_equal = trtcm_policer_qos_is_equal,
+    .qos_run = trtcm_policer_run,
+    .qos_queue_construct = trtcm_policer_qos_queue_construct,
+    .qos_queue_destruct = trtcm_policer_qos_queue_destruct,
+    .qos_queue_get = trtcm_policer_qos_queue_get,
+    .qos_queue_get_stats = trtcm_policer_qos_queue_get_stats,
+    .qos_queue_dump_state_init = trtcm_policer_qos_queue_dump_state_init
 };
 
 static int
@@ -4041,13 +4914,20 @@ netdev_dpdk_reconfigure(struct netdev *netdev)
         && dev->rxq_size == dev->requested_rxq_size
         && dev->txq_size == dev->requested_txq_size
         && dev->socket_id == dev->requested_socket_id
-        && dev->started) {
+        && dev->started && !dev->reset_needed) {
         /* Reconfiguration is unnecessary */
 
         goto out;
     }
 
-    rte_eth_dev_stop(dev->port_id);
+    if (dev->reset_needed) {
+        rte_eth_dev_reset(dev->port_id);
+        if_notifier_manual_report();
+        dev->reset_needed = false;
+    } else {
+        rte_eth_dev_stop(dev->port_id);
+    }
+
     dev->started = false;
 
     err = netdev_dpdk_mempool_configure(dev);
@@ -4065,6 +4945,16 @@ netdev_dpdk_reconfigure(struct netdev *netdev)
 
     rte_free(dev->tx_q);
     err = dpdk_eth_dev_init(dev);
+    if (dev->hw_ol_features & NETDEV_TX_TSO_OFFLOAD) {
+        netdev->ol_flags |= NETDEV_TX_OFFLOAD_TCP_TSO;
+        netdev->ol_flags |= NETDEV_TX_OFFLOAD_TCP_CKSUM;
+        netdev->ol_flags |= NETDEV_TX_OFFLOAD_UDP_CKSUM;
+        netdev->ol_flags |= NETDEV_TX_OFFLOAD_IPV4_CKSUM;
+        if (dev->hw_ol_features & NETDEV_TX_SCTP_CHECKSUM_OFFLOAD) {
+            netdev->ol_flags |= NETDEV_TX_OFFLOAD_SCTP_CKSUM;
+        }
+    }
+
     dev->tx_q = netdev_dpdk_alloc_txq(netdev->n_txq);
     if (!dev->tx_q) {
         err = ENOMEM;
@@ -4085,11 +4975,20 @@ dpdk_vhost_reconfigure_helper(struct netdev_dpdk *dev)
     dev->up.n_rxq = dev->requested_n_rxq;
     int err;
 
+    /* Always keep RX queue 0 enabled for implementations that won't
+     * report vring states. */
+    dev->vhost_rxq_enabled[0] = true;
+
     /* Enable TX queue 0 by default if it wasn't disabled. */
     if (dev->tx_q[0].map == OVS_VHOST_QUEUE_MAP_UNKNOWN) {
         dev->tx_q[0].map = 0;
     }
 
+    if (userspace_tso_enabled()) {
+        dev->hw_ol_features |= NETDEV_TX_TSO_OFFLOAD;
+        VLOG_DBG("%s: TSO enabled on vhost port", netdev_get_name(&dev->up));
+    }
+
     netdev_dpdk_remap_txqs(dev);
 
     err = netdev_dpdk_mempool_configure(dev);
@@ -4129,7 +5028,7 @@ 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;
+    uint64_t vhost_unsup_flags;
 
     ovs_mutex_lock(&dev->mutex);
 
@@ -4142,6 +5041,9 @@ netdev_dpdk_vhost_client_reconfigure(struct netdev *netdev)
         /* Register client-mode device. */
         vhost_flags |= RTE_VHOST_USER_CLIENT;
 
+        /* There is no support for multi-segments buffers. */
+        vhost_flags |= RTE_VHOST_USER_LINEARBUF_SUPPORT;
+
         /* Enable IOMMU support, if explicitly requested. */
         if (dpdk_vhost_iommu_enabled()) {
             vhost_flags |= RTE_VHOST_USER_IOMMU_SUPPORT;
@@ -4152,11 +5054,9 @@ netdev_dpdk_vhost_client_reconfigure(struct netdev *netdev)
             vhost_flags |= RTE_VHOST_USER_POSTCOPY_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;
+        /* Enable External Buffers if TCP Segmentation Offload is enabled. */
+        if (userspace_tso_enabled()) {
+            vhost_flags |= RTE_VHOST_USER_EXTBUF_SUPPORT;
         }
 
         err = rte_vhost_driver_register(dev->vhost_id, vhost_flags);
@@ -4170,9 +5070,6 @@ netdev_dpdk_vhost_client_reconfigure(struct netdev *netdev)
             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,
@@ -4183,13 +5080,26 @@ netdev_dpdk_vhost_client_reconfigure(struct netdev *netdev)
             goto unlock;
         }
 
+        if (userspace_tso_enabled()) {
+            netdev->ol_flags |= NETDEV_TX_OFFLOAD_TCP_TSO;
+            netdev->ol_flags |= NETDEV_TX_OFFLOAD_TCP_CKSUM;
+            netdev->ol_flags |= NETDEV_TX_OFFLOAD_UDP_CKSUM;
+            netdev->ol_flags |= NETDEV_TX_OFFLOAD_SCTP_CKSUM;
+            netdev->ol_flags |= NETDEV_TX_OFFLOAD_IPV4_CKSUM;
+            vhost_unsup_flags = 1ULL << VIRTIO_NET_F_HOST_ECN
+                                | 1ULL << VIRTIO_NET_F_HOST_UFO;
+        } else {
+            /* This disables checksum offloading and all the features
+             * that depends on it (TSO, UFO, ECN) according to virtio
+             * specification. */
+            vhost_unsup_flags = 1ULL << VIRTIO_NET_F_CSUM;
+        }
+
         err = rte_vhost_driver_disable_features(dev->vhost_id,
-                                    1ULL << VIRTIO_NET_F_HOST_TSO4
-                                    | 1ULL << VIRTIO_NET_F_HOST_TSO6
-                                    | 1ULL << VIRTIO_NET_F_CSUM);
+                                                vhost_unsup_flags);
         if (err) {
-            VLOG_ERR("rte_vhost_driver_disable_features failed for vhost user "
-                     "client port: %s\n", dev->up.name);
+            VLOG_ERR("rte_vhost_driver_disable_features failed for "
+                     "vhost user client port: %s\n", dev->up.name);
             goto unlock;
         }
 
@@ -4209,6 +5119,45 @@ unlock:
     return err;
 }
 
+int
+netdev_dpdk_get_port_id(struct netdev *netdev)
+{
+    struct netdev_dpdk *dev;
+    int ret = -1;
+
+    if (!is_dpdk_class(netdev->netdev_class)) {
+        goto out;
+    }
+
+    dev = netdev_dpdk_cast(netdev);
+    ovs_mutex_lock(&dev->mutex);
+    ret = dev->port_id;
+    ovs_mutex_unlock(&dev->mutex);
+out:
+    return ret;
+}
+
+bool
+netdev_dpdk_flow_api_supported(struct netdev *netdev)
+{
+    struct netdev_dpdk *dev;
+    bool ret = false;
+
+    if (!is_dpdk_class(netdev->netdev_class)) {
+        goto out;
+    }
+
+    dev = netdev_dpdk_cast(netdev);
+    ovs_mutex_lock(&dev->mutex);
+    if (dev->type == DPDK_DEV_ETH) {
+        /* TODO: Check if we able to offload some minimal flow. */
+        ret = true;
+    }
+    ovs_mutex_unlock(&dev->mutex);
+out:
+    return ret;
+}
+
 int
 netdev_dpdk_rte_flow_destroy(struct netdev *netdev,
                              struct rte_flow *rte_flow,
@@ -4239,6 +5188,36 @@ netdev_dpdk_rte_flow_create(struct netdev *netdev,
     return flow;
 }
 
+int
+netdev_dpdk_rte_flow_query_count(struct netdev *netdev,
+                                 struct rte_flow *rte_flow,
+                                 struct rte_flow_query_count *query,
+                                 struct rte_flow_error *error)
+{
+    struct rte_flow_action_count count = { .shared = 0, .id = 0 };
+    const struct rte_flow_action actions[] = {
+        {
+            .type = RTE_FLOW_ACTION_TYPE_COUNT,
+            .conf = &count,
+        },
+        {
+            .type = RTE_FLOW_ACTION_TYPE_END,
+        },
+    };
+    struct netdev_dpdk *dev;
+    int ret;
+
+    if (!is_dpdk_class(netdev->netdev_class)) {
+        return -1;
+    }
+
+    dev = netdev_dpdk_cast(netdev);
+    ovs_mutex_lock(&dev->mutex);
+    ret = rte_flow_query(dev->port_id, rte_flow, actions, query, error);
+    ovs_mutex_unlock(&dev->mutex);
+    return ret;
+}
+
 #define NETDEV_DPDK_CLASS_COMMON                            \
     .is_pmd = true,                                         \
     .alloc = netdev_dpdk_alloc,                             \
@@ -4256,6 +5235,13 @@ netdev_dpdk_rte_flow_create(struct netdev *netdev,
     .get_qos_types = netdev_dpdk_get_qos_types,             \
     .get_qos = netdev_dpdk_get_qos,                         \
     .set_qos = netdev_dpdk_set_qos,                         \
+    .get_queue = netdev_dpdk_get_queue,                     \
+    .set_queue = netdev_dpdk_set_queue,                     \
+    .delete_queue = netdev_dpdk_delete_queue,               \
+    .get_queue_stats = netdev_dpdk_get_queue_stats,         \
+    .queue_dump_start = netdev_dpdk_queue_dump_start,       \
+    .queue_dump_next = netdev_dpdk_queue_dump_next,         \
+    .queue_dump_done = netdev_dpdk_queue_dump_done,         \
     .update_flags = netdev_dpdk_update_flags,               \
     .rxq_alloc = netdev_dpdk_rxq_alloc,                     \
     .rxq_construct = netdev_dpdk_rxq_construct,             \
@@ -4273,8 +5259,7 @@ netdev_dpdk_rte_flow_create(struct netdev *netdev,
     .get_features = netdev_dpdk_get_features,           \
     .get_status = netdev_dpdk_get_status,               \
     .reconfigure = netdev_dpdk_reconfigure,             \
-    .rxq_recv = netdev_dpdk_rxq_recv,                   \
-    DPDK_FLOW_OFFLOAD_API
+    .rxq_recv = netdev_dpdk_rxq_recv
 
 static const struct netdev_class dpdk_class = {
     .type = "dpdk",
@@ -4284,14 +5269,6 @@ static const struct netdev_class dpdk_class = {
     .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,
@@ -4300,9 +5277,11 @@ static const struct netdev_class dpdk_vhost_class = {
     .send = netdev_dpdk_vhost_send,
     .get_carrier = netdev_dpdk_vhost_get_carrier,
     .get_stats = netdev_dpdk_vhost_get_stats,
+    .get_custom_stats = netdev_dpdk_get_sw_custom_stats,
     .get_status = netdev_dpdk_vhost_user_get_status,
     .reconfigure = netdev_dpdk_vhost_reconfigure,
-    .rxq_recv = netdev_dpdk_vhost_rxq_recv
+    .rxq_recv = netdev_dpdk_vhost_rxq_recv,
+    .rxq_enabled = netdev_dpdk_vhost_rxq_enabled,
 };
 
 static const struct netdev_class dpdk_vhost_client_class = {
@@ -4314,16 +5293,17 @@ static const struct netdev_class dpdk_vhost_client_class = {
     .send = netdev_dpdk_vhost_send,
     .get_carrier = netdev_dpdk_vhost_get_carrier,
     .get_stats = netdev_dpdk_vhost_get_stats,
+    .get_custom_stats = netdev_dpdk_get_sw_custom_stats,
     .get_status = netdev_dpdk_vhost_user_get_status,
     .reconfigure = netdev_dpdk_vhost_client_reconfigure,
-    .rxq_recv = netdev_dpdk_vhost_rxq_recv
+    .rxq_recv = netdev_dpdk_vhost_rxq_recv,
+    .rxq_enabled = netdev_dpdk_vhost_rxq_enabled,
 };
 
 void
 netdev_dpdk_register(void)
 {
     netdev_register_provider(&dpdk_class);
-    netdev_register_provider(&dpdk_ring_class);
     netdev_register_provider(&dpdk_vhost_class);
     netdev_register_provider(&dpdk_vhost_client_class);
 }