dpif-netdev: Initialize upcall->packet when queuing to userspace.
[cascardo/ovs.git] / lib / dpif-netdev.c
index 0f95c9c..8422c89 100644 (file)
@@ -15,7 +15,7 @@
  */
 
 #include <config.h>
-#include "dpif.h"
+#include "dpif-netdev.h"
 
 #include <ctype.h>
 #include <errno.h>
 #include <unistd.h>
 
 #include "classifier.h"
+#include "cmap.h"
 #include "csum.h"
 #include "dpif.h"
 #include "dpif-provider.h"
 #include "dummy.h"
 #include "dynamic-string.h"
+#include "fat-rwlock.h"
 #include "flow.h"
-#include "hmap.h"
+#include "cmap.h"
+#include "latch.h"
 #include "list.h"
 #include "meta-flow.h"
 #include "netdev.h"
+#include "netdev-dpdk.h"
 #include "netdev-vport.h"
 #include "netlink.h"
 #include "odp-execute.h"
 #include "odp-util.h"
 #include "ofp-print.h"
 #include "ofpbuf.h"
+#include "ovs-rcu.h"
+#include "packet-dpif.h"
 #include "packets.h"
 #include "poll-loop.h"
 #include "random.h"
@@ -64,82 +70,254 @@ VLOG_DEFINE_THIS_MODULE(dpif_netdev);
 /* By default, choose a priority in the middle. */
 #define NETDEV_RULE_PRIORITY 0x8000
 
+#define FLOW_DUMP_MAX_BATCH 50
+/* Use per thread recirc_depth to prevent recirculation loop. */
+#define MAX_RECIRC_DEPTH 5
+DEFINE_STATIC_PER_THREAD_DATA(uint32_t, recirc_depth, 0)
+
 /* Configuration parameters. */
 enum { MAX_FLOWS = 65536 };     /* Maximum number of flows in flow table. */
 
-/* Enough headroom to add a vlan tag, plus an extra 2 bytes to allow IP
- * headers to be aligned on a 4-byte boundary.  */
-enum { DP_NETDEV_HEADROOM = 2 + VLAN_HEADER_LEN };
-
 /* Queues. */
-enum { N_QUEUES = 2 };          /* Number of queues for dpif_recv(). */
 enum { MAX_QUEUE_LEN = 128 };   /* Maximum number of packets per queue. */
 enum { QUEUE_MASK = MAX_QUEUE_LEN - 1 };
 BUILD_ASSERT_DECL(IS_POW2(MAX_QUEUE_LEN));
 
+/* Protects against changes to 'dp_netdevs'. */
+static struct ovs_mutex dp_netdev_mutex = OVS_MUTEX_INITIALIZER;
+
+/* Contains all 'struct dp_netdev's. */
+static struct shash dp_netdevs OVS_GUARDED_BY(dp_netdev_mutex)
+    = SHASH_INITIALIZER(&dp_netdevs);
+
 struct dp_netdev_upcall {
     struct dpif_upcall upcall;  /* Queued upcall information. */
     struct ofpbuf buf;          /* ofpbuf instance for upcall.packet. */
 };
 
+/* A queue passing packets from a struct dp_netdev to its clients (handlers).
+ *
+ *
+ * Thread-safety
+ * =============
+ *
+ * Any access at all requires the owning 'dp_netdev''s queue_rwlock and
+ * its own mutex. */
 struct dp_netdev_queue {
-    struct dp_netdev_upcall upcalls[MAX_QUEUE_LEN];
-    unsigned int head, tail;
+    struct ovs_mutex mutex;
+    struct seq *seq;      /* Incremented whenever a packet is queued. */
+    struct dp_netdev_upcall upcalls[MAX_QUEUE_LEN] OVS_GUARDED;
+    unsigned int head OVS_GUARDED;
+    unsigned int tail OVS_GUARDED;
 };
 
-/* Datapath based on the network device interface from netdev.h. */
+/* Datapath based on the network device interface from netdev.h.
+ *
+ *
+ * Thread-safety
+ * =============
+ *
+ * Some members, marked 'const', are immutable.  Accessing other members
+ * requires synchronization, as noted in more detail below.
+ *
+ * Acquisition order is, from outermost to innermost:
+ *
+ *    dp_netdev_mutex (global)
+ *    port_mutex
+ *    flow_mutex
+ *    queue_rwlock
+ */
 struct dp_netdev {
-    const struct dpif_class *class;
-    char *name;
-    int open_cnt;
-    bool destroyed;
+    const struct dpif_class *const class;
+    const char *const name;
+    struct ovs_refcount ref_cnt;
+    atomic_flag destroyed;
 
-    struct dp_netdev_queue queues[N_QUEUES];
-    struct classifier cls;      /* Classifier. */
-    struct hmap flow_table;     /* Flow table. */
-    struct seq *queue_seq;      /* Incremented whenever a packet is queued. */
-
-    /* Statistics. */
-    struct ovsthread_counter *n_hit;    /* Number of flow table matches. */
-    struct ovsthread_counter *n_missed; /* Number of flow table misses. */
-    struct ovsthread_counter *n_lost;   /* Number of misses not passed up. */
+    /* Flows.
+     *
+     * Writers of 'flow_table' must take the 'flow_mutex'.  Corresponding
+     * changes to 'cls' must be made while still holding the 'flow_mutex'.
+     */
+    struct ovs_mutex flow_mutex;
+    struct classifier cls;
+    struct cmap flow_table OVS_GUARDED; /* Flow table. */
+
+    /* Queues.
+     *
+     * 'queue_rwlock' protects the modification of 'handler_queues' and
+     * 'n_handlers'.  The queue elements are protected by its
+     * 'handler_queues''s mutex. */
+    struct fat_rwlock queue_rwlock;
+    struct dp_netdev_queue *handler_queues;
+    uint32_t n_handlers;
+
+    /* Statistics.
+     *
+     * ovsthread_stats is internally synchronized. */
+    struct ovsthread_stats stats; /* Contains 'struct dp_netdev_stats *'. */
 
-    /* Ports. */
-    struct hmap ports;
+    /* Ports.
+     *
+     * Protected by RCU.  Take the mutex to add or remove ports. */
+    struct ovs_mutex port_mutex;
+    struct cmap ports;
     struct seq *port_seq;       /* Incremented whenever a port changes. */
+
+    /* Forwarding threads. */
+    struct latch exit_latch;
+    struct pmd_thread *pmd_threads;
+    size_t n_pmd_threads;
+    int pmd_count;
 };
 
-static struct dp_netdev_port *dp_netdev_lookup_port(const struct dp_netdev *,
+static struct dp_netdev_port *dp_netdev_lookup_port(const struct dp_netdev *dp,
                                                     odp_port_t);
 
+enum dp_stat_type {
+    DP_STAT_HIT,                /* Packets that matched in the flow table. */
+    DP_STAT_MISS,               /* Packets that did not match. */
+    DP_STAT_LOST,               /* Packets not passed up to the client. */
+    DP_N_STATS
+};
+
+/* Contained by struct dp_netdev's 'stats' member.  */
+struct dp_netdev_stats {
+    struct ovs_mutex mutex;          /* Protects 'n'. */
+
+    /* Indexed by DP_STAT_*, protected by 'mutex'. */
+    unsigned long long int n[DP_N_STATS] OVS_GUARDED;
+};
+
+
 /* A port in a netdev-based datapath. */
 struct dp_netdev_port {
-    struct hmap_node node;      /* Node in dp_netdev's 'ports'. */
+    struct cmap_node node;      /* Node in dp_netdev's 'ports'. */
     odp_port_t port_no;
     struct netdev *netdev;
     struct netdev_saved_flags *sf;
-    struct netdev_rx *rx;
+    struct netdev_rxq **rxq;
+    struct ovs_refcount ref_cnt;
     char *type;                 /* Port type as requested by user. */
 };
 
-/* A flow in dp_netdev's 'flow_table'. */
+
+/* Stores a miniflow */
+
+/* There are fields in the flow structure that we never use. Therefore we can
+ * save a few words of memory */
+#define NETDEV_KEY_BUF_SIZE_U32 (FLOW_U32S - MINI_N_INLINE \
+                                 - FLOW_U32_SIZE(regs) \
+                                 - FLOW_U32_SIZE(metadata) \
+                                )
+struct netdev_flow_key {
+    struct miniflow flow;
+    uint32_t buf[NETDEV_KEY_BUF_SIZE_U32];
+};
+
+/* A flow in dp_netdev's 'flow_table'.
+ *
+ *
+ * Thread-safety
+ * =============
+ *
+ * Except near the beginning or ending of its lifespan, rule 'rule' belongs to
+ * its dp_netdev's classifier.  The text below calls this classifier 'cls'.
+ *
+ * Motivation
+ * ----------
+ *
+ * The thread safety rules described here for "struct dp_netdev_flow" are
+ * motivated by two goals:
+ *
+ *    - Prevent threads that read members of "struct dp_netdev_flow" from
+ *      reading bad data due to changes by some thread concurrently modifying
+ *      those members.
+ *
+ *    - Prevent two threads making changes to members of a given "struct
+ *      dp_netdev_flow" from interfering with each other.
+ *
+ *
+ * Rules
+ * -----
+ *
+ * A flow 'flow' may be accessed without a risk of being freed by code that
+ * holds a read-lock or write-lock on 'cls->rwlock' or that owns a reference to
+ * 'flow->ref_cnt' (or both).  Code that needs to hold onto a flow for a while
+ * should take 'cls->rwlock', find the flow it needs, increment 'flow->ref_cnt'
+ * with dpif_netdev_flow_ref(), and drop 'cls->rwlock'.
+ *
+ * 'flow->ref_cnt' protects 'flow' from being freed.  It doesn't protect the
+ * flow from being deleted from 'cls' (that's 'cls->rwlock') and it doesn't
+ * protect members of 'flow' from modification.
+ *
+ * Some members, marked 'const', are immutable.  Accessing other members
+ * requires synchronization, as noted in more detail below.
+ */
 struct dp_netdev_flow {
     /* Packet classification. */
-    struct cls_rule cr;         /* In owning dp_netdev's 'cls'. */
+    const struct cls_rule cr;   /* In owning dp_netdev's 'cls'. */
 
-    /* Hash table index by unmasked flow.*/
-    struct hmap_node node;      /* In owning dp_netdev's 'flow_table'. */
-    struct flow flow;           /* The flow that created this entry. */
+    /* Hash table index by unmasked flow. */
+    const struct cmap_node node; /* In owning dp_netdev's 'flow_table'. */
+    const struct flow flow;      /* The flow that created this entry. */
 
-    /* Statistics. */
-    long long int used;         /* Last used time, in monotonic msecs. */
-    long long int packet_count; /* Number of packets matched. */
-    long long int byte_count;   /* Number of bytes matched. */
-    uint16_t tcp_flags;         /* Bitwise-OR of seen tcp_flags values. */
+    /* Statistics.
+     *
+     * Reading or writing these members requires 'mutex'. */
+    struct ovsthread_stats stats; /* Contains "struct dp_netdev_flow_stats". */
 
     /* Actions. */
-    struct nlattr *actions;
-    size_t actions_len;
+    OVSRCU_TYPE(struct dp_netdev_actions *) actions;
+};
+
+static void dp_netdev_flow_free(struct dp_netdev_flow *);
+
+/* Contained by struct dp_netdev_flow's 'stats' member.  */
+struct dp_netdev_flow_stats {
+    struct ovs_mutex mutex;         /* Guards all the other members. */
+
+    long long int used OVS_GUARDED; /* Last used time, in monotonic msecs. */
+    long long int packet_count OVS_GUARDED; /* Number of packets matched. */
+    long long int byte_count OVS_GUARDED;   /* Number of bytes matched. */
+    uint16_t tcp_flags OVS_GUARDED; /* Bitwise-OR of seen tcp_flags values. */
+};
+
+/* A set of datapath actions within a "struct dp_netdev_flow".
+ *
+ *
+ * Thread-safety
+ * =============
+ *
+ * A struct dp_netdev_actions 'actions' is protected with RCU. */
+struct dp_netdev_actions {
+    /* These members are immutable: they do not change during the struct's
+     * lifetime.  */
+    struct nlattr *actions;     /* Sequence of OVS_ACTION_ATTR_* attributes. */
+    unsigned int size;          /* Size of 'actions', in bytes. */
+};
+
+struct dp_netdev_actions *dp_netdev_actions_create(const struct nlattr *,
+                                                   size_t);
+struct dp_netdev_actions *dp_netdev_flow_get_actions(
+    const struct dp_netdev_flow *);
+static void dp_netdev_actions_free(struct dp_netdev_actions *);
+
+/* PMD: Poll modes drivers.  PMD accesses devices via polling to eliminate
+ * the performance overhead of interrupt processing.  Therefore netdev can
+ * not implement rx-wait for these devices.  dpif-netdev needs to poll
+ * these device to check for recv buffer.  pmd-thread does polling for
+ * devices assigned to itself thread.
+ *
+ * DPDK used PMD for accessing NIC.
+ *
+ * A thread that receives packets from PMD ports, looks them up in the flow
+ * table, and executes the actions it finds.
+ **/
+struct pmd_thread {
+    struct dp_netdev *dp;
+    pthread_t thread;
+    int id;
+    atomic_uint change_seq;
 };
 
 /* Interface to netdev-based datapath. */
@@ -149,32 +327,36 @@ struct dpif_netdev {
     uint64_t last_port_seq;
 };
 
-/* All netdev-based datapaths. */
-static struct shash dp_netdevs = SHASH_INITIALIZER(&dp_netdevs);
-
-/* Global lock for all data. */
-static struct ovs_mutex dp_netdev_mutex = OVS_MUTEX_INITIALIZER;
-
-static int get_port_by_number(struct dp_netdev *, odp_port_t port_no,
+static int get_port_by_number(struct dp_netdev *dp, odp_port_t port_no,
                               struct dp_netdev_port **portp);
-static int get_port_by_name(struct dp_netdev *, const char *devname,
+static int get_port_by_name(struct dp_netdev *dp, const char *devname,
                             struct dp_netdev_port **portp);
-static void dp_netdev_free(struct dp_netdev *);
+static void dp_netdev_free(struct dp_netdev *)
+    OVS_REQUIRES(dp_netdev_mutex);
 static void dp_netdev_flow_flush(struct dp_netdev *);
-static int do_add_port(struct dp_netdev *, const char *devname,
-                       const char *type, odp_port_t port_no);
-static int do_del_port(struct dp_netdev *, odp_port_t port_no);
+static int do_add_port(struct dp_netdev *dp, const char *devname,
+                       const char *type, odp_port_t port_no)
+    OVS_REQUIRES(dp->port_mutex);
+static void do_del_port(struct dp_netdev *dp, struct dp_netdev_port *)
+    OVS_REQUIRES(dp->port_mutex);
+static void dp_netdev_destroy_all_queues(struct dp_netdev *dp)
+    OVS_REQ_WRLOCK(dp->queue_rwlock);
 static int dpif_netdev_open(const struct dpif_class *, const char *name,
                             bool create, struct dpif **);
-static int dp_netdev_output_userspace(struct dp_netdev *, struct ofpbuf *,
-                                    int queue_no, const struct flow *,
-                                    const struct nlattr *userdata);
-static void dp_netdev_execute_actions(struct dp_netdev *, const struct flow *,
-                                      struct ofpbuf *, struct pkt_metadata *,
+static int dp_netdev_output_userspace(struct dp_netdev *dp, struct ofpbuf *,
+                                      int queue_no, int type,
+                                      const struct miniflow *,
+                                      const struct nlattr *userdata);
+static void dp_netdev_execute_actions(struct dp_netdev *dp,
+                                      struct dpif_packet **, int c,
+                                      bool may_steal, struct pkt_metadata *,
                                       const struct nlattr *actions,
                                       size_t actions_len);
-static void dp_netdev_port_input(struct dp_netdev *dp, struct ofpbuf *packet,
-                                 struct pkt_metadata *md);
+static void dp_netdev_port_input(struct dp_netdev *dp,
+                                 struct dpif_packet **packets, int cnt,
+                                 odp_port_t port_no);
+
+static void dp_netdev_set_pmd_threads(struct dp_netdev *, int n);
 
 static struct dpif_netdev *
 dpif_netdev_cast(const struct dpif *dpif)
@@ -190,12 +372,19 @@ get_dp_netdev(const struct dpif *dpif)
 }
 
 static int
-dpif_netdev_enumerate(struct sset *all_dps)
+dpif_netdev_enumerate(struct sset *all_dps,
+                      const struct dpif_class *dpif_class)
 {
     struct shash_node *node;
 
     ovs_mutex_lock(&dp_netdev_mutex);
     SHASH_FOR_EACH(node, &dp_netdevs) {
+        struct dp_netdev *dp = node->data;
+        if (dpif_class != dp->class) {
+            /* 'dp_netdevs' contains both "netdev" and "dummy" dpifs.
+             * If the class doesn't match, skip this dpif. */
+             continue;
+        }
         sset_add(all_dps, node->name);
     }
     ovs_mutex_unlock(&dp_netdev_mutex);
@@ -223,7 +412,7 @@ create_dpif_netdev(struct dp_netdev *dp)
     uint16_t netflow_id = hash_string(dp->name, 0);
     struct dpif_netdev *dpif;
 
-    dp->open_cnt++;
+    ovs_refcount_ref(&dp->ref_cnt);
 
     dpif = xmalloc(sizeof *dpif);
     dpif_init(&dpif->dpif, dp->class, dp->name, netflow_id >> 8, netflow_id);
@@ -237,6 +426,7 @@ create_dpif_netdev(struct dp_netdev *dp)
  * Return ODPP_NONE on failure. */
 static odp_port_t
 choose_port(struct dp_netdev *dp, const char *name)
+    OVS_REQUIRES(dp->port_mutex)
 {
     uint32_t port_no;
 
@@ -277,37 +467,40 @@ choose_port(struct dp_netdev *dp, const char *name)
 static int
 create_dp_netdev(const char *name, const struct dpif_class *class,
                  struct dp_netdev **dpp)
+    OVS_REQUIRES(dp_netdev_mutex)
 {
     struct dp_netdev *dp;
     int error;
-    int i;
 
     dp = xzalloc(sizeof *dp);
-    dp->class = class;
-    dp->name = xstrdup(name);
-    dp->open_cnt = 0;
-    for (i = 0; i < N_QUEUES; i++) {
-        dp->queues[i].head = dp->queues[i].tail = 0;
-    }
-    dp->queue_seq = seq_create();
+    shash_add(&dp_netdevs, name, dp);
+
+    *CONST_CAST(const struct dpif_class **, &dp->class) = class;
+    *CONST_CAST(const char **, &dp->name) = xstrdup(name);
+    ovs_refcount_init(&dp->ref_cnt);
+    atomic_flag_clear(&dp->destroyed);
+
+    ovs_mutex_init(&dp->flow_mutex);
     classifier_init(&dp->cls, NULL);
-    hmap_init(&dp->flow_table);
+    cmap_init(&dp->flow_table);
 
-    dp->n_hit = ovsthread_counter_create();
-    dp->n_missed = ovsthread_counter_create();
-    dp->n_lost = ovsthread_counter_create();
+    fat_rwlock_init(&dp->queue_rwlock);
 
-    hmap_init(&dp->ports);
+    ovsthread_stats_init(&dp->stats);
+
+    ovs_mutex_init(&dp->port_mutex);
+    cmap_init(&dp->ports);
     dp->port_seq = seq_create();
+    latch_init(&dp->exit_latch);
 
+    ovs_mutex_lock(&dp->port_mutex);
     error = do_add_port(dp, name, "internal", ODPP_LOCAL);
+    ovs_mutex_unlock(&dp->port_mutex);
     if (error) {
         dp_netdev_free(dp);
         return error;
     }
 
-    shash_add(&dp_netdevs, name, dp);
-
     *dpp = dp;
     return 0;
 }
@@ -338,57 +531,88 @@ dpif_netdev_open(const struct dpif_class *class, const char *name,
 
 static void
 dp_netdev_purge_queues(struct dp_netdev *dp)
+    OVS_REQ_WRLOCK(dp->queue_rwlock)
 {
     int i;
 
-    for (i = 0; i < N_QUEUES; i++) {
-        struct dp_netdev_queue *q = &dp->queues[i];
+    for (i = 0; i < dp->n_handlers; i++) {
+        struct dp_netdev_queue *q = &dp->handler_queues[i];
 
+        ovs_mutex_lock(&q->mutex);
         while (q->tail != q->head) {
             struct dp_netdev_upcall *u = &q->upcalls[q->tail++ & QUEUE_MASK];
             ofpbuf_uninit(&u->upcall.packet);
             ofpbuf_uninit(&u->buf);
         }
+        ovs_mutex_unlock(&q->mutex);
     }
 }
 
+/* Requires dp_netdev_mutex so that we can't get a new reference to 'dp'
+ * through the 'dp_netdevs' shash while freeing 'dp'. */
 static void
 dp_netdev_free(struct dp_netdev *dp)
+    OVS_REQUIRES(dp_netdev_mutex)
 {
-    struct dp_netdev_port *port, *next;
+    struct dp_netdev_port *port;
+    struct dp_netdev_stats *bucket;
+    int i;
+
+    shash_find_and_delete(&dp_netdevs, dp->name);
+
+    dp_netdev_set_pmd_threads(dp, 0);
+    free(dp->pmd_threads);
 
     dp_netdev_flow_flush(dp);
-    HMAP_FOR_EACH_SAFE (port, next, node, &dp->ports) {
-        do_del_port(dp, port->port_no);
+    ovs_mutex_lock(&dp->port_mutex);
+    CMAP_FOR_EACH (port, node, &dp->ports) {
+        do_del_port(dp, port);
     }
-    ovsthread_counter_destroy(dp->n_hit);
-    ovsthread_counter_destroy(dp->n_missed);
-    ovsthread_counter_destroy(dp->n_lost);
-    dp_netdev_purge_queues(dp);
-    seq_destroy(dp->queue_seq);
+    ovs_mutex_unlock(&dp->port_mutex);
+
+    OVSTHREAD_STATS_FOR_EACH_BUCKET (bucket, i, &dp->stats) {
+        ovs_mutex_destroy(&bucket->mutex);
+        free_cacheline(bucket);
+    }
+    ovsthread_stats_destroy(&dp->stats);
+
+    fat_rwlock_wrlock(&dp->queue_rwlock);
+    dp_netdev_destroy_all_queues(dp);
+    fat_rwlock_unlock(&dp->queue_rwlock);
+
+    fat_rwlock_destroy(&dp->queue_rwlock);
+
     classifier_destroy(&dp->cls);
-    hmap_destroy(&dp->flow_table);
+    cmap_destroy(&dp->flow_table);
+    ovs_mutex_destroy(&dp->flow_mutex);
     seq_destroy(dp->port_seq);
-    hmap_destroy(&dp->ports);
-    free(dp->name);
+    cmap_destroy(&dp->ports);
+    latch_destroy(&dp->exit_latch);
+    free(CONST_CAST(char *, dp->name));
     free(dp);
 }
 
+static void
+dp_netdev_unref(struct dp_netdev *dp)
+{
+    if (dp) {
+        /* Take dp_netdev_mutex so that, if dp->ref_cnt falls to zero, we can't
+         * get a new reference to 'dp' through the 'dp_netdevs' shash. */
+        ovs_mutex_lock(&dp_netdev_mutex);
+        if (ovs_refcount_unref_relaxed(&dp->ref_cnt) == 1) {
+            dp_netdev_free(dp);
+        }
+        ovs_mutex_unlock(&dp_netdev_mutex);
+    }
+}
+
 static void
 dpif_netdev_close(struct dpif *dpif)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-
-    ovs_assert(dp->open_cnt > 0);
-    if (--dp->open_cnt == 0 && dp->destroyed) {
-        shash_find_and_delete(&dp_netdevs, dp->name);
-        dp_netdev_free(dp);
-    }
+    dp_netdev_unref(dp);
     free(dpif);
-
-    ovs_mutex_unlock(&dp_netdev_mutex);
 }
 
 static int
@@ -396,9 +620,12 @@ dpif_netdev_destroy(struct dpif *dpif)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    dp->destroyed = true;
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    if (!atomic_flag_test_and_set(&dp->destroyed)) {
+        if (ovs_refcount_unref_relaxed(&dp->ref_cnt) == 1) {
+            /* Can't happen: 'dpif' still owns a reference to 'dp'. */
+            OVS_NOT_REACHED();
+        }
+    }
 
     return 0;
 }
@@ -407,30 +634,56 @@ static int
 dpif_netdev_get_stats(const struct dpif *dpif, struct dpif_dp_stats *stats)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
-
-    ovs_mutex_lock(&dp_netdev_mutex);
-    stats->n_flows = hmap_count(&dp->flow_table);
-    stats->n_hit = ovsthread_counter_read(dp->n_hit);
-    stats->n_missed = ovsthread_counter_read(dp->n_missed);
-    stats->n_lost = ovsthread_counter_read(dp->n_lost);
+    struct dp_netdev_stats *bucket;
+    size_t i;
+
+    stats->n_flows = cmap_count(&dp->flow_table);
+
+    stats->n_hit = stats->n_missed = stats->n_lost = 0;
+    OVSTHREAD_STATS_FOR_EACH_BUCKET (bucket, i, &dp->stats) {
+        ovs_mutex_lock(&bucket->mutex);
+        stats->n_hit += bucket->n[DP_STAT_HIT];
+        stats->n_missed += bucket->n[DP_STAT_MISS];
+        stats->n_lost += bucket->n[DP_STAT_LOST];
+        ovs_mutex_unlock(&bucket->mutex);
+    }
     stats->n_masks = UINT32_MAX;
     stats->n_mask_hit = UINT64_MAX;
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return 0;
 }
 
+static void
+dp_netdev_reload_pmd_threads(struct dp_netdev *dp)
+{
+    int i;
+
+    for (i = 0; i < dp->n_pmd_threads; i++) {
+        struct pmd_thread *f = &dp->pmd_threads[i];
+        int id;
+
+        atomic_add(&f->change_seq, 1, &id);
+   }
+}
+
+static uint32_t
+hash_port_no(odp_port_t port_no)
+{
+    return hash_int(odp_to_u32(port_no), 0);
+}
+
 static int
 do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
             odp_port_t port_no)
+    OVS_REQUIRES(dp->port_mutex)
 {
     struct netdev_saved_flags *sf;
     struct dp_netdev_port *port;
     struct netdev *netdev;
-    struct netdev_rx *rx;
     enum netdev_flags flags;
     const char *open_type;
     int error;
+    int i;
 
     /* XXX reject devices already in some dp_netdev. */
 
@@ -449,30 +702,42 @@ do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
         return EINVAL;
     }
 
-    error = netdev_rx_open(netdev, &rx);
-    if (error
-        && !(error == EOPNOTSUPP && dpif_netdev_class_is_dummy(dp->class))) {
-        VLOG_ERR("%s: cannot receive packets on this network device (%s)",
-                 devname, ovs_strerror(errno));
-        netdev_close(netdev);
-        return error;
+    port = xzalloc(sizeof *port);
+    port->port_no = port_no;
+    port->netdev = netdev;
+    port->rxq = xmalloc(sizeof *port->rxq * netdev_n_rxq(netdev));
+    port->type = xstrdup(type);
+    for (i = 0; i < netdev_n_rxq(netdev); i++) {
+        error = netdev_rxq_open(netdev, &port->rxq[i], i);
+        if (error
+            && !(error == EOPNOTSUPP && dpif_netdev_class_is_dummy(dp->class))) {
+            VLOG_ERR("%s: cannot receive packets on this network device (%s)",
+                     devname, ovs_strerror(errno));
+            netdev_close(netdev);
+            return error;
+        }
     }
 
     error = netdev_turn_flags_on(netdev, NETDEV_PROMISC, &sf);
     if (error) {
-        netdev_rx_close(rx);
+        for (i = 0; i < netdev_n_rxq(netdev); i++) {
+            netdev_rxq_close(port->rxq[i]);
+        }
         netdev_close(netdev);
+        free(port->rxq);
+        free(port);
         return error;
     }
-
-    port = xmalloc(sizeof *port);
-    port->port_no = port_no;
-    port->netdev = netdev;
     port->sf = sf;
-    port->rx = rx;
-    port->type = xstrdup(type);
 
-    hmap_insert(&dp->ports, &port->node, hash_int(odp_to_u32(port_no), 0));
+    if (netdev_is_pmd(netdev)) {
+        dp->pmd_count++;
+        dp_netdev_set_pmd_threads(dp, NR_PMD_THREADS);
+        dp_netdev_reload_pmd_threads(dp);
+    }
+    ovs_refcount_init(&port->ref_cnt);
+
+    cmap_insert(&dp->ports, &port->node, hash_port_no(port_no));
     seq_change(dp->port_seq);
 
     return 0;
@@ -488,7 +753,7 @@ dpif_netdev_port_add(struct dpif *dpif, struct netdev *netdev,
     odp_port_t port_no;
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    ovs_mutex_lock(&dp->port_mutex);
     dpif_port = netdev_vport_get_dpif_port(netdev, namebuf, sizeof namebuf);
     if (*port_nop != ODPP_NONE) {
         port_no = *port_nop;
@@ -501,7 +766,7 @@ dpif_netdev_port_add(struct dpif *dpif, struct netdev *netdev,
         *port_nop = port_no;
         error = do_add_port(dp, dpif_port, netdev_get_type(netdev), port_no);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->port_mutex);
 
     return error;
 }
@@ -512,9 +777,18 @@ dpif_netdev_port_del(struct dpif *dpif, odp_port_t port_no)
     struct dp_netdev *dp = get_dp_netdev(dpif);
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    error = port_no == ODPP_LOCAL ? EINVAL : do_del_port(dp, port_no);
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_lock(&dp->port_mutex);
+    if (port_no == ODPP_LOCAL) {
+        error = EINVAL;
+    } else {
+        struct dp_netdev_port *port;
+
+        error = get_port_by_number(dp, port_no, &port);
+        if (!error) {
+            do_del_port(dp, port);
+        }
+    }
+    ovs_mutex_unlock(&dp->port_mutex);
 
     return error;
 }
@@ -530,8 +804,7 @@ dp_netdev_lookup_port(const struct dp_netdev *dp, odp_port_t port_no)
 {
     struct dp_netdev_port *port;
 
-    HMAP_FOR_EACH_IN_BUCKET (port, node, hash_int(odp_to_u32(port_no), 0),
-                             &dp->ports) {
+    CMAP_FOR_EACH_WITH_HASH (port, node, hash_port_no(port_no), &dp->ports) {
         if (port->port_no == port_no) {
             return port;
         }
@@ -552,13 +825,47 @@ get_port_by_number(struct dp_netdev *dp,
     }
 }
 
+static void
+port_ref(struct dp_netdev_port *port)
+{
+    if (port) {
+        ovs_refcount_ref(&port->ref_cnt);
+    }
+}
+
+static void
+port_destroy__(struct dp_netdev_port *port)
+{
+    int n_rxq = netdev_n_rxq(port->netdev);
+    int i;
+
+    netdev_close(port->netdev);
+    netdev_restore_flags(port->sf);
+
+    for (i = 0; i < n_rxq; i++) {
+        netdev_rxq_close(port->rxq[i]);
+    }
+    free(port->rxq);
+    free(port->type);
+    free(port);
+}
+
+static void
+port_unref(struct dp_netdev_port *port)
+{
+    if (port && ovs_refcount_unref_relaxed(&port->ref_cnt) == 1) {
+        ovsrcu_postpone(port_destroy__, port);
+    }
+}
+
 static int
 get_port_by_name(struct dp_netdev *dp,
                  const char *devname, struct dp_netdev_port **portp)
+    OVS_REQUIRES(dp->port_mutex)
 {
     struct dp_netdev_port *port;
 
-    HMAP_FOR_EACH (port, node, &dp->ports) {
+    CMAP_FOR_EACH (port, node, &dp->ports) {
         if (!strcmp(netdev_get_name(port->netdev), devname)) {
             *portp = port;
             return 0;
@@ -567,27 +874,17 @@ get_port_by_name(struct dp_netdev *dp,
     return ENOENT;
 }
 
-static int
-do_del_port(struct dp_netdev *dp, odp_port_t port_no)
+static void
+do_del_port(struct dp_netdev *dp, struct dp_netdev_port *port)
+    OVS_REQUIRES(dp->port_mutex)
 {
-    struct dp_netdev_port *port;
-    int error;
-
-    error = get_port_by_number(dp, port_no, &port);
-    if (error) {
-        return error;
-    }
-
-    hmap_remove(&dp->ports, &port->node);
+    cmap_remove(&dp->ports, &port->node, hash_odp_port(port->port_no));
     seq_change(dp->port_seq);
+    if (netdev_is_pmd(port->netdev)) {
+        dp_netdev_reload_pmd_threads(dp);
+    }
 
-    netdev_close(port->netdev);
-    netdev_restore_flags(port->sf);
-    netdev_rx_close(port->rx);
-    free(port->type);
-    free(port);
-
-    return 0;
+    port_unref(port);
 }
 
 static void
@@ -607,12 +904,10 @@ dpif_netdev_port_query_by_number(const struct dpif *dpif, odp_port_t port_no,
     struct dp_netdev_port *port;
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
     error = get_port_by_number(dp, port_no, &port);
     if (!error && dpif_port) {
         answer_port_query(port, dpif_port);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return error;
 }
@@ -625,37 +920,55 @@ dpif_netdev_port_query_by_name(const struct dpif *dpif, const char *devname,
     struct dp_netdev_port *port;
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    ovs_mutex_lock(&dp->port_mutex);
     error = get_port_by_name(dp, devname, &port);
     if (!error && dpif_port) {
         answer_port_query(port, dpif_port);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->port_mutex);
 
     return error;
 }
 
 static void
-dp_netdev_free_flow(struct dp_netdev *dp, struct dp_netdev_flow *netdev_flow)
+dp_netdev_flow_free(struct dp_netdev_flow *flow)
+{
+    struct dp_netdev_flow_stats *bucket;
+    size_t i;
+
+    OVSTHREAD_STATS_FOR_EACH_BUCKET (bucket, i, &flow->stats) {
+        ovs_mutex_destroy(&bucket->mutex);
+        free_cacheline(bucket);
+    }
+    ovsthread_stats_destroy(&flow->stats);
+
+    cls_rule_destroy(CONST_CAST(struct cls_rule *, &flow->cr));
+    dp_netdev_actions_free(dp_netdev_flow_get_actions(flow));
+    free(flow);
+}
+
+static void
+dp_netdev_remove_flow(struct dp_netdev *dp, struct dp_netdev_flow *flow)
+    OVS_REQUIRES(dp->flow_mutex)
 {
-    ovs_rwlock_wrlock(&dp->cls.rwlock);
-    classifier_remove(&dp->cls, &netdev_flow->cr);
-    ovs_rwlock_unlock(&dp->cls.rwlock);
-    cls_rule_destroy(&netdev_flow->cr);
+    struct cls_rule *cr = CONST_CAST(struct cls_rule *, &flow->cr);
+    struct cmap_node *node = CONST_CAST(struct cmap_node *, &flow->node);
 
-    hmap_remove(&dp->flow_table, &netdev_flow->node);
-    free(netdev_flow->actions);
-    free(netdev_flow);
+    classifier_remove(&dp->cls, cr);
+    cmap_remove(&dp->flow_table, node, flow_hash(&flow->flow, 0));
+    ovsrcu_postpone(dp_netdev_flow_free, flow);
 }
 
 static void
 dp_netdev_flow_flush(struct dp_netdev *dp)
 {
-    struct dp_netdev_flow *netdev_flow, *next;
+    struct dp_netdev_flow *netdev_flow;
 
-    HMAP_FOR_EACH_SAFE (netdev_flow, next, node, &dp->flow_table) {
-        dp_netdev_free_flow(dp, netdev_flow);
+    ovs_mutex_lock(&dp->flow_mutex);
+    CMAP_FOR_EACH_SAFE (netdev_flow, node, &dp->flow_table) {
+        dp_netdev_remove_flow(dp, netdev_flow);
     }
+    ovs_mutex_unlock(&dp->flow_mutex);
 }
 
 static int
@@ -663,16 +976,12 @@ dpif_netdev_flow_flush(struct dpif *dpif)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
 
-    ovs_mutex_lock(&dp_netdev_mutex);
     dp_netdev_flow_flush(dp);
-    ovs_mutex_unlock(&dp_netdev_mutex);
-
     return 0;
 }
 
 struct dp_netdev_port_state {
-    uint32_t bucket;
-    uint32_t offset;
+    struct cmap_position position;
     char *name;
 };
 
@@ -689,11 +998,10 @@ dpif_netdev_port_dump_next(const struct dpif *dpif, void *state_,
 {
     struct dp_netdev_port_state *state = state_;
     struct dp_netdev *dp = get_dp_netdev(dpif);
-    struct hmap_node *node;
+    struct cmap_node *node;
     int retval;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    node = hmap_at_position(&dp->ports, &state->bucket, &state->offset);
+    node = cmap_next_position(&dp->ports, &state->position);
     if (node) {
         struct dp_netdev_port *port;
 
@@ -709,7 +1017,6 @@ dpif_netdev_port_dump_next(const struct dpif *dpif, void *state_,
     } else {
         retval = EOF;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return retval;
 }
@@ -730,7 +1037,6 @@ dpif_netdev_port_poll(const struct dpif *dpif_, char **devnamep OVS_UNUSED)
     uint64_t new_port_seq;
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
     new_port_seq = seq_read(dpif->dp->port_seq);
     if (dpif->last_port_seq != new_port_seq) {
         dpif->last_port_seq = new_port_seq;
@@ -738,7 +1044,6 @@ dpif_netdev_port_poll(const struct dpif *dpif_, char **devnamep OVS_UNUSED)
     } else {
         error = EAGAIN;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return error;
 }
@@ -748,23 +1053,25 @@ dpif_netdev_port_poll_wait(const struct dpif *dpif_)
 {
     struct dpif_netdev *dpif = dpif_netdev_cast(dpif_);
 
-    ovs_mutex_lock(&dp_netdev_mutex);
     seq_wait(dpif->dp->port_seq, dpif->last_port_seq);
-    ovs_mutex_unlock(&dp_netdev_mutex);
 }
 
 static struct dp_netdev_flow *
-dp_netdev_lookup_flow(const struct dp_netdev *dp, const struct flow *flow)
+dp_netdev_flow_cast(const struct cls_rule *cr)
+{
+    return cr ? CONTAINER_OF(cr, struct dp_netdev_flow, cr) : NULL;
+}
+
+static struct dp_netdev_flow *
+dp_netdev_lookup_flow(const struct dp_netdev *dp, const struct miniflow *key)
 {
-    struct cls_rule *cr;
+    struct dp_netdev_flow *netdev_flow;
+    struct cls_rule *rule;
 
-    ovs_rwlock_wrlock(&dp->cls.rwlock);
-    cr = classifier_lookup(&dp->cls, flow, NULL);
-    ovs_rwlock_unlock(&dp->cls.rwlock);
+    classifier_lookup_miniflow_batch(&dp->cls, &key, &rule, 1);
+    netdev_flow = dp_netdev_flow_cast(rule);
 
-    return (cr
-            ? CONTAINER_OF(cr, struct dp_netdev_flow, cr)
-            : NULL);
+    return netdev_flow;
 }
 
 static struct dp_netdev_flow *
@@ -772,12 +1079,13 @@ dp_netdev_find_flow(const struct dp_netdev *dp, const struct flow *flow)
 {
     struct dp_netdev_flow *netdev_flow;
 
-    HMAP_FOR_EACH_WITH_HASH (netdev_flow, node, flow_hash(flow, 0),
+    CMAP_FOR_EACH_WITH_HASH (netdev_flow, node, flow_hash(flow, 0),
                              &dp->flow_table) {
         if (flow_equal(&netdev_flow->flow, flow)) {
             return netdev_flow;
         }
     }
+
     return NULL;
 }
 
@@ -785,10 +1093,18 @@ static void
 get_dpif_flow_stats(struct dp_netdev_flow *netdev_flow,
                     struct dpif_flow_stats *stats)
 {
-    stats->n_packets = netdev_flow->packet_count;
-    stats->n_bytes = netdev_flow->byte_count;
-    stats->used = netdev_flow->used;
-    stats->tcp_flags = netdev_flow->tcp_flags;
+    struct dp_netdev_flow_stats *bucket;
+    size_t i;
+
+    memset(stats, 0, sizeof *stats);
+    OVSTHREAD_STATS_FOR_EACH_BUCKET (bucket, i, &netdev_flow->stats) {
+        ovs_mutex_lock(&bucket->mutex);
+        stats->n_packets += bucket->packet_count;
+        stats->n_bytes += bucket->byte_count;
+        stats->used = MAX(stats->used, bucket->used);
+        stats->tcp_flags |= bucket->tcp_flags;
+        ovs_mutex_unlock(&bucket->mutex);
+    }
 }
 
 static int
@@ -798,7 +1114,10 @@ dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
                               struct flow *mask)
 {
     if (mask_key_len) {
-        if (odp_flow_key_to_mask(mask_key, mask_key_len, mask, flow)) {
+        enum odp_key_fitness fitness;
+
+        fitness = odp_flow_key_to_mask(mask_key, mask_key_len, mask, flow);
+        if (fitness) {
             /* This should not happen: it indicates that
              * odp_flow_key_from_mask() and odp_flow_key_to_mask()
              * disagree on the acceptable form of a mask.  Log the problem
@@ -811,14 +1130,13 @@ dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
                 ds_init(&s);
                 odp_flow_format(key, key_len, mask_key, mask_key_len, NULL, &s,
                                 true);
-                VLOG_ERR("internal error parsing flow mask %s", ds_cstr(&s));
+                VLOG_ERR("internal error parsing flow mask %s (%s)",
+                         ds_cstr(&s), odp_key_fitness_to_string(fitness));
                 ds_destroy(&s);
             }
 
             return EINVAL;
         }
-        /* Force unwildcard the in_port. */
-        mask->in_port.odp_port = u32_to_odp(UINT32_MAX);
     } else {
         enum mf_field_id id;
         /* No mask key, unwildcard everything except fields whose
@@ -837,6 +1155,14 @@ dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
         }
     }
 
+    /* Force unwildcard the in_port.
+     *
+     * We need to do this even in the case where we unwildcard "everything"
+     * above because "everything" only includes the 16-bit OpenFlow port number
+     * mask->in_port.ofp_port, which only covers half of the 32-bit datapath
+     * port number mask->in_port.odp_port. */
+    mask->in_port.odp_port = u32_to_odp(UINT32_MAX);
+
     return 0;
 }
 
@@ -876,7 +1202,10 @@ dpif_netdev_flow_from_nlattrs(const struct nlattr *key, uint32_t key_len,
 static int
 dpif_netdev_flow_get(const struct dpif *dpif,
                      const struct nlattr *nl_key, size_t nl_key_len,
-                     struct ofpbuf **actionsp, struct dpif_flow_stats *stats)
+                     struct ofpbuf **bufp,
+                     struct nlattr **maskp, size_t *mask_len,
+                     struct nlattr **actionsp, size_t *actions_len,
+                     struct dpif_flow_stats *stats)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
     struct dp_netdev_flow *netdev_flow;
@@ -888,75 +1217,82 @@ dpif_netdev_flow_get(const struct dpif *dpif,
         return error;
     }
 
-    ovs_mutex_lock(&dp_netdev_mutex);
     netdev_flow = dp_netdev_find_flow(dp, &key);
+
     if (netdev_flow) {
         if (stats) {
             get_dpif_flow_stats(netdev_flow, stats);
         }
+
+        if (maskp) {
+            struct flow_wildcards wc;
+
+            *bufp = ofpbuf_new(sizeof(struct odputil_keybuf));
+            minimask_expand(&netdev_flow->cr.match.mask, &wc);
+            odp_flow_key_from_mask(*bufp, &wc.masks, &netdev_flow->flow,
+                                   odp_to_u32(wc.masks.in_port.odp_port),
+                                   SIZE_MAX, true);
+            *maskp = ofpbuf_data(*bufp);
+            *mask_len = ofpbuf_size(*bufp);
+        }
         if (actionsp) {
-            *actionsp = ofpbuf_clone_data(netdev_flow->actions,
-                                          netdev_flow->actions_len);
+            struct dp_netdev_actions *actions;
+
+            actions = dp_netdev_flow_get_actions(netdev_flow);
+            *actionsp = actions->actions;
+            *actions_len = actions->size;
         }
-    } else {
+     } else {
         error = ENOENT;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return error;
 }
 
-static int
-set_flow_actions(struct dp_netdev_flow *netdev_flow,
-                 const struct nlattr *actions, size_t actions_len)
-{
-    netdev_flow->actions = xrealloc(netdev_flow->actions, actions_len);
-    netdev_flow->actions_len = actions_len;
-    memcpy(netdev_flow->actions, actions, actions_len);
-    return 0;
-}
-
 static int
 dp_netdev_flow_add(struct dp_netdev *dp, const struct flow *flow,
                    const struct flow_wildcards *wc,
                    const struct nlattr *actions,
                    size_t actions_len)
+    OVS_REQUIRES(dp->flow_mutex)
 {
     struct dp_netdev_flow *netdev_flow;
     struct match match;
-    int error;
 
     netdev_flow = xzalloc(sizeof *netdev_flow);
-    netdev_flow->flow = *flow;
+    *CONST_CAST(struct flow *, &netdev_flow->flow) = *flow;
 
-    match_init(&match, flow, wc);
-    cls_rule_init(&netdev_flow->cr, &match, NETDEV_RULE_PRIORITY);
-    ovs_rwlock_wrlock(&dp->cls.rwlock);
-    classifier_insert(&dp->cls, &netdev_flow->cr);
-    ovs_rwlock_unlock(&dp->cls.rwlock);
+    ovsthread_stats_init(&netdev_flow->stats);
 
-    error = set_flow_actions(netdev_flow, actions, actions_len);
-    if (error) {
-        ovs_rwlock_wrlock(&dp->cls.rwlock);
-        classifier_remove(&dp->cls, &netdev_flow->cr);
-        ovs_rwlock_unlock(&dp->cls.rwlock);
-        cls_rule_destroy(&netdev_flow->cr);
+    ovsrcu_set(&netdev_flow->actions,
+               dp_netdev_actions_create(actions, actions_len));
 
-        free(netdev_flow);
-        return error;
-    }
+    match_init(&match, flow, wc);
+    cls_rule_init(CONST_CAST(struct cls_rule *, &netdev_flow->cr),
+                  &match, NETDEV_RULE_PRIORITY);
+    cmap_insert(&dp->flow_table,
+                CONST_CAST(struct cmap_node *, &netdev_flow->node),
+                flow_hash(flow, 0));
+    classifier_insert(&dp->cls,
+                      CONST_CAST(struct cls_rule *, &netdev_flow->cr));
 
-    hmap_insert(&dp->flow_table, &netdev_flow->node, flow_hash(flow, 0));
     return 0;
 }
 
 static void
 clear_stats(struct dp_netdev_flow *netdev_flow)
 {
-    netdev_flow->used = 0;
-    netdev_flow->packet_count = 0;
-    netdev_flow->byte_count = 0;
-    netdev_flow->tcp_flags = 0;
+    struct dp_netdev_flow_stats *bucket;
+    size_t i;
+
+    OVSTHREAD_STATS_FOR_EACH_BUCKET (bucket, i, &netdev_flow->stats) {
+        ovs_mutex_lock(&bucket->mutex);
+        bucket->used = 0;
+        bucket->packet_count = 0;
+        bucket->byte_count = 0;
+        bucket->tcp_flags = 0;
+        ovs_mutex_unlock(&bucket->mutex);
+    }
 }
 
 static int
@@ -965,6 +1301,7 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put)
     struct dp_netdev *dp = get_dp_netdev(dpif);
     struct dp_netdev_flow *netdev_flow;
     struct flow flow;
+    struct miniflow miniflow;
     struct flow_wildcards wc;
     int error;
 
@@ -978,12 +1315,13 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put)
     if (error) {
         return error;
     }
+    miniflow_init(&miniflow, &flow);
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    netdev_flow = dp_netdev_lookup_flow(dp, &flow);
+    ovs_mutex_lock(&dp->flow_mutex);
+    netdev_flow = dp_netdev_lookup_flow(dp, &miniflow);
     if (!netdev_flow) {
         if (put->flags & DPIF_FP_CREATE) {
-            if (hmap_count(&dp->flow_table) < MAX_FLOWS) {
+            if (cmap_count(&dp->flow_table) < MAX_FLOWS) {
                 if (put->stats) {
                     memset(put->stats, 0, sizeof *put->stats);
                 }
@@ -998,16 +1336,23 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put)
     } else {
         if (put->flags & DPIF_FP_MODIFY
             && flow_equal(&flow, &netdev_flow->flow)) {
-            error = set_flow_actions(netdev_flow, put->actions,
-                                     put->actions_len);
-            if (!error) {
-                if (put->stats) {
-                    get_dpif_flow_stats(netdev_flow, put->stats);
-                }
-                if (put->flags & DPIF_FP_ZERO_STATS) {
-                    clear_stats(netdev_flow);
-                }
+            struct dp_netdev_actions *new_actions;
+            struct dp_netdev_actions *old_actions;
+
+            new_actions = dp_netdev_actions_create(put->actions,
+                                                   put->actions_len);
+
+            old_actions = dp_netdev_flow_get_actions(netdev_flow);
+            ovsrcu_set(&netdev_flow->actions, new_actions);
+
+            if (put->stats) {
+                get_dpif_flow_stats(netdev_flow, put->stats);
+            }
+            if (put->flags & DPIF_FP_ZERO_STATS) {
+                clear_stats(netdev_flow);
             }
+
+            ovsrcu_postpone(dp_netdev_actions_free, old_actions);
         } else if (put->flags & DPIF_FP_CREATE) {
             error = EEXIST;
         } else {
@@ -1015,7 +1360,8 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put)
             error = EINVAL;
         }
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->flow_mutex);
+    miniflow_destroy(&miniflow);
 
     return error;
 }
@@ -1033,140 +1379,257 @@ dpif_netdev_flow_del(struct dpif *dpif, const struct dpif_flow_del *del)
         return error;
     }
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    ovs_mutex_lock(&dp->flow_mutex);
     netdev_flow = dp_netdev_find_flow(dp, &key);
     if (netdev_flow) {
         if (del->stats) {
             get_dpif_flow_stats(netdev_flow, del->stats);
         }
-        dp_netdev_free_flow(dp, netdev_flow);
+        dp_netdev_remove_flow(dp, netdev_flow);
     } else {
         error = ENOENT;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->flow_mutex);
 
     return error;
 }
 
-struct dp_netdev_flow_state {
-    uint32_t bucket;
-    uint32_t offset;
-    struct nlattr *actions;
-    struct odputil_keybuf keybuf;
-    struct odputil_keybuf maskbuf;
-    struct dpif_flow_stats stats;
+struct dpif_netdev_flow_dump {
+    struct dpif_flow_dump up;
+    struct cmap_position pos;
+    int status;
+    struct ovs_mutex mutex;
 };
 
-static int
-dpif_netdev_flow_dump_start(const struct dpif *dpif OVS_UNUSED, void **statep)
+static struct dpif_netdev_flow_dump *
+dpif_netdev_flow_dump_cast(struct dpif_flow_dump *dump)
 {
-    struct dp_netdev_flow_state *state;
-
-    *statep = state = xmalloc(sizeof *state);
-    state->bucket = 0;
-    state->offset = 0;
-    state->actions = NULL;
-    return 0;
+    return CONTAINER_OF(dump, struct dpif_netdev_flow_dump, up);
 }
 
-static int
-dpif_netdev_flow_dump_next(const struct dpif *dpif, void *state_,
-                           const struct nlattr **key, size_t *key_len,
-                           const struct nlattr **mask, size_t *mask_len,
-                           const struct nlattr **actions, size_t *actions_len,
-                           const struct dpif_flow_stats **stats)
+static struct dpif_flow_dump *
+dpif_netdev_flow_dump_create(const struct dpif *dpif_)
 {
-    struct dp_netdev_flow_state *state = state_;
-    struct dp_netdev *dp = get_dp_netdev(dpif);
-    struct dp_netdev_flow *netdev_flow;
-    struct hmap_node *node;
+    struct dpif_netdev_flow_dump *dump;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    node = hmap_at_position(&dp->flow_table, &state->bucket, &state->offset);
-    if (!node) {
-        ovs_mutex_unlock(&dp_netdev_mutex);
-        return EOF;
-    }
+    dump = xmalloc(sizeof *dump);
+    dpif_flow_dump_init(&dump->up, dpif_);
+    memset(&dump->pos, 0, sizeof dump->pos);
+    dump->status = 0;
+    ovs_mutex_init(&dump->mutex);
 
-    netdev_flow = CONTAINER_OF(node, struct dp_netdev_flow, node);
+    return &dump->up;
+}
 
-    if (key) {
-        struct ofpbuf buf;
+static int
+dpif_netdev_flow_dump_destroy(struct dpif_flow_dump *dump_)
+{
+    struct dpif_netdev_flow_dump *dump = dpif_netdev_flow_dump_cast(dump_);
 
-        ofpbuf_use_stack(&buf, &state->keybuf, sizeof state->keybuf);
-        odp_flow_key_from_flow(&buf, &netdev_flow->flow,
-                               netdev_flow->flow.in_port.odp_port);
+    ovs_mutex_destroy(&dump->mutex);
+    free(dump);
+    return 0;
+}
 
-        *key = buf.data;
-        *key_len = buf.size;
-    }
+struct dpif_netdev_flow_dump_thread {
+    struct dpif_flow_dump_thread up;
+    struct dpif_netdev_flow_dump *dump;
+    struct odputil_keybuf keybuf[FLOW_DUMP_MAX_BATCH];
+    struct odputil_keybuf maskbuf[FLOW_DUMP_MAX_BATCH];
+};
 
-    if (key && mask) {
-        struct ofpbuf buf;
+static struct dpif_netdev_flow_dump_thread *
+dpif_netdev_flow_dump_thread_cast(struct dpif_flow_dump_thread *thread)
+{
+    return CONTAINER_OF(thread, struct dpif_netdev_flow_dump_thread, up);
+}
+
+static struct dpif_flow_dump_thread *
+dpif_netdev_flow_dump_thread_create(struct dpif_flow_dump *dump_)
+{
+    struct dpif_netdev_flow_dump *dump = dpif_netdev_flow_dump_cast(dump_);
+    struct dpif_netdev_flow_dump_thread *thread;
+
+    thread = xmalloc(sizeof *thread);
+    dpif_flow_dump_thread_init(&thread->up, &dump->up);
+    thread->dump = dump;
+    return &thread->up;
+}
+
+static void
+dpif_netdev_flow_dump_thread_destroy(struct dpif_flow_dump_thread *thread_)
+{
+    struct dpif_netdev_flow_dump_thread *thread
+        = dpif_netdev_flow_dump_thread_cast(thread_);
+
+    free(thread);
+}
+
+static int
+dpif_netdev_flow_dump_next(struct dpif_flow_dump_thread *thread_,
+                           struct dpif_flow *flows, int max_flows)
+{
+    struct dpif_netdev_flow_dump_thread *thread
+        = dpif_netdev_flow_dump_thread_cast(thread_);
+    struct dpif_netdev_flow_dump *dump = thread->dump;
+    struct dpif_netdev *dpif = dpif_netdev_cast(thread->up.dpif);
+    struct dp_netdev_flow *netdev_flows[FLOW_DUMP_MAX_BATCH];
+    struct dp_netdev *dp = get_dp_netdev(&dpif->dpif);
+    int n_flows = 0;
+    int i;
+
+    ovs_mutex_lock(&dump->mutex);
+    if (!dump->status) {
+        for (n_flows = 0; n_flows < MIN(max_flows, FLOW_DUMP_MAX_BATCH);
+             n_flows++) {
+            struct cmap_node *node;
+
+            node = cmap_next_position(&dp->flow_table, &dump->pos);
+            if (!node) {
+                dump->status = EOF;
+                break;
+            }
+            netdev_flows[n_flows] = CONTAINER_OF(node, struct dp_netdev_flow,
+                                                 node);
+        }
+    }
+    ovs_mutex_unlock(&dump->mutex);
+
+    for (i = 0; i < n_flows; i++) {
+        struct odputil_keybuf *maskbuf = &thread->maskbuf[i];
+        struct odputil_keybuf *keybuf = &thread->keybuf[i];
+        struct dp_netdev_flow *netdev_flow = netdev_flows[i];
+        struct dpif_flow *f = &flows[i];
+        struct dp_netdev_actions *dp_actions;
         struct flow_wildcards wc;
+        struct ofpbuf buf;
 
-        ofpbuf_use_stack(&buf, &state->maskbuf, sizeof state->maskbuf);
         minimask_expand(&netdev_flow->cr.match.mask, &wc);
-        odp_flow_key_from_mask(&buf, &wc.masks, &netdev_flow->flow,
-                               odp_to_u32(wc.masks.in_port.odp_port));
 
-        *mask = buf.data;
-        *mask_len = buf.size;
+        /* Key. */
+        ofpbuf_use_stack(&buf, keybuf, sizeof *keybuf);
+        odp_flow_key_from_flow(&buf, &netdev_flow->flow, &wc.masks,
+                               netdev_flow->flow.in_port.odp_port, true);
+        f->key = ofpbuf_data(&buf);
+        f->key_len = ofpbuf_size(&buf);
+
+        /* Mask. */
+        ofpbuf_use_stack(&buf, maskbuf, sizeof *maskbuf);
+        odp_flow_key_from_mask(&buf, &wc.masks, &netdev_flow->flow,
+                               odp_to_u32(wc.masks.in_port.odp_port),
+                               SIZE_MAX, true);
+        f->mask = ofpbuf_data(&buf);
+        f->mask_len = ofpbuf_size(&buf);
+
+        /* Actions. */
+        dp_actions = dp_netdev_flow_get_actions(netdev_flow);
+        f->actions = dp_actions->actions;
+        f->actions_len = dp_actions->size;
+
+        /* Stats. */
+        get_dpif_flow_stats(netdev_flow, &f->stats);
     }
 
-    if (actions) {
-        free(state->actions);
-        state->actions = xmemdup(netdev_flow->actions,
-                         netdev_flow->actions_len);
+    return n_flows;
+}
 
-        *actions = state->actions;
-        *actions_len = netdev_flow->actions_len;
-    }
+static int
+dpif_netdev_execute(struct dpif *dpif, struct dpif_execute *execute)
+{
+    struct dp_netdev *dp = get_dp_netdev(dpif);
+    struct dpif_packet packet, *pp;
+    struct pkt_metadata *md = &execute->md;
 
-    if (stats) {
-        get_dpif_flow_stats(netdev_flow, &state->stats);
-        *stats = &state->stats;
+    if (ofpbuf_size(execute->packet) < ETH_HEADER_LEN ||
+        ofpbuf_size(execute->packet) > UINT16_MAX) {
+        return EINVAL;
     }
 
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    packet.ofpbuf = *execute->packet;
+    pp = &packet;
+
+    dp_netdev_execute_actions(dp, &pp, 1, false, md,
+                              execute->actions, execute->actions_len);
+
+    /* Even though may_steal is set to false, some actions could modify or
+     * reallocate the ofpbuf memory. We need to pass those changes to the
+     * caller */
+    *execute->packet = packet.ofpbuf;
+
     return 0;
 }
 
-static int
-dpif_netdev_flow_dump_done(const struct dpif *dpif OVS_UNUSED, void *state_)
+static void
+dp_netdev_destroy_all_queues(struct dp_netdev *dp)
+    OVS_REQ_WRLOCK(dp->queue_rwlock)
 {
-    struct dp_netdev_flow_state *state = state_;
+    size_t i;
 
-    free(state->actions);
-    free(state);
-    return 0;
+    dp_netdev_purge_queues(dp);
+
+    for (i = 0; i < dp->n_handlers; i++) {
+        struct dp_netdev_queue *q = &dp->handler_queues[i];
+
+        ovs_mutex_destroy(&q->mutex);
+        seq_destroy(q->seq);
+    }
+    free(dp->handler_queues);
+    dp->handler_queues = NULL;
+    dp->n_handlers = 0;
+}
+
+static void
+dp_netdev_refresh_queues(struct dp_netdev *dp, uint32_t n_handlers)
+    OVS_REQ_WRLOCK(dp->queue_rwlock)
+{
+    if (dp->n_handlers != n_handlers) {
+        size_t i;
+
+        dp_netdev_destroy_all_queues(dp);
+
+        dp->n_handlers = n_handlers;
+        dp->handler_queues = xzalloc(n_handlers * sizeof *dp->handler_queues);
+
+        for (i = 0; i < n_handlers; i++) {
+            struct dp_netdev_queue *q = &dp->handler_queues[i];
+
+            ovs_mutex_init(&q->mutex);
+            q->seq = seq_create();
+        }
+    }
 }
 
 static int
-dpif_netdev_execute(struct dpif *dpif, struct dpif_execute *execute)
+dpif_netdev_recv_set(struct dpif *dpif, bool enable)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
-    struct pkt_metadata *md = &execute->md;
-    struct flow key;
 
-    if (execute->packet->size < ETH_HEADER_LEN ||
-        execute->packet->size > UINT16_MAX) {
-        return EINVAL;
+    if ((dp->handler_queues != NULL) == enable) {
+        return 0;
     }
 
-    /* Extract flow key. */
-    flow_extract(execute->packet, md->skb_priority, md->pkt_mark, &md->tunnel,
-                 (union flow_in_port *)&md->in_port, &key);
-    ovs_mutex_lock(&dp_netdev_mutex);
-    dp_netdev_execute_actions(dp, &key, execute->packet, md, execute->actions,
-                              execute->actions_len);
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    fat_rwlock_wrlock(&dp->queue_rwlock);
+    if (!enable) {
+        dp_netdev_destroy_all_queues(dp);
+    } else {
+        dp_netdev_refresh_queues(dp, 1);
+    }
+    fat_rwlock_unlock(&dp->queue_rwlock);
+
     return 0;
 }
 
 static int
-dpif_netdev_recv_set(struct dpif *dpif OVS_UNUSED, bool enable OVS_UNUSED)
+dpif_netdev_handlers_set(struct dpif *dpif, uint32_t n_handlers)
 {
+    struct dp_netdev *dp = get_dp_netdev(dpif);
+
+    fat_rwlock_wrlock(&dp->queue_rwlock);
+    if (dp->handler_queues) {
+        dp_netdev_refresh_queues(dp, n_handlers);
+    }
+    fat_rwlock_unlock(&dp->queue_rwlock);
+
     return 0;
 }
 
@@ -1178,104 +1641,145 @@ dpif_netdev_queue_to_priority(const struct dpif *dpif OVS_UNUSED,
     return 0;
 }
 
-static struct dp_netdev_queue *
-find_nonempty_queue(struct dpif *dpif)
+static bool
+dp_netdev_recv_check(const struct dp_netdev *dp, const uint32_t handler_id)
+    OVS_REQ_RDLOCK(dp->queue_rwlock)
 {
-    struct dp_netdev *dp = get_dp_netdev(dpif);
-    int i;
+    static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
 
-    for (i = 0; i < N_QUEUES; i++) {
-        struct dp_netdev_queue *q = &dp->queues[i];
-        if (q->head != q->tail) {
-            return q;
-        }
+    if (!dp->handler_queues) {
+        VLOG_WARN_RL(&rl, "receiving upcall disabled");
+        return false;
     }
-    return NULL;
+
+    if (handler_id >= dp->n_handlers) {
+        VLOG_WARN_RL(&rl, "handler index out of bound");
+        return false;
+    }
+
+    return true;
 }
 
 static int
-dpif_netdev_recv(struct dpif *dpif, struct dpif_upcall *upcall,
-                 struct ofpbuf *buf)
+dpif_netdev_recv(struct dpif *dpif, uint32_t handler_id,
+                 struct dpif_upcall *upcall, struct ofpbuf *buf)
 {
+    struct dp_netdev *dp = get_dp_netdev(dpif);
     struct dp_netdev_queue *q;
-    int error;
+    int error = 0;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    q = find_nonempty_queue(dpif);
-    if (q) {
+    fat_rwlock_rdlock(&dp->queue_rwlock);
+
+    if (!dp_netdev_recv_check(dp, handler_id)) {
+        error = EAGAIN;
+        goto out;
+    }
+
+    q = &dp->handler_queues[handler_id];
+    ovs_mutex_lock(&q->mutex);
+    if (q->head != q->tail) {
         struct dp_netdev_upcall *u = &q->upcalls[q->tail++ & QUEUE_MASK];
 
         *upcall = u->upcall;
 
         ofpbuf_uninit(buf);
         *buf = u->buf;
-
-        error = 0;
     } else {
         error = EAGAIN;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&q->mutex);
+
+out:
+    fat_rwlock_unlock(&dp->queue_rwlock);
 
     return error;
 }
 
 static void
-dpif_netdev_recv_wait(struct dpif *dpif)
+dpif_netdev_recv_wait(struct dpif *dpif, uint32_t handler_id)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
+    struct dp_netdev_queue *q;
     uint64_t seq;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    seq = seq_read(dp->queue_seq);
-    if (find_nonempty_queue(dpif)) {
+    fat_rwlock_rdlock(&dp->queue_rwlock);
+
+    if (!dp_netdev_recv_check(dp, handler_id)) {
+        goto out;
+    }
+
+    q = &dp->handler_queues[handler_id];
+    ovs_mutex_lock(&q->mutex);
+    seq = seq_read(q->seq);
+    if (q->head != q->tail) {
         poll_immediate_wake();
     } else {
-        seq_wait(dp->queue_seq, seq);
+        seq_wait(q->seq, seq);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+
+    ovs_mutex_unlock(&q->mutex);
+
+out:
+    fat_rwlock_unlock(&dp->queue_rwlock);
 }
 
 static void
 dpif_netdev_recv_purge(struct dpif *dpif)
 {
     struct dpif_netdev *dpif_netdev = dpif_netdev_cast(dpif);
-    ovs_mutex_lock(&dp_netdev_mutex);
+
+    fat_rwlock_wrlock(&dpif_netdev->dp->queue_rwlock);
     dp_netdev_purge_queues(dpif_netdev->dp);
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    fat_rwlock_unlock(&dpif_netdev->dp->queue_rwlock);
 }
 \f
+/* Creates and returns a new 'struct dp_netdev_actions', with a reference count
+ * of 1, whose actions are a copy of from the 'ofpacts_len' bytes of
+ * 'ofpacts'. */
+struct dp_netdev_actions *
+dp_netdev_actions_create(const struct nlattr *actions, size_t size)
+{
+    struct dp_netdev_actions *netdev_actions;
+
+    netdev_actions = xmalloc(sizeof *netdev_actions);
+    netdev_actions->actions = xmemdup(actions, size);
+    netdev_actions->size = size;
+
+    return netdev_actions;
+}
+
+struct dp_netdev_actions *
+dp_netdev_flow_get_actions(const struct dp_netdev_flow *flow)
+{
+    return ovsrcu_get(struct dp_netdev_actions *, &flow->actions);
+}
+
 static void
-dp_netdev_flow_used(struct dp_netdev_flow *netdev_flow,
-                    const struct ofpbuf *packet)
+dp_netdev_actions_free(struct dp_netdev_actions *actions)
 {
-    netdev_flow->used = time_msec();
-    netdev_flow->packet_count++;
-    netdev_flow->byte_count += packet->size;
-    netdev_flow->tcp_flags |= packet_get_tcp_flags(packet, &netdev_flow->flow);
+    free(actions->actions);
+    free(actions);
 }
+\f
 
 static void
-dp_netdev_port_input(struct dp_netdev *dp, struct ofpbuf *packet,
-                     struct pkt_metadata *md)
+dp_netdev_process_rxq_port(struct dp_netdev *dp,
+                          struct dp_netdev_port *port,
+                          struct netdev_rxq *rxq)
 {
-    struct dp_netdev_flow *netdev_flow;
-    struct flow key;
+    struct dpif_packet *packets[NETDEV_MAX_RX_BATCH];
+    int error, cnt;
 
-    if (packet->size < ETH_HEADER_LEN) {
-        return;
-    }
-    flow_extract(packet, md->skb_priority, md->pkt_mark, &md->tunnel,
-                 (union flow_in_port *)&md->in_port, &key);
-    netdev_flow = dp_netdev_lookup_flow(dp, &key);
-    if (netdev_flow) {
-        dp_netdev_flow_used(netdev_flow, packet);
-        dp_netdev_execute_actions(dp, &key, packet, md,
-                                  netdev_flow->actions,
-                                  netdev_flow->actions_len);
-        ovsthread_counter_inc(dp->n_hit, 1);
-    } else {
-        ovsthread_counter_inc(dp->n_missed, 1);
-        dp_netdev_output_userspace(dp, packet, DPIF_UC_MISS, &key, NULL);
+    error = netdev_rxq_recv(rxq, packets, &cnt);
+    if (!error) {
+        dp_netdev_port_input(dp, packets, cnt, port->port_no);
+    } else if (error != EAGAIN && error != EOPNOTSUPP) {
+        static struct vlog_rate_limit rl
+            = VLOG_RATE_LIMIT_INIT(1, 5);
+
+        VLOG_ERR_RL(&rl, "error receiving data from %s: %s",
+                    netdev_get_name(port->netdev),
+                    ovs_strerror(error));
     }
 }
 
@@ -1283,159 +1787,556 @@ static void
 dpif_netdev_run(struct dpif *dpif)
 {
     struct dp_netdev_port *port;
-    struct dp_netdev *dp;
-    struct ofpbuf packet;
-
-    ovs_mutex_lock(&dp_netdev_mutex);
-    dp = get_dp_netdev(dpif);
-    ofpbuf_init(&packet, 0);
+    struct dp_netdev *dp = get_dp_netdev(dpif);
 
-    HMAP_FOR_EACH (port, node, &dp->ports) {
-        int buf_size;
-        int error;
-        int mtu;
+    CMAP_FOR_EACH (port, node, &dp->ports) {
+        if (!netdev_is_pmd(port->netdev)) {
+            int i;
 
-        error = netdev_get_mtu(port->netdev, &mtu);
-        if (error) {
-            mtu = ETH_PAYLOAD_MAX;
+            for (i = 0; i < netdev_n_rxq(port->netdev); i++) {
+                dp_netdev_process_rxq_port(dp, port, port->rxq[i]);
+            }
         }
-        buf_size = DP_NETDEV_HEADROOM + VLAN_ETH_HEADER_LEN + mtu;
+    }
+}
 
-        ofpbuf_clear(&packet);
-        ofpbuf_reserve_with_tailroom(&packet, DP_NETDEV_HEADROOM, buf_size);
+static void
+dpif_netdev_wait(struct dpif *dpif)
+{
+    struct dp_netdev_port *port;
+    struct dp_netdev *dp = get_dp_netdev(dpif);
 
-        error = port->rx ? netdev_rx_recv(port->rx, &packet) : EOPNOTSUPP;
-        if (!error) {
-            struct pkt_metadata md = PKT_METADATA_INITIALIZER(port->port_no);
-            dp_netdev_port_input(dp, &packet, &md);
-        } else if (error != EAGAIN && error != EOPNOTSUPP) {
-            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
+    ovs_mutex_lock(&dp_netdev_mutex);
+    CMAP_FOR_EACH (port, node, &dp->ports) {
+        if (!netdev_is_pmd(port->netdev)) {
+            int i;
 
-            VLOG_ERR_RL(&rl, "error receiving data from %s: %s",
-                        netdev_get_name(port->netdev), ovs_strerror(error));
+            for (i = 0; i < netdev_n_rxq(port->netdev); i++) {
+                netdev_rxq_wait(port->rxq[i]);
+            }
         }
     }
-    ofpbuf_uninit(&packet);
     ovs_mutex_unlock(&dp_netdev_mutex);
 }
 
-static void
-dpif_netdev_wait(struct dpif *dpif)
+struct rxq_poll {
+    struct dp_netdev_port *port;
+    struct netdev_rxq *rx;
+};
+
+static int
+pmd_load_queues(struct pmd_thread *f,
+                struct rxq_poll **ppoll_list, int poll_cnt)
 {
+    struct dp_netdev *dp = f->dp;
+    struct rxq_poll *poll_list = *ppoll_list;
     struct dp_netdev_port *port;
+    int id = f->id;
+    int index;
+    int i;
 
-    /* There is a race here, if thread A calls dpif_netdev_wait(dpif) and
-     * thread B calls dpif_port_add(dpif) or dpif_port_remove(dpif) before
-     * A makes it to poll_block().
-     *
-     * But I think it doesn't matter:
-     *
-     *     - In the dpif_port_add() case, A will not wake up when a packet
-     *       arrives on the new port, but this would also happen if the
-     *       ordering were reversed.
-     *
-     *     - In the dpif_port_remove() case, A might wake up spuriously, but
-     *       that is harmless. */
+    /* Simple scheduler for netdev rx polling. */
+    for (i = 0; i < poll_cnt; i++) {
+         port_unref(poll_list[i].port);
+    }
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    HMAP_FOR_EACH (port, node, &get_dp_netdev(dpif)->ports) {
-        if (port->rx) {
-            netdev_rx_wait(port->rx);
+    poll_cnt = 0;
+    index = 0;
+
+    CMAP_FOR_EACH (port, node, &f->dp->ports) {
+        if (netdev_is_pmd(port->netdev)) {
+            int i;
+
+            for (i = 0; i < netdev_n_rxq(port->netdev); i++) {
+                if ((index % dp->n_pmd_threads) == id) {
+                    poll_list = xrealloc(poll_list, sizeof *poll_list * (poll_cnt + 1));
+
+                    port_ref(port);
+                    poll_list[poll_cnt].port = port;
+                    poll_list[poll_cnt].rx = port->rxq[i];
+                    poll_cnt++;
+                }
+                index++;
+            }
         }
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+
+    *ppoll_list = poll_list;
+    return poll_cnt;
+}
+
+static void *
+pmd_thread_main(void *f_)
+{
+    struct pmd_thread *f = f_;
+    struct dp_netdev *dp = f->dp;
+    unsigned int lc = 0;
+    struct rxq_poll *poll_list;
+    unsigned int port_seq;
+    int poll_cnt;
+    int i;
+
+    poll_cnt = 0;
+    poll_list = NULL;
+
+    pmd_thread_setaffinity_cpu(f->id);
+reload:
+    poll_cnt = pmd_load_queues(f, &poll_list, poll_cnt);
+    atomic_read(&f->change_seq, &port_seq);
+
+    for (;;) {
+        unsigned int c_port_seq;
+        int i;
+
+        for (i = 0; i < poll_cnt; i++) {
+            dp_netdev_process_rxq_port(dp,  poll_list[i].port, poll_list[i].rx);
+        }
+
+        if (lc++ > 1024) {
+            ovsrcu_quiesce();
+
+            /* TODO: need completely userspace based signaling method.
+             * to keep this thread entirely in userspace.
+             * For now using atomic counter. */
+            lc = 0;
+            atomic_read_explicit(&f->change_seq, &c_port_seq, memory_order_consume);
+            if (c_port_seq != port_seq) {
+                break;
+            }
+        }
+    }
+
+    if (!latch_is_set(&f->dp->exit_latch)){
+        goto reload;
+    }
+
+    for (i = 0; i < poll_cnt; i++) {
+         port_unref(poll_list[i].port);
+    }
+
+    free(poll_list);
+    return NULL;
+}
+
+static void
+dp_netdev_set_pmd_threads(struct dp_netdev *dp, int n)
+{
+    int i;
+
+    if (n == dp->n_pmd_threads) {
+        return;
+    }
+
+    /* Stop existing threads. */
+    latch_set(&dp->exit_latch);
+    dp_netdev_reload_pmd_threads(dp);
+    for (i = 0; i < dp->n_pmd_threads; i++) {
+        struct pmd_thread *f = &dp->pmd_threads[i];
+
+        xpthread_join(f->thread, NULL);
+    }
+    latch_poll(&dp->exit_latch);
+    free(dp->pmd_threads);
+
+    /* Start new threads. */
+    dp->pmd_threads = xmalloc(n * sizeof *dp->pmd_threads);
+    dp->n_pmd_threads = n;
+
+    for (i = 0; i < n; i++) {
+        struct pmd_thread *f = &dp->pmd_threads[i];
+
+        f->dp = dp;
+        f->id = i;
+        atomic_store(&f->change_seq, 1);
+
+        /* Each thread will distribute all devices rx-queues among
+         * themselves. */
+        f->thread = ovs_thread_create("pmd", pmd_thread_main, f);
+    }
+}
+
+\f
+static void *
+dp_netdev_flow_stats_new_cb(void)
+{
+    struct dp_netdev_flow_stats *bucket = xzalloc_cacheline(sizeof *bucket);
+    ovs_mutex_init(&bucket->mutex);
+    return bucket;
 }
 
 static void
-dp_netdev_output_port(struct dp_netdev *dp, struct ofpbuf *packet,
-                      odp_port_t out_port)
+dp_netdev_flow_used(struct dp_netdev_flow *netdev_flow,
+                    int cnt, int size,
+                    uint16_t tcp_flags)
+{
+    long long int now = time_msec();
+    struct dp_netdev_flow_stats *bucket;
+
+    bucket = ovsthread_stats_bucket_get(&netdev_flow->stats,
+                                        dp_netdev_flow_stats_new_cb);
+
+    ovs_mutex_lock(&bucket->mutex);
+    bucket->used = MAX(now, bucket->used);
+    bucket->packet_count += cnt;
+    bucket->byte_count += size;
+    bucket->tcp_flags |= tcp_flags;
+    ovs_mutex_unlock(&bucket->mutex);
+}
+
+static void *
+dp_netdev_stats_new_cb(void)
+{
+    struct dp_netdev_stats *bucket = xzalloc_cacheline(sizeof *bucket);
+    ovs_mutex_init(&bucket->mutex);
+    return bucket;
+}
+
+static void
+dp_netdev_count_packet(struct dp_netdev *dp, enum dp_stat_type type, int cnt)
+{
+    struct dp_netdev_stats *bucket;
+
+    bucket = ovsthread_stats_bucket_get(&dp->stats, dp_netdev_stats_new_cb);
+    ovs_mutex_lock(&bucket->mutex);
+    bucket->n[type] += cnt;
+    ovs_mutex_unlock(&bucket->mutex);
+}
+
+struct packet_batch {
+    unsigned int packet_count;
+    unsigned int byte_count;
+    uint16_t tcp_flags;
+
+    struct dp_netdev_flow *flow;
+
+    struct dpif_packet *packets[NETDEV_MAX_RX_BATCH];
+    struct pkt_metadata md;
+};
+
+static inline void
+packet_batch_update(struct packet_batch *batch,
+                    struct dpif_packet *packet, const struct miniflow *mf)
 {
-    struct dp_netdev_port *p = dp_netdev_lookup_port(dp, out_port);
-    if (p) {
-        netdev_send(p->netdev, packet);
+    batch->tcp_flags |= miniflow_get_tcp_flags(mf);
+    batch->packets[batch->packet_count++] = packet;
+    batch->byte_count += ofpbuf_size(&packet->ofpbuf);
+}
+
+static inline void
+packet_batch_init(struct packet_batch *batch, struct dp_netdev_flow *flow,
+                  struct pkt_metadata *md)
+{
+    batch->flow = flow;
+    batch->md = *md;
+
+    batch->packet_count = 0;
+    batch->byte_count = 0;
+    batch->tcp_flags = 0;
+}
+
+static inline void
+packet_batch_execute(struct packet_batch *batch, struct dp_netdev *dp)
+{
+    struct dp_netdev_actions *actions;
+    struct dp_netdev_flow *flow = batch->flow;
+
+    dp_netdev_flow_used(batch->flow, batch->packet_count, batch->byte_count,
+                        batch->tcp_flags);
+
+    actions = dp_netdev_flow_get_actions(flow);
+
+    dp_netdev_execute_actions(dp, batch->packets,
+                              batch->packet_count, true, &batch->md,
+                              actions->actions, actions->size);
+
+    dp_netdev_count_packet(dp, DP_STAT_HIT, batch->packet_count);
+}
+
+static void
+dp_netdev_input(struct dp_netdev *dp, struct dpif_packet **packets, int cnt,
+                struct pkt_metadata *md)
+{
+    struct packet_batch batches[NETDEV_MAX_RX_BATCH];
+    struct netdev_flow_key keys[NETDEV_MAX_RX_BATCH];
+    const struct miniflow *mfs[NETDEV_MAX_RX_BATCH]; /* NULL at bad packets. */
+    struct cls_rule *rules[NETDEV_MAX_RX_BATCH];
+    size_t n_batches, i;
+
+    for (i = 0; i < cnt; i++) {
+        if (OVS_UNLIKELY(ofpbuf_size(&packets[i]->ofpbuf) < ETH_HEADER_LEN)) {
+            dpif_packet_delete(packets[i]);
+            mfs[i] = NULL;
+            continue;
+        }
+
+        miniflow_initialize(&keys[i].flow, keys[i].buf);
+        miniflow_extract(&packets[i]->ofpbuf, md, &keys[i].flow);
+        mfs[i] = &keys[i].flow;
+    }
+
+    classifier_lookup_miniflow_batch(&dp->cls, mfs, rules, cnt);
+
+    n_batches = 0;
+    for (i = 0; i < cnt; i++) {
+        struct dp_netdev_flow *flow;
+        struct packet_batch *batch;
+        size_t j;
+
+        if (OVS_UNLIKELY(!mfs[i])) {
+            continue;
+        }
+
+        if (OVS_UNLIKELY(!rules[i])) {
+
+            dp_netdev_count_packet(dp, DP_STAT_MISS, 1);
+
+            if (OVS_LIKELY(dp->handler_queues)) {
+                uint32_t hash = miniflow_hash_5tuple(mfs[i], 0);
+                struct ofpbuf *buf = &packets[i]->ofpbuf;
+
+                dp_netdev_output_userspace(dp, buf, hash % dp->n_handlers,
+                                           DPIF_UC_MISS, mfs[i], NULL);
+            }
+
+            dpif_packet_delete(packets[i]);
+            continue;
+        }
+
+        /* XXX: This O(n^2) algortihm makes sense if we're operating under the
+         * assumption that the number of distinct flows (and therefore the
+         * number of distinct batches) is quite small.  If this turns out not
+         * to be the case, it may make sense to pre sort based on the
+         * netdev_flow pointer.  That done we can get the appropriate batching
+         * in O(n * log(n)) instead. */
+        batch = NULL;
+        flow = dp_netdev_flow_cast(rules[i]);
+        for (j = 0; j < n_batches; j++) {
+            if (batches[j].flow == flow) {
+                batch = &batches[j];
+                break;
+            }
+        }
+
+        if (!batch) {
+            batch = &batches[n_batches++];
+            packet_batch_init(batch, flow, md);
+        }
+        packet_batch_update(batch, packets[i], mfs[i]);
     }
+
+    for (i = 0; i < n_batches; i++) {
+        packet_batch_execute(&batches[i], dp);
+    }
+}
+
+static void
+dp_netdev_port_input(struct dp_netdev *dp, struct dpif_packet **packets,
+                     int cnt, odp_port_t port_no)
+{
+    uint32_t *recirc_depth = recirc_depth_get();
+    struct pkt_metadata md = PKT_METADATA_INITIALIZER(port_no);
+
+    *recirc_depth = 0;
+    dp_netdev_input(dp, packets, cnt, &md);
 }
 
 static int
-dp_netdev_output_userspace(struct dp_netdev *dp, struct ofpbuf *packet,
-                           int queue_no, const struct flow *flow,
-                           const struct nlattr *userdata)
+dp_netdev_queue_userspace_packet(struct dp_netdev_queue *q,
+                                 struct ofpbuf *packet, int type,
+                                 const struct miniflow *key,
+                                 const struct nlattr *userdata)
+OVS_REQUIRES(q->mutex)
 {
-    struct dp_netdev_queue *q = &dp->queues[queue_no];
     if (q->head - q->tail < MAX_QUEUE_LEN) {
         struct dp_netdev_upcall *u = &q->upcalls[q->head++ & QUEUE_MASK];
         struct dpif_upcall *upcall = &u->upcall;
         struct ofpbuf *buf = &u->buf;
         size_t buf_size;
+        struct flow flow;
+        void *data;
 
-        upcall->type = queue_no;
+        upcall->type = type;
 
         /* Allocate buffer big enough for everything. */
         buf_size = ODPUTIL_FLOW_KEY_BYTES;
         if (userdata) {
             buf_size += NLA_ALIGN(userdata->nla_len);
         }
+        buf_size += ofpbuf_size(packet);
         ofpbuf_init(buf, buf_size);
 
         /* Put ODP flow. */
-        odp_flow_key_from_flow(buf, flow, flow->in_port.odp_port);
-        upcall->key = buf->data;
-        upcall->key_len = buf->size;
+        miniflow_expand(key, &flow);
+        odp_flow_key_from_flow(buf, &flow, NULL, flow.in_port.odp_port, true);
+        upcall->key = ofpbuf_data(buf);
+        upcall->key_len = ofpbuf_size(buf);
 
         /* Put userdata. */
         if (userdata) {
             upcall->userdata = ofpbuf_put(buf, userdata,
-                                          NLA_ALIGN(userdata->nla_len));
+                    NLA_ALIGN(userdata->nla_len));
         }
 
-        /* Steal packet data. */
-        ovs_assert(packet->source == OFPBUF_MALLOC);
-        upcall->packet = *packet;
-        ofpbuf_use(packet, NULL, 0);
+        /* We have to perform a copy of the packet, because we cannot send DPDK
+         * mbufs to a non pmd thread. When the upcall processing will be done
+         * in the pmd thread, this copy can be avoided */
+        data = ofpbuf_put(buf, ofpbuf_data(packet), ofpbuf_size(packet));
+        ofpbuf_use_stub(&upcall->packet, data, ofpbuf_size(packet));
+        ofpbuf_set_size(&upcall->packet, ofpbuf_size(packet));
 
-        seq_change(dp->queue_seq);
+        seq_change(q->seq);
 
         return 0;
     } else {
-        ovsthread_counter_inc(dp->n_lost, 1);
         return ENOBUFS;
     }
 }
 
+static int
+dp_netdev_output_userspace(struct dp_netdev *dp, struct ofpbuf *packet,
+                           int queue_no, int type,
+                           const struct miniflow *key,
+                           const struct nlattr *userdata)
+{
+    struct dp_netdev_queue *q;
+    int error;
+
+    fat_rwlock_rdlock(&dp->queue_rwlock);
+    q = &dp->handler_queues[queue_no];
+    ovs_mutex_lock(&q->mutex);
+    error = dp_netdev_queue_userspace_packet(q, packet, type, key,
+                                             userdata);
+    if (error == ENOBUFS) {
+        dp_netdev_count_packet(dp, DP_STAT_LOST, 1);
+    }
+    ovs_mutex_unlock(&q->mutex);
+    fat_rwlock_unlock(&dp->queue_rwlock);
+
+    return error;
+}
+
 struct dp_netdev_execute_aux {
     struct dp_netdev *dp;
-    const struct flow *key;
 };
 
 static void
-dp_execute_cb(void *aux_, struct ofpbuf *packet,
-              const struct pkt_metadata *md OVS_UNUSED,
+dp_execute_cb(void *aux_, struct dpif_packet **packets, int cnt,
+              struct pkt_metadata *md,
               const struct nlattr *a, bool may_steal)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
 {
     struct dp_netdev_execute_aux *aux = aux_;
     int type = nl_attr_type(a);
+    struct dp_netdev_port *p;
+    uint32_t *depth = recirc_depth_get();
+    int i;
 
     switch ((enum ovs_action_attr)type) {
     case OVS_ACTION_ATTR_OUTPUT:
-        dp_netdev_output_port(aux->dp, packet, u32_to_odp(nl_attr_get_u32(a)));
+        p = dp_netdev_lookup_port(aux->dp, u32_to_odp(nl_attr_get_u32(a)));
+        if (OVS_LIKELY(p)) {
+            netdev_send(p->netdev, packets, cnt, may_steal);
+        } else if (may_steal) {
+            for (i = 0; i < cnt; i++) {
+                dpif_packet_delete(packets[i]);
+            }
+        }
         break;
 
     case OVS_ACTION_ATTR_USERSPACE: {
         const struct nlattr *userdata;
+        struct netdev_flow_key key;
 
         userdata = nl_attr_find_nested(a, OVS_USERSPACE_ATTR_USERDATA);
 
-        /* Make a copy if we are not allowed to steal the packet's data. */
-        if (!may_steal) {
-            packet = ofpbuf_clone_with_headroom(packet, DP_NETDEV_HEADROOM);
+        miniflow_initialize(&key.flow, key.buf);
+
+        for (i = 0; i < cnt; i++) {
+            struct ofpbuf *packet;
+
+            packet = &packets[i]->ofpbuf;
+
+            miniflow_extract(packet, md, &key.flow);
+
+            dp_netdev_output_userspace(aux->dp, packet,
+                                       miniflow_hash_5tuple(&key.flow, 0)
+                                           % aux->dp->n_handlers,
+                                       DPIF_UC_ACTION, &key.flow,
+                                       userdata);
+            if (may_steal) {
+                dpif_packet_delete(packets[i]);
+            }
         }
-        dp_netdev_output_userspace(aux->dp, packet, DPIF_UC_ACTION, aux->key,
-                                   userdata);
-        if (!may_steal) {
-            ofpbuf_uninit(packet);
+        break;
+    }
+
+    case OVS_ACTION_ATTR_HASH: {
+        const struct ovs_action_hash *hash_act;
+        struct netdev_flow_key key;
+        uint32_t hash;
+
+        hash_act = nl_attr_get(a);
+
+        miniflow_initialize(&key.flow, key.buf);
+
+        for (i = 0; i < cnt; i++) {
+
+            /* TODO: this is slow. Use RSS hash in the future */
+            miniflow_extract(&packets[i]->ofpbuf, md, &key.flow);
+
+            if (hash_act->hash_alg == OVS_HASH_ALG_L4) {
+                /* Hash need not be symmetric, nor does it need to include
+                 * L2 fields. */
+                hash = miniflow_hash_5tuple(&key.flow, hash_act->hash_basis);
+            } else {
+                VLOG_WARN("Unknown hash algorithm specified "
+                          "for the hash action.");
+                hash = 2;
+            }
+
+            if (!hash) {
+                hash = 1; /* 0 is not valid */
+            }
+
+            if (i == 0) {
+                md->dp_hash = hash;
+            }
+            packets[i]->dp_hash = hash;
         }
         break;
     }
+
+    case OVS_ACTION_ATTR_RECIRC:
+        if (*depth < MAX_RECIRC_DEPTH) {
+
+            (*depth)++;
+            for (i = 0; i < cnt; i++) {
+                struct dpif_packet *recirc_pkt;
+                struct pkt_metadata recirc_md = *md;
+
+                recirc_pkt = (may_steal) ? packets[i]
+                                    : dpif_packet_clone(packets[i]);
+
+                recirc_md.recirc_id = nl_attr_get_u32(a);
+
+                /* Hash is private to each packet */
+                recirc_md.dp_hash = packets[i]->dp_hash;
+
+                dp_netdev_input(aux->dp, &recirc_pkt, 1, &recirc_md);
+            }
+            (*depth)--;
+
+            break;
+        } else {
+            VLOG_WARN("Packet dropped. Max recirculation depth exceeded.");
+            if (may_steal) {
+                for (i = 0; i < cnt; i++) {
+                    dpif_packet_delete(packets[i]);
+                }
+            }
+        }
+        break;
+
     case OVS_ACTION_ATTR_PUSH_VLAN:
     case OVS_ACTION_ATTR_POP_VLAN:
     case OVS_ACTION_ATTR_PUSH_MPLS:
@@ -1449,13 +2350,15 @@ dp_execute_cb(void *aux_, struct ofpbuf *packet,
 }
 
 static void
-dp_netdev_execute_actions(struct dp_netdev *dp, const struct flow *key,
-                          struct ofpbuf *packet, struct pkt_metadata *md,
+dp_netdev_execute_actions(struct dp_netdev *dp,
+                          struct dpif_packet **packets, int cnt,
+                          bool may_steal, struct pkt_metadata *md,
                           const struct nlattr *actions, size_t actions_len)
 {
-    struct dp_netdev_execute_aux aux = {dp, key};
+    struct dp_netdev_execute_aux aux = {dp};
 
-    odp_execute_actions(&aux, packet, md, actions, actions_len, dp_execute_cb);
+    odp_execute_actions(&aux, packets, cnt, may_steal, md, actions,
+                        actions_len, dp_execute_cb);
 }
 
 const struct dpif_class dpif_netdev_class = {
@@ -1482,12 +2385,15 @@ const struct dpif_class dpif_netdev_class = {
     dpif_netdev_flow_put,
     dpif_netdev_flow_del,
     dpif_netdev_flow_flush,
-    dpif_netdev_flow_dump_start,
+    dpif_netdev_flow_dump_create,
+    dpif_netdev_flow_dump_destroy,
+    dpif_netdev_flow_dump_thread_create,
+    dpif_netdev_flow_dump_thread_destroy,
     dpif_netdev_flow_dump_next,
-    dpif_netdev_flow_dump_done,
     dpif_netdev_execute,
     NULL,                       /* operate */
     dpif_netdev_recv_set,
+    dpif_netdev_handlers_set,
     dpif_netdev_queue_to_priority,
     dpif_netdev_recv,
     dpif_netdev_recv_wait,
@@ -1498,35 +2404,83 @@ static void
 dpif_dummy_change_port_number(struct unixctl_conn *conn, int argc OVS_UNUSED,
                               const char *argv[], void *aux OVS_UNUSED)
 {
-    struct dp_netdev_port *port;
+    struct dp_netdev_port *old_port;
+    struct dp_netdev_port *new_port;
     struct dp_netdev *dp;
     odp_port_t port_no;
 
+    ovs_mutex_lock(&dp_netdev_mutex);
     dp = shash_find_data(&dp_netdevs, argv[1]);
     if (!dp || !dpif_netdev_class_is_dummy(dp->class)) {
+        ovs_mutex_unlock(&dp_netdev_mutex);
         unixctl_command_reply_error(conn, "unknown datapath or not a dummy");
         return;
     }
+    ovs_refcount_ref(&dp->ref_cnt);
+    ovs_mutex_unlock(&dp_netdev_mutex);
 
-    if (get_port_by_name(dp, argv[2], &port)) {
+    ovs_mutex_lock(&dp->port_mutex);
+    if (get_port_by_name(dp, argv[2], &old_port)) {
         unixctl_command_reply_error(conn, "unknown port");
-        return;
+        goto exit;
     }
 
     port_no = u32_to_odp(atoi(argv[3]));
     if (!port_no || port_no == ODPP_NONE) {
         unixctl_command_reply_error(conn, "bad port number");
-        return;
+        goto exit;
     }
     if (dp_netdev_lookup_port(dp, port_no)) {
         unixctl_command_reply_error(conn, "port number already in use");
-        return;
+        goto exit;
     }
-    hmap_remove(&dp->ports, &port->node);
-    port->port_no = port_no;
-    hmap_insert(&dp->ports, &port->node, hash_int(odp_to_u32(port_no), 0));
+
+    /* Remove old port. */
+    cmap_remove(&dp->ports, &old_port->node, hash_port_no(old_port->port_no));
+    ovsrcu_postpone(free, old_port);
+
+    /* Insert new port (cmap semantics mean we cannot re-insert 'old_port'). */
+    new_port = xmemdup(old_port, sizeof *old_port);
+    new_port->port_no = port_no;
+    cmap_insert(&dp->ports, &new_port->node, hash_port_no(port_no));
+
     seq_change(dp->port_seq);
     unixctl_command_reply(conn, NULL);
+
+exit:
+    ovs_mutex_unlock(&dp->port_mutex);
+    dp_netdev_unref(dp);
+}
+
+static void
+dpif_dummy_delete_port(struct unixctl_conn *conn, int argc OVS_UNUSED,
+                       const char *argv[], void *aux OVS_UNUSED)
+{
+    struct dp_netdev_port *port;
+    struct dp_netdev *dp;
+
+    ovs_mutex_lock(&dp_netdev_mutex);
+    dp = shash_find_data(&dp_netdevs, argv[1]);
+    if (!dp || !dpif_netdev_class_is_dummy(dp->class)) {
+        ovs_mutex_unlock(&dp_netdev_mutex);
+        unixctl_command_reply_error(conn, "unknown datapath or not a dummy");
+        return;
+    }
+    ovs_refcount_ref(&dp->ref_cnt);
+    ovs_mutex_unlock(&dp_netdev_mutex);
+
+    ovs_mutex_lock(&dp->port_mutex);
+    if (get_port_by_name(dp, argv[2], &port)) {
+        unixctl_command_reply_error(conn, "unknown port");
+    } else if (port->port_no == ODPP_LOCAL) {
+        unixctl_command_reply_error(conn, "can't delete local port");
+    } else {
+        do_del_port(dp, port);
+        unixctl_command_reply(conn, NULL);
+    }
+    ovs_mutex_unlock(&dp->port_mutex);
+
+    dp_netdev_unref(dp);
 }
 
 static void
@@ -1562,4 +2516,6 @@ dpif_dummy_register(bool override)
     unixctl_command_register("dpif-dummy/change-port-number",
                              "DP PORT NEW-NUMBER",
                              3, 3, dpif_dummy_change_port_number, NULL);
+    unixctl_command_register("dpif-dummy/delete-port", "DP PORT",
+                             2, 2, dpif_dummy_delete_port, NULL);
 }