Merge branch 'mainstream'
[sliver-openvswitch.git] / lib / dpif-netdev.c
index 11f010c..d62949b 100644 (file)
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2009, 2010, 2011, 2012, 2013 Nicira, Inc.
+ * Copyright (c) 2009, 2010, 2011, 2012, 2013, 2014 Nicira, Inc.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -39,7 +39,9 @@
 #include "dynamic-string.h"
 #include "flow.h"
 #include "hmap.h"
+#include "latch.h"
 #include "list.h"
+#include "meta-flow.h"
 #include "netdev.h"
 #include "netdev-vport.h"
 #include "netlink.h"
@@ -64,7 +66,6 @@ VLOG_DEFINE_THIS_MODULE(dpif_netdev);
 #define NETDEV_RULE_PRIORITY 0x8000
 
 /* Configuration parameters. */
-enum { MAX_PORTS = 256 };       /* Maximum number of ports. */
 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
@@ -77,68 +78,222 @@ 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.
+ *
+ *
+ * Thread-safety
+ * =============
+ *
+ * Any access at all requires the owning 'dp_netdev''s queue_mutex. */
 struct dp_netdev_queue {
-    struct dp_netdev_upcall upcalls[MAX_QUEUE_LEN];
-    unsigned int head, tail;
+    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_rwlock
+ *    flow_mutex
+ *    cls.rwlock
+ *    queue_mutex
+ */
 struct dp_netdev {
-    const struct dpif_class *class;
-    char *name;
-    int open_cnt;
-    bool destroyed;
-    int max_mtu;                /* Maximum MTU of any port added so far. */
+    const struct dpif_class *const class;
+    const char *const name;
+    struct ovs_refcount ref_cnt;
+    atomic_flag destroyed;
 
+    /* Flows.
+     *
+     * Readers of 'cls' and 'flow_table' must take a 'cls->rwlock' read lock.
+     *
+     * Writers of 'cls' and 'flow_table' must take the 'flow_mutex' and then
+     * the 'cls->rwlock' write lock.  (The outer 'flow_mutex' allows writers to
+     * atomically perform multiple operations on 'cls' and 'flow_table'.)
+     */
+    struct ovs_mutex flow_mutex;
+    struct classifier cls;      /* Classifier.  Protected by cls.rwlock. */
+    struct hmap flow_table OVS_GUARDED; /* Flow table. */
+
+    /* Queues.
+     *
+     * Everything in 'queues' is protected by 'queue_mutex'. */
+    struct ovs_mutex queue_mutex;
     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. */
-    long long int n_hit;        /* Number of flow table matches. */
-    long long int n_missed;     /* Number of flow table misses. */
-    long long int n_lost;       /* Number of misses not passed to client. */
+    /* Statistics.
+     *
+     * ovsthread_counter is internally synchronized. */
+    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. */
 
-    /* Ports. */
-    struct dp_netdev_port *ports[MAX_PORTS];
-    struct list port_list;
+    /* Ports.
+     *
+     * Any lookup into 'ports' or any access to the dp_netdev_ports found
+     * through 'ports' requires taking 'port_rwlock'. */
+    struct ovs_rwlock port_rwlock;
+    struct hmap ports OVS_GUARDED;
     struct seq *port_seq;       /* Incremented whenever a port changes. */
+
+    /* Forwarding threads. */
+    struct latch exit_latch;
+    struct dp_forwarder *forwarders;
+    size_t n_forwarders;
 };
 
+static struct dp_netdev_port *dp_netdev_lookup_port(const struct dp_netdev *dp,
+                                                    odp_port_t)
+    OVS_REQ_RDLOCK(dp->port_rwlock);
+
 /* A port in a netdev-based datapath. */
 struct dp_netdev_port {
-    odp_port_t port_no;         /* Index into dp_netdev's 'ports'. */
-    struct list node;           /* Element in dp_netdev's 'port_list'. */
+    struct hmap_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;
     char *type;                 /* Port type as requested by user. */
 };
 
-/* A flow in dp_netdev's 'flow_table'. */
+/* 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 (that's 'flow->mutex').
+ *
+ * 'flow->mutex' protects the members of 'flow' from modification.  It doesn't
+ * protect the flow from being deleted from 'cls' (that's 'cls->rwlock') and it
+ * doesn't prevent the flow from being freed (that's 'flow->ref_cnt').
+ *
+ * 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. */
+    const struct hmap_node node; /* In owning dp_netdev's 'flow_table'. */
+    const struct flow flow;      /* The flow that created this entry. */
+
+    /* Number of references.
+     * The classifier owns one reference.
+     * Any thread trying to keep a rule from being freed should hold its own
+     * reference. */
+    struct ovs_refcount ref_cnt;
+
+    /* Protects members marked OVS_GUARDED.
+     *
+     * Acquire after datapath's flow_mutex. */
+    struct ovs_mutex mutex OVS_ACQ_AFTER(dp_netdev_mutex);
+
+    /* Statistics.
+     *
+     * Reading or writing these members requires 'mutex'. */
+    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. */
+
+    /* Actions.
+     *
+     * Reading 'actions' requires 'mutex'.
+     * Writing 'actions' requires 'mutex' and (to allow for transactions) the
+     * datapath's flow_mutex. */
+    struct dp_netdev_actions *actions OVS_GUARDED;
+};
+
+static struct dp_netdev_flow *dp_netdev_flow_ref(
+    const struct dp_netdev_flow *);
+static void dp_netdev_flow_unref(struct dp_netdev_flow *);
 
-    /* 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. */
+/* A set of datapath actions within a "struct dp_netdev_flow".
+ *
+ *
+ * Thread-safety
+ * =============
+ *
+ * A struct dp_netdev_actions 'actions' may be accessed without a risk of being
+ * freed by code that holds a read-lock or write-lock on 'flow->mutex' (where
+ * 'flow' is the dp_netdev_flow for which 'flow->actions == actions') or that
+ * owns a reference to 'actions->ref_cnt' (or both). */
+struct dp_netdev_actions {
+    struct ovs_refcount ref_cnt;
+
+    /* 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. */
+};
 
-    /* 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. */
+struct dp_netdev_actions *dp_netdev_actions_create(const struct nlattr *,
+                                                   size_t);
+struct dp_netdev_actions *dp_netdev_actions_ref(
+    const struct dp_netdev_actions *);
+void dp_netdev_actions_unref(struct dp_netdev_actions *);
 
-    /* Actions. */
-    struct nlattr *actions;
-    size_t actions_len;
+/* A thread that receives packets from some ports, looks them up in the flow
+ * table, and executes the actions it finds. */
+struct dp_forwarder {
+    struct dp_netdev *dp;
+    pthread_t thread;
+    char *name;
+    uint32_t min_hash, max_hash;
 };
 
 /* Interface to netdev-based datapath. */
@@ -148,34 +303,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,
-                              struct dp_netdev_port **portp);
-static int get_port_by_name(struct dp_netdev *, const char *devname,
-                            struct dp_netdev_port **portp);
-static void dp_netdev_free(struct dp_netdev *);
+static int get_port_by_number(struct dp_netdev *dp, odp_port_t port_no,
+                              struct dp_netdev_port **portp)
+    OVS_REQ_RDLOCK(dp->port_rwlock);
+static int get_port_by_name(struct dp_netdev *dp, const char *devname,
+                            struct dp_netdev_port **portp)
+    OVS_REQ_RDLOCK(dp->port_rwlock);
+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_REQ_WRLOCK(dp->port_rwlock);
+static int do_del_port(struct dp_netdev *dp, odp_port_t port_no)
+    OVS_REQ_WRLOCK(dp->port_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 *, const struct ofpbuf *,
+static int dp_netdev_output_userspace(struct dp_netdev *dp, 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 *,
+                                    const struct nlattr *userdata)
+    OVS_EXCLUDED(dp->queue_mutex);
+static void dp_netdev_execute_actions(struct dp_netdev *dp,
+                                      const struct flow *, struct ofpbuf *,
+                                      struct pkt_metadata *,
                                       const struct nlattr *actions,
-                                      size_t actions_len);
-static void dp_netdev_port_input(struct dp_netdev *dp,
-                                 struct dp_netdev_port *port,
-                                 struct ofpbuf *packet, uint32_t skb_priority,
-                                 uint32_t pkt_mark, const struct flow_tnl *tnl);
+                                      size_t actions_len)
+    OVS_REQ_RDLOCK(dp->port_rwlock);
+static void dp_netdev_port_input(struct dp_netdev *dp, struct ofpbuf *packet,
+                                 struct pkt_metadata *)
+    OVS_REQ_RDLOCK(dp->port_rwlock);
+static void dp_netdev_set_threads(struct dp_netdev *, int n);
 
 static struct dpif_netdev *
 dpif_netdev_cast(const struct dpif *dpif)
@@ -231,7 +388,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);
@@ -245,6 +402,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_REQ_RDLOCK(dp->port_rwlock)
 {
     uint32_t port_no;
 
@@ -265,8 +423,8 @@ choose_port(struct dp_netdev *dp, const char *name)
         for (p = name; *p != '\0'; p++) {
             if (isdigit((unsigned char) *p)) {
                 port_no = start_no + strtol(p, NULL, 10);
-                if (port_no > 0 && port_no < MAX_PORTS
-                    && !dp->ports[port_no]) {
+                if (port_no > 0 && port_no != odp_to_u32(ODPP_NONE)
+                    && !dp_netdev_lookup_port(dp, u32_to_odp(port_no))) {
                     return u32_to_odp(port_no);
                 }
                 break;
@@ -274,8 +432,8 @@ choose_port(struct dp_netdev *dp, const char *name)
         }
     }
 
-    for (port_no = 1; port_no < MAX_PORTS; port_no++) {
-        if (!dp->ports[port_no]) {
+    for (port_no = 1; port_no <= UINT16_MAX; port_no++) {
+        if (!dp_netdev_lookup_port(dp, u32_to_odp(port_no))) {
             return u32_to_odp(port_no);
         }
     }
@@ -286,32 +444,49 @@ 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;
-    dp->max_mtu = ETH_PAYLOAD_MAX;
+    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_init(&dp->destroyed);
+
+    ovs_mutex_init(&dp->flow_mutex);
+    classifier_init(&dp->cls, NULL);
+    hmap_init(&dp->flow_table);
+
+    ovs_mutex_init(&dp->queue_mutex);
+    ovs_mutex_lock(&dp->queue_mutex);
     for (i = 0; i < N_QUEUES; i++) {
         dp->queues[i].head = dp->queues[i].tail = 0;
     }
+    ovs_mutex_unlock(&dp->queue_mutex);
     dp->queue_seq = seq_create();
-    classifier_init(&dp->cls, NULL);
-    hmap_init(&dp->flow_table);
-    list_init(&dp->port_list);
+
+    dp->n_hit = ovsthread_counter_create();
+    dp->n_missed = ovsthread_counter_create();
+    dp->n_lost = ovsthread_counter_create();
+
+    ovs_rwlock_init(&dp->port_rwlock);
+    hmap_init(&dp->ports);
     dp->port_seq = seq_create();
+    latch_init(&dp->exit_latch);
 
+    ovs_rwlock_wrlock(&dp->port_rwlock);
     error = do_add_port(dp, name, "internal", ODPP_LOCAL);
+    ovs_rwlock_unlock(&dp->port_rwlock);
     if (error) {
         dp_netdev_free(dp);
         return error;
     }
-
-    shash_add(&dp_netdevs, name, dp);
+    dp_netdev_set_threads(dp, 2);
 
     *dpp = dp;
     return 0;
@@ -346,49 +521,79 @@ dp_netdev_purge_queues(struct dp_netdev *dp)
 {
     int i;
 
+    ovs_mutex_lock(&dp->queue_mutex);
     for (i = 0; i < N_QUEUES; i++) {
         struct dp_netdev_queue *q = &dp->queues[i];
 
         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(&dp->queue_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;
 
+    shash_find_and_delete(&dp_netdevs, dp->name);
+
+    dp_netdev_set_threads(dp, 0);
+    free(dp->forwarders);
+
     dp_netdev_flow_flush(dp);
-    LIST_FOR_EACH_SAFE (port, next, node, &dp->port_list) {
+    ovs_rwlock_wrlock(&dp->port_rwlock);
+    HMAP_FOR_EACH_SAFE (port, next, node, &dp->ports) {
         do_del_port(dp, port->port_no);
     }
+    ovs_rwlock_unlock(&dp->port_rwlock);
+    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_destroy(&dp->queue_mutex);
+
     classifier_destroy(&dp->cls);
     hmap_destroy(&dp->flow_table);
+    ovs_mutex_destroy(&dp->flow_mutex);
     seq_destroy(dp->port_seq);
-    free(dp->name);
+    hmap_destroy(&dp->ports);
+    atomic_flag_destroy(&dp->destroyed);
+    ovs_refcount_destroy(&dp->ref_cnt);
+    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(&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 +601,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(&dp->ref_cnt) == 1) {
+            /* Can't happen: 'dpif' still owns a reference to 'dp'. */
+            OVS_NOT_REACHED();
+        }
+    }
 
     return 0;
 }
@@ -408,14 +616,15 @@ 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);
+    fat_rwlock_rdlock(&dp->cls.rwlock);
     stats->n_flows = hmap_count(&dp->flow_table);
-    stats->n_hit = dp->n_hit;
-    stats->n_missed = dp->n_missed;
-    stats->n_lost = dp->n_lost;
-    stats->n_masks = UINT64_MAX;
+    fat_rwlock_unlock(&dp->cls.rwlock);
+
+    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);
+    stats->n_masks = UINT32_MAX;
     stats->n_mask_hit = UINT64_MAX;
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return 0;
 }
@@ -423,6 +632,7 @@ dpif_netdev_get_stats(const struct dpif *dpif, struct dpif_dp_stats *stats)
 static int
 do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
             odp_port_t port_no)
+    OVS_REQ_WRLOCK(dp->port_rwlock)
 {
     struct netdev_saved_flags *sf;
     struct dp_netdev_port *port;
@@ -430,7 +640,6 @@ do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
     struct netdev_rx *rx;
     enum netdev_flags flags;
     const char *open_type;
-    int mtu;
     int error;
 
     /* XXX reject devices already in some dp_netdev. */
@@ -473,13 +682,7 @@ do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
     port->rx = rx;
     port->type = xstrdup(type);
 
-    error = netdev_get_mtu(netdev, &mtu);
-    if (!error && mtu > dp->max_mtu) {
-        dp->max_mtu = mtu;
-    }
-
-    list_push_back(&dp->port_list, &port->node);
-    dp->ports[odp_to_u32(port_no)] = port;
+    hmap_insert(&dp->ports, &port->node, hash_int(odp_to_u32(port_no), 0));
     seq_change(dp->port_seq);
 
     return 0;
@@ -495,18 +698,11 @@ dpif_netdev_port_add(struct dpif *dpif, struct netdev *netdev,
     odp_port_t port_no;
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    ovs_rwlock_wrlock(&dp->port_rwlock);
     dpif_port = netdev_vport_get_dpif_port(netdev, namebuf, sizeof namebuf);
     if (*port_nop != ODPP_NONE) {
-        uint32_t port_idx = odp_to_u32(*port_nop);
-        if (port_idx >= MAX_PORTS) {
-            error = EFBIG;
-        } else if (dp->ports[port_idx]) {
-            error = EBUSY;
-        } else {
-            error = 0;
-            port_no = *port_nop;
-        }
+        port_no = *port_nop;
+        error = dp_netdev_lookup_port(dp, *port_nop) ? EBUSY : 0;
     } else {
         port_no = choose_port(dp, dpif_port);
         error = port_no == ODPP_NONE ? EFBIG : 0;
@@ -515,7 +711,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_rwlock_unlock(&dp->port_rwlock);
 
     return error;
 }
@@ -526,9 +722,9 @@ 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);
+    ovs_rwlock_wrlock(&dp->port_rwlock);
     error = port_no == ODPP_LOCAL ? EINVAL : do_del_port(dp, port_no);
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_rwlock_unlock(&dp->port_rwlock);
 
     return error;
 }
@@ -536,18 +732,34 @@ dpif_netdev_port_del(struct dpif *dpif, odp_port_t port_no)
 static bool
 is_valid_port_number(odp_port_t port_no)
 {
-    return odp_to_u32(port_no) < MAX_PORTS;
+    return port_no != ODPP_NONE;
+}
+
+static struct dp_netdev_port *
+dp_netdev_lookup_port(const struct dp_netdev *dp, odp_port_t port_no)
+    OVS_REQ_RDLOCK(dp->port_rwlock)
+{
+    struct dp_netdev_port *port;
+
+    HMAP_FOR_EACH_IN_BUCKET (port, node, hash_int(odp_to_u32(port_no), 0),
+                             &dp->ports) {
+        if (port->port_no == port_no) {
+            return port;
+        }
+    }
+    return NULL;
 }
 
 static int
 get_port_by_number(struct dp_netdev *dp,
                    odp_port_t port_no, struct dp_netdev_port **portp)
+    OVS_REQ_RDLOCK(dp->port_rwlock)
 {
     if (!is_valid_port_number(port_no)) {
         *portp = NULL;
         return EINVAL;
     } else {
-        *portp = dp->ports[odp_to_u32(port_no)];
+        *portp = dp_netdev_lookup_port(dp, port_no);
         return *portp ? 0 : ENOENT;
     }
 }
@@ -555,10 +767,11 @@ get_port_by_number(struct dp_netdev *dp,
 static int
 get_port_by_name(struct dp_netdev *dp,
                  const char *devname, struct dp_netdev_port **portp)
+    OVS_REQ_RDLOCK(dp->port_rwlock)
 {
     struct dp_netdev_port *port;
 
-    LIST_FOR_EACH (port, node, &dp->port_list) {
+    HMAP_FOR_EACH (port, node, &dp->ports) {
         if (!strcmp(netdev_get_name(port->netdev), devname)) {
             *portp = port;
             return 0;
@@ -569,6 +782,7 @@ get_port_by_name(struct dp_netdev *dp,
 
 static int
 do_del_port(struct dp_netdev *dp, odp_port_t port_no)
+    OVS_REQ_WRLOCK(dp->port_rwlock)
 {
     struct dp_netdev_port *port;
     int error;
@@ -578,8 +792,7 @@ do_del_port(struct dp_netdev *dp, odp_port_t port_no)
         return error;
     }
 
-    list_remove(&port->node);
-    dp->ports[odp_to_u32(port_no)] = NULL;
+    hmap_remove(&dp->ports, &port->node);
     seq_change(dp->port_seq);
 
     netdev_close(port->netdev);
@@ -608,12 +821,12 @@ 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);
+    ovs_rwlock_rdlock(&dp->port_rwlock);
     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);
+    ovs_rwlock_unlock(&dp->port_rwlock);
 
     return error;
 }
@@ -626,33 +839,50 @@ 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_rwlock_rdlock(&dp->port_rwlock);
     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_rwlock_unlock(&dp->port_rwlock);
 
     return error;
 }
 
-static uint32_t
-dpif_netdev_get_max_ports(const struct dpif *dpif OVS_UNUSED)
+static void
+dp_netdev_remove_flow(struct dp_netdev *dp, struct dp_netdev_flow *flow)
+    OVS_REQ_WRLOCK(dp->cls.rwlock)
+    OVS_REQUIRES(dp->flow_mutex)
 {
-    return MAX_PORTS;
+    struct cls_rule *cr = CONST_CAST(struct cls_rule *, &flow->cr);
+    struct hmap_node *node = CONST_CAST(struct hmap_node *, &flow->node);
+
+    classifier_remove(&dp->cls, cr);
+    hmap_remove(&dp->flow_table, node);
+    dp_netdev_flow_unref(flow);
 }
 
-static void
-dp_netdev_free_flow(struct dp_netdev *dp, struct dp_netdev_flow *netdev_flow)
+static struct dp_netdev_flow *
+dp_netdev_flow_ref(const struct dp_netdev_flow *flow_)
 {
-    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 dp_netdev_flow *flow = CONST_CAST(struct dp_netdev_flow *, flow_);
+    if (flow) {
+        ovs_refcount_ref(&flow->ref_cnt);
+    }
+    return flow;
+}
 
-    hmap_remove(&dp->flow_table, &netdev_flow->node);
-    free(netdev_flow->actions);
-    free(netdev_flow);
+static void
+dp_netdev_flow_unref(struct dp_netdev_flow *flow)
+{
+    if (flow && ovs_refcount_unref(&flow->ref_cnt) == 1) {
+        cls_rule_destroy(CONST_CAST(struct cls_rule *, &flow->cr));
+        ovs_mutex_lock(&flow->mutex);
+        dp_netdev_actions_unref(flow->actions);
+        ovs_mutex_unlock(&flow->mutex);
+        ovs_mutex_destroy(&flow->mutex);
+        free(flow);
+    }
 }
 
 static void
@@ -660,9 +890,13 @@ dp_netdev_flow_flush(struct dp_netdev *dp)
 {
     struct dp_netdev_flow *netdev_flow, *next;
 
+    ovs_mutex_lock(&dp->flow_mutex);
+    fat_rwlock_wrlock(&dp->cls.rwlock);
     HMAP_FOR_EACH_SAFE (netdev_flow, next, node, &dp->flow_table) {
-        dp_netdev_free_flow(dp, netdev_flow);
+        dp_netdev_remove_flow(dp, netdev_flow);
     }
+    fat_rwlock_unlock(&dp->cls.rwlock);
+    ovs_mutex_unlock(&dp->flow_mutex);
 }
 
 static int
@@ -670,15 +904,13 @@ 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 {
-    odp_port_t port_no;
+    uint32_t bucket;
+    uint32_t offset;
     char *name;
 };
 
@@ -695,27 +927,29 @@ 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);
-    uint32_t port_idx;
+    struct hmap_node *node;
+    int retval;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    for (port_idx = odp_to_u32(state->port_no);
-         port_idx < MAX_PORTS; port_idx++) {
-        struct dp_netdev_port *port = dp->ports[port_idx];
-        if (port) {
-            free(state->name);
-            state->name = xstrdup(netdev_get_name(port->netdev));
-            dpif_port->name = state->name;
-            dpif_port->type = port->type;
-            dpif_port->port_no = port->port_no;
-            state->port_no = u32_to_odp(port_idx + 1);
-            ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_rwlock_rdlock(&dp->port_rwlock);
+    node = hmap_at_position(&dp->ports, &state->bucket, &state->offset);
+    if (node) {
+        struct dp_netdev_port *port;
 
-            return 0;
-        }
+        port = CONTAINER_OF(node, struct dp_netdev_port, node);
+
+        free(state->name);
+        state->name = xstrdup(netdev_get_name(port->netdev));
+        dpif_port->name = state->name;
+        dpif_port->type = port->type;
+        dpif_port->port_no = port->port_no;
+
+        retval = 0;
+    } else {
+        retval = EOF;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_rwlock_unlock(&dp->port_rwlock);
 
-    return EOF;
+    return retval;
 }
 
 static int
@@ -734,7 +968,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;
@@ -742,7 +975,6 @@ dpif_netdev_port_poll(const struct dpif *dpif_, char **devnamep OVS_UNUSED)
     } else {
         error = EAGAIN;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
 
     return error;
 }
@@ -752,42 +984,49 @@ 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_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 flow *flow)
+    OVS_EXCLUDED(dp->cls.rwlock)
 {
-    struct cls_rule *cr;
+    struct dp_netdev_flow *netdev_flow;
 
-    ovs_rwlock_wrlock(&dp->cls.rwlock);
-    cr = classifier_lookup(&dp->cls, flow, NULL);
-    ovs_rwlock_unlock(&dp->cls.rwlock);
+    fat_rwlock_rdlock(&dp->cls.rwlock);
+    netdev_flow = dp_netdev_flow_cast(classifier_lookup(&dp->cls, flow, NULL));
+    dp_netdev_flow_ref(netdev_flow);
+    fat_rwlock_unlock(&dp->cls.rwlock);
 
-    return (cr
-            ? CONTAINER_OF(cr, struct dp_netdev_flow, cr)
-            : NULL);
+    return netdev_flow;
 }
 
 static struct dp_netdev_flow *
 dp_netdev_find_flow(const struct dp_netdev *dp, const struct flow *flow)
+    OVS_REQ_RDLOCK(dp->cls.rwlock)
 {
     struct dp_netdev_flow *netdev_flow;
 
     HMAP_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 dp_netdev_flow_ref(netdev_flow);
         }
     }
+
     return NULL;
 }
 
 static void
 get_dpif_flow_stats(struct dp_netdev_flow *netdev_flow,
                     struct dpif_flow_stats *stats)
+    OVS_REQ_RDLOCK(netdev_flow->mutex)
 {
     stats->n_packets = netdev_flow->packet_count;
     stats->n_bytes = netdev_flow->byte_count;
@@ -796,29 +1035,72 @@ get_dpif_flow_stats(struct dp_netdev_flow *netdev_flow,
 }
 
 static int
-dpif_netdev_flow_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
-                                   const struct nlattr *mask_key,
-                                   uint32_t mask_key_len, struct flow *flow,
-                                   struct flow *mask)
+dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
+                              const struct nlattr *mask_key,
+                              uint32_t mask_key_len, const struct flow *flow,
+                              struct flow *mask)
+{
+    if (mask_key_len) {
+        if (odp_flow_key_to_mask(mask_key, mask_key_len, mask, flow)) {
+            /* 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
+             * as an error, with enough details to enable debugging. */
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
+
+            if (!VLOG_DROP_ERR(&rl)) {
+                struct ds s;
+
+                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));
+                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
+         * prerequisities are not met. */
+        memset(mask, 0x0, sizeof *mask);
+
+        for (id = 0; id < MFF_N_IDS; ++id) {
+            /* Skip registers and metadata. */
+            if (!(id >= MFF_REG0 && id < MFF_REG0 + FLOW_N_REGS)
+                && id != MFF_METADATA) {
+                const struct mf_field *mf = mf_from_id(id);
+                if (mf_are_prereqs_ok(mf, flow)) {
+                    mf_mask_field(mf, mask);
+                }
+            }
+        }
+    }
+
+    return 0;
+}
+
+static int
+dpif_netdev_flow_from_nlattrs(const struct nlattr *key, uint32_t key_len,
+                              struct flow *flow)
 {
     odp_port_t in_port;
 
-    if (odp_flow_key_to_flow(key, key_len, flow)
-        || (mask_key
-            && odp_flow_key_to_mask(mask_key, mask_key_len, mask, flow))) {
+    if (odp_flow_key_to_flow(key, key_len, flow)) {
         /* This should not happen: it indicates that odp_flow_key_from_flow()
-         * and odp_flow_key_to_flow() disagree on the acceptable form of a flow
-         * or odp_flow_key_from_mask() and odp_flow_key_to_mask() disagree on
-         * the acceptable form of a mask.  Log the problem as an error, with
-         * enough details to enable debugging. */
+         * and odp_flow_key_to_flow() disagree on the acceptable form of a
+         * flow.  Log the problem as an error, with enough details to enable
+         * debugging. */
         static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
 
         if (!VLOG_DROP_ERR(&rl)) {
             struct ds s;
 
             ds_init(&s);
-            odp_flow_format(key, key_len, mask_key, mask_key_len, NULL, &s,
-                            true);
+            odp_flow_format(key, key_len, NULL, 0, NULL, &s, true);
             VLOG_ERR("internal error parsing flow key %s", ds_cstr(&s));
             ds_destroy(&s);
         }
@@ -826,11 +1108,6 @@ dpif_netdev_flow_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
         return EINVAL;
     }
 
-    if (mask_key) {
-        /* Force unwildcard the in_port. */
-        mask->in_port.odp_port = u32_to_odp(UINT32_MAX);
-    }
-
     in_port = flow->in_port.odp_port;
     if (!is_valid_port_number(in_port) && in_port != ODPP_NONE) {
         return EINVAL;
@@ -839,14 +1116,6 @@ dpif_netdev_flow_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
     return 0;
 }
 
-static int
-dpif_netdev_flow_from_nlattrs(const struct nlattr *key, uint32_t key_len,
-                              struct flow *flow)
-{
-    return dpif_netdev_flow_mask_from_nlattrs(key, key_len, NULL, 0, flow,
-                                              NULL);
-}
-
 static int
 dpif_netdev_flow_get(const struct dpif *dpif,
                      const struct nlattr *nl_key, size_t nl_key_len,
@@ -862,70 +1131,73 @@ dpif_netdev_flow_get(const struct dpif *dpif,
         return error;
     }
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    fat_rwlock_rdlock(&dp->cls.rwlock);
     netdev_flow = dp_netdev_find_flow(dp, &key);
+    fat_rwlock_unlock(&dp->cls.rwlock);
+
     if (netdev_flow) {
+        struct dp_netdev_actions *actions = NULL;
+
+        ovs_mutex_lock(&netdev_flow->mutex);
         if (stats) {
             get_dpif_flow_stats(netdev_flow, stats);
         }
         if (actionsp) {
-            *actionsp = ofpbuf_clone_data(netdev_flow->actions,
-                                          netdev_flow->actions_len);
+            actions = dp_netdev_actions_ref(netdev_flow->actions);
+        }
+        ovs_mutex_unlock(&netdev_flow->mutex);
+
+        dp_netdev_flow_unref(netdev_flow);
+
+        if (actionsp) {
+            *actionsp = ofpbuf_clone_data(actions->actions, actions->size);
+            dp_netdev_actions_unref(actions);
         }
     } 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;
+    ovs_refcount_init(&netdev_flow->ref_cnt);
 
-    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);
+    ovs_mutex_init(&netdev_flow->mutex);
+    ovs_mutex_lock(&netdev_flow->mutex);
 
-    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);
+    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);
+    fat_rwlock_wrlock(&dp->cls.rwlock);
+    classifier_insert(&dp->cls,
+                      CONST_CAST(struct cls_rule *, &netdev_flow->cr));
+    hmap_insert(&dp->flow_table,
+                CONST_CAST(struct hmap_node *, &netdev_flow->node),
+                flow_hash(flow, 0));
+    fat_rwlock_unlock(&dp->cls.rwlock);
+
+    ovs_mutex_unlock(&netdev_flow->mutex);
 
-    hmap_insert(&dp->flow_table, &netdev_flow->node, flow_hash(flow, 0));
     return 0;
 }
 
 static void
 clear_stats(struct dp_netdev_flow *netdev_flow)
+    OVS_REQUIRES(netdev_flow->mutex)
 {
     netdev_flow->used = 0;
     netdev_flow->packet_count = 0;
@@ -942,13 +1214,18 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put)
     struct flow_wildcards wc;
     int error;
 
-    error = dpif_netdev_flow_mask_from_nlattrs(put->key, put->key_len,
-                put->mask, put->mask_len, &flow, &wc.masks);
+    error = dpif_netdev_flow_from_nlattrs(put->key, put->key_len, &flow);
+    if (error) {
+        return error;
+    }
+    error = dpif_netdev_mask_from_nlattrs(put->key, put->key_len,
+                                          put->mask, put->mask_len,
+                                          &flow, &wc.masks);
     if (error) {
         return error;
     }
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    ovs_mutex_lock(&dp->flow_mutex);
     netdev_flow = dp_netdev_lookup_flow(dp, &flow);
     if (!netdev_flow) {
         if (put->flags & DPIF_FP_CREATE) {
@@ -967,24 +1244,33 @@ 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);
+
+            ovs_mutex_lock(&netdev_flow->mutex);
+            old_actions = netdev_flow->actions;
+            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);
             }
+            ovs_mutex_unlock(&netdev_flow->mutex);
+
+            dp_netdev_actions_unref(old_actions);
         } else if (put->flags & DPIF_FP_CREATE) {
             error = EEXIST;
         } else {
             /* Overlapping flow. */
             error = EINVAL;
         }
+        dp_netdev_flow_unref(netdev_flow);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->flow_mutex);
 
     return error;
 }
@@ -1002,17 +1288,21 @@ 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);
+    fat_rwlock_wrlock(&dp->cls.rwlock);
     netdev_flow = dp_netdev_find_flow(dp, &key);
     if (netdev_flow) {
         if (del->stats) {
+            ovs_mutex_lock(&netdev_flow->mutex);
             get_dpif_flow_stats(netdev_flow, del->stats);
+            ovs_mutex_unlock(&netdev_flow->mutex);
         }
-        dp_netdev_free_flow(dp, netdev_flow);
+        dp_netdev_remove_flow(dp, netdev_flow);
     } else {
         error = ENOENT;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    fat_rwlock_unlock(&dp->cls.rwlock);
+    ovs_mutex_unlock(&dp->flow_mutex);
 
     return error;
 }
@@ -1020,7 +1310,7 @@ dpif_netdev_flow_del(struct dpif *dpif, const struct dpif_flow_del *del)
 struct dp_netdev_flow_state {
     uint32_t bucket;
     uint32_t offset;
-    struct nlattr *actions;
+    struct dp_netdev_actions *actions;
     struct odputil_keybuf keybuf;
     struct odputil_keybuf maskbuf;
     struct dpif_flow_stats stats;
@@ -1050,15 +1340,17 @@ dpif_netdev_flow_dump_next(const struct dpif *dpif, void *state_,
     struct dp_netdev_flow *netdev_flow;
     struct hmap_node *node;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    fat_rwlock_rdlock(&dp->cls.rwlock);
     node = hmap_at_position(&dp->flow_table, &state->bucket, &state->offset);
+    if (node) {
+        netdev_flow = CONTAINER_OF(node, struct dp_netdev_flow, node);
+        dp_netdev_flow_ref(netdev_flow);
+    }
+    fat_rwlock_unlock(&dp->cls.rwlock);
     if (!node) {
-        ovs_mutex_unlock(&dp_netdev_mutex);
         return EOF;
     }
 
-    netdev_flow = CONTAINER_OF(node, struct dp_netdev_flow, node);
-
     if (key) {
         struct ofpbuf buf;
 
@@ -1083,21 +1375,25 @@ dpif_netdev_flow_dump_next(const struct dpif *dpif, void *state_,
         *mask_len = buf.size;
     }
 
-    if (actions) {
-        free(state->actions);
-        state->actions = xmemdup(netdev_flow->actions,
-                         netdev_flow->actions_len);
+    if (actions || stats) {
+        dp_netdev_actions_unref(state->actions);
+        state->actions = NULL;
 
-        *actions = state->actions;
-        *actions_len = netdev_flow->actions_len;
+        ovs_mutex_lock(&netdev_flow->mutex);
+        if (actions) {
+            state->actions = dp_netdev_actions_ref(netdev_flow->actions);
+            *actions = state->actions->actions;
+            *actions_len = state->actions->size;
+        }
+        if (stats) {
+            get_dpif_flow_stats(netdev_flow, &state->stats);
+            *stats = &state->stats;
+        }
+        ovs_mutex_unlock(&netdev_flow->mutex);
     }
 
-    if (stats) {
-        get_dpif_flow_stats(netdev_flow, &state->stats);
-        *stats = &state->stats;
-    }
+    dp_netdev_flow_unref(netdev_flow);
 
-    ovs_mutex_unlock(&dp_netdev_mutex);
     return 0;
 }
 
@@ -1106,42 +1402,33 @@ dpif_netdev_flow_dump_done(const struct dpif *dpif OVS_UNUSED, void *state_)
 {
     struct dp_netdev_flow_state *state = state_;
 
-    free(state->actions);
+    dp_netdev_actions_unref(state->actions);
     free(state);
     return 0;
 }
 
 static int
-dpif_netdev_execute(struct dpif *dpif, const struct dpif_execute *execute)
+dpif_netdev_execute(struct dpif *dpif, struct dpif_execute *execute)
 {
     struct dp_netdev *dp = get_dp_netdev(dpif);
-    struct flow md;
-    int error;
+    struct pkt_metadata *md = &execute->md;
+    struct flow key;
 
     if (execute->packet->size < ETH_HEADER_LEN ||
         execute->packet->size > UINT16_MAX) {
         return EINVAL;
     }
 
-    /* Get packet metadata. */
-    error = dpif_netdev_flow_from_nlattrs(execute->key, execute->key_len, &md);
-    if (!error) {
-        struct ofpbuf *copy;
-        struct flow key;
+    /* Extract flow key. */
+    flow_extract(execute->packet, md->skb_priority, md->pkt_mark, &md->tunnel,
+                 (union flow_in_port *)&md->in_port, &key);
 
-        /* Make a deep copy of 'packet', because we might modify its data. */
-        copy = ofpbuf_clone_with_headroom(execute->packet, DP_NETDEV_HEADROOM);
+    ovs_rwlock_rdlock(&dp->port_rwlock);
+    dp_netdev_execute_actions(dp, &key, execute->packet, md, execute->actions,
+                              execute->actions_len);
+    ovs_rwlock_unlock(&dp->port_rwlock);
 
-        /* Extract flow key. */
-        flow_extract(copy, md.skb_priority, md.pkt_mark, &md.tunnel,
-                     &md.in_port, &key);
-        ovs_mutex_lock(&dp_netdev_mutex);
-        dp_netdev_execute_actions(dp, &key, copy,
-                                  execute->actions, execute->actions_len);
-        ovs_mutex_unlock(&dp_netdev_mutex);
-        ofpbuf_delete(copy);
-    }
-    return error;
+    return 0;
 }
 
 static int
@@ -1159,9 +1446,9 @@ dpif_netdev_queue_to_priority(const struct dpif *dpif OVS_UNUSED,
 }
 
 static struct dp_netdev_queue *
-find_nonempty_queue(struct dpif *dpif)
+find_nonempty_queue(struct dp_netdev *dp)
+    OVS_REQUIRES(dp->queue_mutex)
 {
-    struct dp_netdev *dp = get_dp_netdev(dpif);
     int i;
 
     for (i = 0; i < N_QUEUES; i++) {
@@ -1177,16 +1464,16 @@ static int
 dpif_netdev_recv(struct dpif *dpif, struct dpif_upcall *upcall,
                  struct ofpbuf *buf)
 {
+    struct dp_netdev *dp = get_dp_netdev(dpif);
     struct dp_netdev_queue *q;
     int error;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    q = find_nonempty_queue(dpif);
+    ovs_mutex_lock(&dp->queue_mutex);
+    q = find_nonempty_queue(dp);
     if (q) {
         struct dp_netdev_upcall *u = &q->upcalls[q->tail++ & QUEUE_MASK];
 
         *upcall = u->upcall;
-        upcall->packet = buf;
 
         ofpbuf_uninit(buf);
         *buf = u->buf;
@@ -1195,7 +1482,7 @@ dpif_netdev_recv(struct dpif *dpif, struct dpif_upcall *upcall,
     } else {
         error = EAGAIN;
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->queue_mutex);
 
     return error;
 }
@@ -1206,128 +1493,237 @@ dpif_netdev_recv_wait(struct dpif *dpif)
     struct dp_netdev *dp = get_dp_netdev(dpif);
     uint64_t seq;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
+    ovs_mutex_lock(&dp->queue_mutex);
     seq = seq_read(dp->queue_seq);
-    if (find_nonempty_queue(dpif)) {
+    if (find_nonempty_queue(dp)) {
         poll_immediate_wake();
     } else {
         seq_wait(dp->queue_seq, seq);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
+    ovs_mutex_unlock(&dp->queue_mutex);
 }
 
 static void
 dpif_netdev_recv_purge(struct dpif *dpif)
 {
     struct dpif_netdev *dpif_netdev = dpif_netdev_cast(dpif);
-    ovs_mutex_lock(&dp_netdev_mutex);
+
     dp_netdev_purge_queues(dpif_netdev->dp);
-    ovs_mutex_unlock(&dp_netdev_mutex);
 }
 \f
-static void
-dp_netdev_flow_used(struct dp_netdev_flow *netdev_flow,
-                    const struct ofpbuf *packet)
+/* 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)
 {
-    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);
+    struct dp_netdev_actions *netdev_actions;
+
+    netdev_actions = xmalloc(sizeof *netdev_actions);
+    ovs_refcount_init(&netdev_actions->ref_cnt);
+    netdev_actions->actions = xmemdup(actions, size);
+    netdev_actions->size = size;
+
+    return netdev_actions;
 }
 
-static void
-dp_netdev_port_input(struct dp_netdev *dp, struct dp_netdev_port *port,
-                     struct ofpbuf *packet, uint32_t skb_priority,
-                     uint32_t pkt_mark, const struct flow_tnl *tnl)
+/* Increments 'actions''s refcount. */
+struct dp_netdev_actions *
+dp_netdev_actions_ref(const struct dp_netdev_actions *actions_)
 {
-    struct dp_netdev_flow *netdev_flow;
-    struct flow key;
-    union flow_in_port in_port_;
+    struct dp_netdev_actions *actions;
 
-    if (packet->size < ETH_HEADER_LEN) {
-        return;
-    }
-    in_port_.odp_port = port->port_no;
-    flow_extract(packet, skb_priority, pkt_mark, tnl, &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,
-                                  netdev_flow->actions,
-                                  netdev_flow->actions_len);
-        dp->n_hit++;
-    } else {
-        dp->n_missed++;
-        dp_netdev_output_userspace(dp, packet, DPIF_UC_MISS, &key, NULL);
+    actions = CONST_CAST(struct dp_netdev_actions *, actions_);
+    if (actions) {
+        ovs_refcount_ref(&actions->ref_cnt);
     }
+    return actions;
 }
 
-static void
-dpif_netdev_run(struct dpif *dpif)
+/* Decrements 'actions''s refcount and frees 'actions' if the refcount reaches
+ * 0. */
+void
+dp_netdev_actions_unref(struct dp_netdev_actions *actions)
 {
-    struct dp_netdev_port *port;
-    struct dp_netdev *dp;
+    if (actions && ovs_refcount_unref(&actions->ref_cnt) == 1) {
+        free(actions->actions);
+        free(actions);
+    }
+}
+\f
+static void *
+dp_forwarder_main(void *f_)
+{
+    struct dp_forwarder *f = f_;
+    struct dp_netdev *dp = f->dp;
     struct ofpbuf packet;
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    dp = get_dp_netdev(dpif);
-    ofpbuf_init(&packet,
-                DP_NETDEV_HEADROOM + VLAN_ETH_HEADER_LEN + dp->max_mtu);
-
-    LIST_FOR_EACH (port, node, &dp->port_list) {
-        int error;
+    f->name = xasprintf("forwarder_%u", ovsthread_id_self());
+    set_subprogram_name("%s", f->name);
+
+    ofpbuf_init(&packet, 0);
+    while (!latch_is_set(&dp->exit_latch)) {
+        bool received_anything;
+        int i;
+
+        ovs_rwlock_rdlock(&dp->port_rwlock);
+        for (i = 0; i < 50; i++) {
+            struct dp_netdev_port *port;
+
+            received_anything = false;
+            HMAP_FOR_EACH (port, node, &f->dp->ports) {
+                if (port->rx
+                    && port->node.hash >= f->min_hash
+                    && port->node.hash <= f->max_hash) {
+                    int buf_size;
+                    int error;
+                    int mtu;
+
+                    if (netdev_get_mtu(port->netdev, &mtu)) {
+                        mtu = ETH_PAYLOAD_MAX;
+                    }
+                    buf_size = DP_NETDEV_HEADROOM + VLAN_ETH_HEADER_LEN + mtu;
+
+                    ofpbuf_clear(&packet);
+                    ofpbuf_reserve_with_tailroom(&packet, DP_NETDEV_HEADROOM,
+                                                 buf_size);
+
+                    error = netdev_rx_recv(port->rx, &packet);
+                    if (!error) {
+                        struct pkt_metadata md
+                            = PKT_METADATA_INITIALIZER(port->port_no);
+                        dp_netdev_port_input(dp, &packet, &md);
+
+                        received_anything = true;
+                    } 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));
+                    }
+                }
+            }
 
-        /* Reset packet contents. */
-        ofpbuf_clear(&packet);
-        ofpbuf_reserve(&packet, DP_NETDEV_HEADROOM);
+            if (!received_anything) {
+                break;
+            }
+        }
 
-        error = port->rx ? netdev_rx_recv(port->rx, &packet) : EOPNOTSUPP;
-        if (!error) {
-            dp_netdev_port_input(dp, port, &packet, 0, 0, NULL);
-        } else if (error != EAGAIN && error != EOPNOTSUPP) {
-            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
+        if (received_anything) {
+            poll_immediate_wake();
+        } else {
+            struct dp_netdev_port *port;
 
-            VLOG_ERR_RL(&rl, "error receiving data from %s: %s",
-                        netdev_get_name(port->netdev), ovs_strerror(error));
+            HMAP_FOR_EACH (port, node, &f->dp->ports)
+                if (port->rx
+                    && port->node.hash >= f->min_hash
+                    && port->node.hash <= f->max_hash) {
+                    netdev_rx_wait(port->rx);
+                }
+            seq_wait(dp->port_seq, seq_read(dp->port_seq));
+            latch_wait(&dp->exit_latch);
         }
+        ovs_rwlock_unlock(&dp->port_rwlock);
+
+        poll_block();
     }
     ofpbuf_uninit(&packet);
-    ovs_mutex_unlock(&dp_netdev_mutex);
+
+    free(f->name);
+
+    return NULL;
 }
 
 static void
-dpif_netdev_wait(struct dpif *dpif)
+dp_netdev_set_threads(struct dp_netdev *dp, int n)
 {
-    struct dp_netdev_port *port;
+    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. */
+    if (n == dp->n_forwarders) {
+        return;
+    }
 
-    ovs_mutex_lock(&dp_netdev_mutex);
-    LIST_FOR_EACH (port, node, &get_dp_netdev(dpif)->port_list) {
-        if (port->rx) {
-            netdev_rx_wait(port->rx);
+    /* Stop existing threads. */
+    latch_set(&dp->exit_latch);
+    for (i = 0; i < dp->n_forwarders; i++) {
+        struct dp_forwarder *f = &dp->forwarders[i];
+
+        xpthread_join(f->thread, NULL);
+    }
+    latch_poll(&dp->exit_latch);
+    free(dp->forwarders);
+
+    /* Start new threads. */
+    dp->forwarders = xmalloc(n * sizeof *dp->forwarders);
+    dp->n_forwarders = n;
+    for (i = 0; i < n; i++) {
+        struct dp_forwarder *f = &dp->forwarders[i];
+
+        f->dp = dp;
+        f->min_hash = UINT32_MAX / n * i;
+        f->max_hash = UINT32_MAX / n * (i + 1) - 1;
+        if (i == n - 1) {
+            f->max_hash = UINT32_MAX;
         }
+        xpthread_create(&f->thread, NULL, dp_forwarder_main, f);
+    }
+}
+\f
+static void
+dp_netdev_flow_used(struct dp_netdev_flow *netdev_flow,
+                    const struct ofpbuf *packet)
+    OVS_REQUIRES(netdev_flow->mutex)
+{
+    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);
+}
+
+static void
+dp_netdev_port_input(struct dp_netdev *dp, struct ofpbuf *packet,
+                     struct pkt_metadata *md)
+    OVS_REQ_RDLOCK(dp->port_rwlock)
+{
+    struct dp_netdev_flow *netdev_flow;
+    struct flow key;
+
+    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) {
+        struct dp_netdev_actions *actions;
+
+        ovs_mutex_lock(&netdev_flow->mutex);
+        dp_netdev_flow_used(netdev_flow, packet);
+        actions = dp_netdev_actions_ref(netdev_flow->actions);
+        ovs_mutex_unlock(&netdev_flow->mutex);
+
+        dp_netdev_execute_actions(dp, &key, packet, md,
+                                  actions->actions, actions->size);
+        dp_netdev_actions_unref(actions);
+        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);
     }
-    ovs_mutex_unlock(&dp_netdev_mutex);
 }
 
 static int
-dp_netdev_output_userspace(struct dp_netdev *dp, const struct ofpbuf *packet,
+dp_netdev_output_userspace(struct dp_netdev *dp, struct ofpbuf *packet,
                            int queue_no, const struct flow *flow,
                            const struct nlattr *userdata)
+    OVS_EXCLUDED(dp->queue_mutex)
 {
     struct dp_netdev_queue *q = &dp->queues[queue_no];
+    int error;
+
+    ovs_mutex_lock(&dp->queue_mutex);
     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;
@@ -1337,7 +1733,7 @@ dp_netdev_output_userspace(struct dp_netdev *dp, const struct ofpbuf *packet,
         upcall->type = queue_no;
 
         /* Allocate buffer big enough for everything. */
-        buf_size = ODPUTIL_FLOW_KEY_BYTES + 2 + packet->size;
+        buf_size = ODPUTIL_FLOW_KEY_BYTES;
         if (userdata) {
             buf_size += NLA_ALIGN(userdata->nla_len);
         }
@@ -1354,23 +1750,21 @@ dp_netdev_output_userspace(struct dp_netdev *dp, const struct ofpbuf *packet,
                                           NLA_ALIGN(userdata->nla_len));
         }
 
-        /* Put packet.
-         *
-         * We adjust 'data' and 'size' in 'buf' so that only the packet itself
-         * is visible in 'upcall->packet'.  The ODP flow and (if present)
-         * userdata become part of the headroom. */
-        ofpbuf_put_zeros(buf, 2);
-        buf->data = ofpbuf_put(buf, packet->data, packet->size);
-        buf->size = packet->size;
-        upcall->packet = buf;
+        /* Steal packet data. */
+        ovs_assert(packet->source == OFPBUF_MALLOC);
+        upcall->packet = *packet;
+        ofpbuf_use(packet, NULL, 0);
 
         seq_change(dp->queue_seq);
 
-        return 0;
+        error = 0;
     } else {
-        dp->n_lost++;
-        return ENOBUFS;
+        ovsthread_counter_inc(dp->n_lost, 1);
+        error = ENOBUFS;
     }
+    ovs_mutex_unlock(&dp->queue_mutex);
+
+    return error;
 }
 
 struct dp_netdev_execute_aux {
@@ -1379,40 +1773,60 @@ struct dp_netdev_execute_aux {
 };
 
 static void
-dp_netdev_action_output(void *aux_, struct ofpbuf *packet,
-                        const struct flow *flow OVS_UNUSED,
-                        odp_port_t out_port)
+dp_execute_cb(void *aux_, struct ofpbuf *packet,
+              const struct pkt_metadata *md OVS_UNUSED,
+              const struct nlattr *a, bool may_steal)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
 {
     struct dp_netdev_execute_aux *aux = aux_;
-    struct dp_netdev_port *p = aux->dp->ports[odp_to_u32(out_port)];
-    if (p) {
-        netdev_send(p->netdev, packet);
-    }
-}
+    int type = nl_attr_type(a);
+    struct dp_netdev_port *p;
+
+    switch ((enum ovs_action_attr)type) {
+    case OVS_ACTION_ATTR_OUTPUT:
+        p = dp_netdev_lookup_port(aux->dp, u32_to_odp(nl_attr_get_u32(a)));
+        if (p) {
+            netdev_send(p->netdev, packet);
+        }
+        break;
 
-static void
-dp_netdev_action_userspace(void *aux_, struct ofpbuf *packet,
-                           const struct flow *flow OVS_UNUSED,
-                           const struct nlattr *a)
-{
-    struct dp_netdev_execute_aux *aux = aux_;
-    const struct nlattr *userdata;
+    case OVS_ACTION_ATTR_USERSPACE: {
+        const struct nlattr *userdata;
+
+        userdata = nl_attr_find_nested(a, OVS_USERSPACE_ATTR_USERDATA);
 
-    userdata = nl_attr_find_nested(a, OVS_USERSPACE_ATTR_USERDATA);
-    dp_netdev_output_userspace(aux->dp, packet, DPIF_UC_ACTION, aux->key,
-                               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);
+        }
+        dp_netdev_output_userspace(aux->dp, packet, DPIF_UC_ACTION, aux->key,
+                                   userdata);
+        if (!may_steal) {
+            ofpbuf_uninit(packet);
+        }
+        break;
+    }
+    case OVS_ACTION_ATTR_PUSH_VLAN:
+    case OVS_ACTION_ATTR_POP_VLAN:
+    case OVS_ACTION_ATTR_PUSH_MPLS:
+    case OVS_ACTION_ATTR_POP_MPLS:
+    case OVS_ACTION_ATTR_SET:
+    case OVS_ACTION_ATTR_SAMPLE:
+    case OVS_ACTION_ATTR_UNSPEC:
+    case __OVS_ACTION_ATTR_MAX:
+        OVS_NOT_REACHED();
+    }
 }
 
 static void
 dp_netdev_execute_actions(struct dp_netdev *dp, const struct flow *key,
-                          struct ofpbuf *packet,
+                          struct ofpbuf *packet, struct pkt_metadata *md,
                           const struct nlattr *actions, size_t actions_len)
+    OVS_REQ_RDLOCK(dp->port_rwlock)
 {
     struct dp_netdev_execute_aux aux = {dp, key};
-    struct flow md = *key;   /* Packet metadata, may be modified by actions. */
 
-    odp_execute_actions(&aux, packet, &md, actions, actions_len,
-                        dp_netdev_action_output, dp_netdev_action_userspace);
+    odp_execute_actions(&aux, packet, md, actions, actions_len, dp_execute_cb);
 }
 
 #define DPIF_NETDEV_CLASS_FUNCTIONS                    \
@@ -1421,14 +1835,13 @@ dp_netdev_execute_actions(struct dp_netdev *dp, const struct flow *key,
     dpif_netdev_open,                                  \
     dpif_netdev_close,                                 \
     dpif_netdev_destroy,                               \
-    dpif_netdev_run,                                   \
-    dpif_netdev_wait,                                  \
+    NULL,                                      \
+    NULL,                                      \
     dpif_netdev_get_stats,                             \
     dpif_netdev_port_add,                              \
     dpif_netdev_port_del,                              \
     dpif_netdev_port_query_by_number,                  \
     dpif_netdev_port_query_by_name,                    \
-    dpif_netdev_get_max_ports,                         \
     NULL,                       /* port_get_pid */     \
     dpif_netdev_port_dump_start,                       \
     dpif_netdev_port_dump_next,                                \
@@ -1466,33 +1879,42 @@ dpif_dummy_change_port_number(struct unixctl_conn *conn, int argc OVS_UNUSED,
 {
     struct dp_netdev_port *port;
     struct dp_netdev *dp;
-    int port_no;
+    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);
 
+    ovs_rwlock_wrlock(&dp->port_rwlock);
     if (get_port_by_name(dp, argv[2], &port)) {
         unixctl_command_reply_error(conn, "unknown port");
-        return;
+        goto exit;
     }
 
-    port_no = atoi(argv[3]);
-    if (port_no <= 0 || port_no >= MAX_PORTS) {
+    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->ports[port_no]) {
+    if (dp_netdev_lookup_port(dp, port_no)) {
         unixctl_command_reply_error(conn, "port number already in use");
-        return;
+        goto exit;
     }
-    dp->ports[odp_to_u32(port->port_no)] = NULL;
-    dp->ports[port_no] = port;
-    port->port_no = u32_to_odp(port_no);
+    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));
     seq_change(dp->port_seq);
     unixctl_command_reply(conn, NULL);
+
+exit:
+    ovs_rwlock_unlock(&dp->port_rwlock);
+    dp_netdev_unref(dp);
 }
 
 static void