dpif-netdev: Fix a race.
[cascardo/ovs.git] / lib / dpif-netdev.c
index bee9330..5233130 100644 (file)
@@ -63,6 +63,7 @@
 #include "shash.h"
 #include "sset.h"
 #include "timeval.h"
+#include "tnl-arp-cache.h"
 #include "unixctl.h"
 #include "util.h"
 #include "vlog.h"
@@ -128,6 +129,7 @@ struct emc_entry {
 
 struct emc_cache {
     struct emc_entry entries[EM_FLOW_HASH_ENTRIES];
+    int sweep_idx;                /* For emc_cache_slow_sweep(). */
 };
 
 /* Iterate in the exact match cache through every entry that might contain a
@@ -226,6 +228,7 @@ struct dp_netdev {
      * for pin of pmd threads. */
     size_t n_dpdk_rxqs;
     char *pmd_cmask;
+    uint64_t last_tnl_conf_seq;
 };
 
 static struct dp_netdev_port *dp_netdev_lookup_port(const struct dp_netdev *dp,
@@ -370,6 +373,10 @@ static void dp_netdev_actions_free(struct dp_netdev_actions *);
 struct dp_netdev_pmd_thread {
     struct dp_netdev *dp;
     struct cmap_node node;          /* In 'dp->poll_threads'. */
+
+    pthread_cond_t cond;            /* For synchronizing pmd thread reload. */
+    struct ovs_mutex cond_mutex;    /* Mutex for condition variable. */
+
     /* Per thread exact-match cache.  Note, the instance for cpu core
      * NON_PMD_CORE_ID can be accessed by multiple threads, and thusly
      * need to be protected (e.g. by 'dp_netdev_mutex').  All other
@@ -416,6 +423,7 @@ static void dp_netdev_input(struct dp_netdev_pmd_thread *,
                             struct dpif_packet **, int cnt);
 
 static void dp_netdev_disable_upcall(struct dp_netdev *);
+void dp_netdev_pmd_reload_done(struct dp_netdev_pmd_thread *pmd);
 static void dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd,
                                     struct dp_netdev *dp, int index,
                                     int core_id, int numa_id);
@@ -426,6 +434,7 @@ static void dp_netdev_del_pmds_on_numa(struct dp_netdev *dp, int numa_id);
 static void dp_netdev_set_pmds_on_numa(struct dp_netdev *dp, int numa_id);
 static void dp_netdev_reset_pmd_threads(struct dp_netdev *dp);
 
+static inline bool emc_entry_alive(struct emc_entry *ce);
 static void emc_clear_entry(struct emc_entry *ce);
 
 static void
@@ -435,6 +444,7 @@ emc_cache_init(struct emc_cache *flow_cache)
 
     BUILD_ASSERT(offsetof(struct miniflow, inline_values) == sizeof(uint64_t));
 
+    flow_cache->sweep_idx = 0;
     for (i = 0; i < ARRAY_SIZE(flow_cache->entries); i++) {
         flow_cache->entries[i].flow = NULL;
         flow_cache->entries[i].key.hash = 0;
@@ -455,6 +465,19 @@ emc_cache_uninit(struct emc_cache *flow_cache)
     }
 }
 
+/* Check and clear dead flow references slowly (one entry at each
+ * invocation).  */
+static void
+emc_cache_slow_sweep(struct emc_cache *flow_cache)
+{
+    struct emc_entry *entry = &flow_cache->entries[flow_cache->sweep_idx];
+
+    if (!emc_entry_alive(entry)) {
+        emc_clear_entry(entry);
+    }
+    flow_cache->sweep_idx = (flow_cache->sweep_idx + 1) & EM_FLOW_HASH_MASK;
+}
+
 static struct dpif_netdev *
 dpif_netdev_cast(const struct dpif *dpif)
 {
@@ -610,6 +633,7 @@ create_dp_netdev(const char *name, const struct dpif_class *class,
         return error;
     }
 
+    dp->last_tnl_conf_seq = seq_read(tnl_conf_seq);
     *dpp = dp;
     return 0;
 }
@@ -761,7 +785,14 @@ dp_netdev_reload_pmd__(struct dp_netdev_pmd_thread *pmd)
 {
     int old_seq;
 
+    if (pmd->core_id == NON_PMD_CORE_ID) {
+        return;
+    }
+
+    ovs_mutex_lock(&pmd->cond_mutex);
     atomic_add_relaxed(&pmd->change_seq, 1, &old_seq);
+    ovs_mutex_cond_wait(&pmd->cond, &pmd->cond_mutex);
+    ovs_mutex_unlock(&pmd->cond_mutex);
 }
 
 /* Causes all pmd threads to reload its tx/rx devices.
@@ -859,13 +890,13 @@ do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
     }
     port->sf = sf;
 
+    ovs_refcount_init(&port->ref_cnt);
+    cmap_insert(&dp->ports, &port->node, hash_port_no(port_no));
+
     if (netdev_is_pmd(netdev)) {
         dp_netdev_set_pmds_on_numa(dp, netdev_get_numa_id(netdev));
         dp_netdev_reload_pmds(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;
@@ -971,28 +1002,22 @@ port_try_ref(struct dp_netdev_port *port)
     return false;
 }
 
-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);
+        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);
     }
 }
 
@@ -1538,24 +1563,44 @@ get_dpif_flow_stats(const struct dp_netdev_flow *netdev_flow,
     }
 }
 
+/* Converts to the dpif_flow format, using 'key_buf' and 'mask_buf' for
+ * storing the netlink-formatted key/mask. 'key_buf' may be the same as
+ * 'mask_buf'. Actions will be returned without copying, by relying on RCU to
+ * protect them. */
 static void
-dp_netdev_flow_to_dpif_flow(const struct dp_netdev_flow *netdev_flow,
-                            struct ofpbuf *buffer, struct dpif_flow *flow)
+dp_netdev_flow_to_dpif_flow(const struct dpif *dpif,
+                            const struct dp_netdev_flow *netdev_flow,
+                            struct ofpbuf *key_buf, struct ofpbuf *mask_buf,
+                            struct dpif_flow *flow)
 {
     struct flow_wildcards wc;
     struct dp_netdev_actions *actions;
+    size_t offset;
 
     miniflow_expand(&netdev_flow->cr.mask->mf, &wc.masks);
-    odp_flow_key_from_mask(buffer, &wc.masks, &netdev_flow->flow,
+
+    /* Key */
+    offset = ofpbuf_size(key_buf);
+    flow->key = ofpbuf_tail(key_buf);
+    odp_flow_key_from_flow(key_buf, &netdev_flow->flow, &wc.masks,
+                           netdev_flow->flow.in_port.odp_port, true);
+    flow->key_len = ofpbuf_size(key_buf) - offset;
+
+    /* Mask */
+    offset = ofpbuf_size(mask_buf);
+    flow->mask = ofpbuf_tail(mask_buf);
+    odp_flow_key_from_mask(mask_buf, &wc.masks, &netdev_flow->flow,
                            odp_to_u32(wc.masks.in_port.odp_port),
                            SIZE_MAX, true);
-    flow->mask = ofpbuf_data(buffer);
-    flow->mask_len = ofpbuf_size(buffer);
+    flow->mask_len = ofpbuf_size(mask_buf) - offset;
 
+    /* Actions */
     actions = dp_netdev_flow_get_actions(netdev_flow);
     flow->actions = actions->actions;
     flow->actions_len = actions->size;
 
+    dpif_flow_hash(dpif, &netdev_flow->flow, sizeof netdev_flow->flow,
+                   &flow->ufid);
     get_dpif_flow_stats(netdev_flow, &flow->stats);
 }
 
@@ -1667,7 +1712,8 @@ dpif_netdev_flow_get(const struct dpif *dpif, const struct dpif_flow_get *get)
     netdev_flow = dp_netdev_find_flow(dp, &key);
 
     if (netdev_flow) {
-        dp_netdev_flow_to_dpif_flow(netdev_flow, get->buffer, get->flow);
+        dp_netdev_flow_to_dpif_flow(dpif, netdev_flow, get->buffer,
+                                    get->buffer, get->flow);
      } else {
         error = ENOENT;
     }
@@ -1944,34 +1990,11 @@ dpif_netdev_flow_dump_next(struct dpif_flow_dump_thread *thread_,
         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;
-
-        miniflow_expand(&netdev_flow->cr.mask->mf, &wc.masks);
-
-        /* 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);
+        struct ofpbuf key, mask;
 
-        /* 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);
+        ofpbuf_use_stack(&key, keybuf, sizeof *keybuf);
+        ofpbuf_use_stack(&mask, maskbuf, sizeof *maskbuf);
+        dp_netdev_flow_to_dpif_flow(&dpif->dpif, netdev_flow, &key, &mask, f);
     }
 
     return n_flows;
@@ -2006,10 +2029,12 @@ dpif_netdev_execute(struct dpif *dpif, struct dpif_execute *execute)
      * the 'non_pmd_mutex'. */
     if (pmd->core_id == NON_PMD_CORE_ID) {
         ovs_mutex_lock(&dp->non_pmd_mutex);
+        ovs_mutex_lock(&dp->port_mutex);
     }
     dp_netdev_execute_actions(pmd, &pp, 1, false, execute->actions,
                               execute->actions_len);
     if (pmd->core_id == NON_PMD_CORE_ID) {
+        ovs_mutex_unlock(&dp->port_mutex);
         ovs_mutex_unlock(&dp->non_pmd_mutex);
     }
 
@@ -2185,12 +2210,14 @@ dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd,
     }
 }
 
-static void
+/* Return true if needs to revalidate datapath flows. */
+static bool
 dpif_netdev_run(struct dpif *dpif)
 {
     struct dp_netdev_port *port;
     struct dp_netdev *dp = get_dp_netdev(dpif);
     struct dp_netdev_pmd_thread *non_pmd = dp_netdev_get_nonpmd(dp);
+    uint64_t new_tnl_seq;
 
     ovs_mutex_lock(&dp->non_pmd_mutex);
     CMAP_FOR_EACH (port, node, &dp->ports) {
@@ -2203,6 +2230,14 @@ dpif_netdev_run(struct dpif *dpif)
         }
     }
     ovs_mutex_unlock(&dp->non_pmd_mutex);
+    tnl_arp_cache_run();
+    new_tnl_seq = seq_read(tnl_conf_seq);
+
+    if (dp->last_tnl_conf_seq != new_tnl_seq) {
+        dp->last_tnl_conf_seq = new_tnl_seq;
+        return true;
+    }
+    return false;
 }
 
 static void
@@ -2222,6 +2257,7 @@ dpif_netdev_wait(struct dpif *dpif)
         }
     }
     ovs_mutex_unlock(&dp_netdev_mutex);
+    seq_wait(tnl_conf_seq, dp->last_tnl_conf_seq);
 }
 
 struct rxq_poll {
@@ -2296,6 +2332,10 @@ reload:
     emc_cache_init(&pmd->flow_cache);
     poll_cnt = pmd_load_queues(pmd, &poll_list, poll_cnt);
 
+    /* Signal here to make sure the pmd finishes
+     * reloading the updated configuration. */
+    dp_netdev_pmd_reload_done(pmd);
+
     for (;;) {
         int i;
 
@@ -2308,6 +2348,7 @@ reload:
 
             lc = 0;
 
+            emc_cache_slow_sweep(&pmd->flow_cache);
             ovsrcu_quiesce();
 
             atomic_read_relaxed(&pmd->change_seq, &seq);
@@ -2328,6 +2369,8 @@ reload:
          port_unref(poll_list[i].port);
     }
 
+    dp_netdev_pmd_reload_done(pmd);
+
     free(poll_list);
     return NULL;
 }
@@ -2362,6 +2405,14 @@ dpif_netdev_enable_upcall(struct dpif *dpif)
     dp_netdev_enable_upcall(dp);
 }
 
+void
+dp_netdev_pmd_reload_done(struct dp_netdev_pmd_thread *pmd)
+{
+    ovs_mutex_lock(&pmd->cond_mutex);
+    xpthread_cond_signal(&pmd->cond);
+    ovs_mutex_unlock(&pmd->cond_mutex);
+}
+
 /* Returns the pointer to the dp_netdev_pmd_thread for non-pmd threads. */
 static struct dp_netdev_pmd_thread *
 dp_netdev_get_nonpmd(struct dp_netdev *dp)
@@ -2398,6 +2449,8 @@ dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd, struct dp_netdev *dp,
     pmd->numa_id = numa_id;
     latch_init(&pmd->exit_latch);
     atomic_init(&pmd->change_seq, PMD_INITIAL_SEQ);
+    xpthread_cond_init(&pmd->cond, NULL);
+    ovs_mutex_init(&pmd->cond_mutex);
     /* init the 'flow_cache' since there is no
      * actual thread created for NON_PMD_CORE_ID. */
     if (core_id == NON_PMD_CORE_ID) {
@@ -2424,6 +2477,8 @@ dp_netdev_del_pmd(struct dp_netdev_pmd_thread *pmd)
     }
     cmap_remove(&pmd->dp->poll_threads, &pmd->node, hash_int(pmd->core_id, 0));
     latch_destroy(&pmd->exit_latch);
+    xpthread_cond_destroy(&pmd->cond);
+    ovs_mutex_destroy(&pmd->cond_mutex);
     free(pmd);
 }
 
@@ -2566,7 +2621,7 @@ dp_netdev_count_packet(struct dp_netdev *dp, enum dp_stat_type type, int cnt)
 
 static int
 dp_netdev_upcall(struct dp_netdev *dp, struct dpif_packet *packet_,
-                 struct flow *flow, struct flow_wildcards *wc,
+                 struct flow *flow, struct flow_wildcards *wc, ovs_u128 *ufid,
                  enum dpif_upcall_type type, const struct nlattr *userdata,
                  struct ofpbuf *actions, struct ofpbuf *put_actions)
 {
@@ -2602,7 +2657,7 @@ dp_netdev_upcall(struct dp_netdev *dp, struct dpif_packet *packet_,
         ds_destroy(&ds);
     }
 
-    return dp->upcall_cb(packet, flow, type, userdata, actions, wc,
+    return dp->upcall_cb(packet, flow, ufid, type, userdata, actions, wc,
                          put_actions, dp->upcall_aux);
 }
 
@@ -2787,6 +2842,7 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
     if (OVS_UNLIKELY(any_miss) && !fat_rwlock_tryrdlock(&dp->upcall_rwlock)) {
         uint64_t actions_stub[512 / 8], slow_stub[512 / 8];
         struct ofpbuf actions, put_actions;
+        ovs_u128 ufid;
 
         ofpbuf_use_stub(&actions, actions_stub, sizeof actions_stub);
         ofpbuf_use_stub(&put_actions, slow_stub, sizeof slow_stub);
@@ -2815,8 +2871,9 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
             ofpbuf_clear(&actions);
             ofpbuf_clear(&put_actions);
 
+            dpif_flow_hash(dp->dpif, &match.flow, sizeof match.flow, &ufid);
             error = dp_netdev_upcall(dp, packets[i], &match.flow, &match.wc,
-                                     DPIF_UC_MISS, NULL, &actions,
+                                     &ufid, DPIF_UC_MISS, NULL, &actions,
                                      &put_actions);
             if (OVS_UNLIKELY(error && error != ENOSPC)) {
                 continue;
@@ -2925,15 +2982,45 @@ dpif_netdev_register_upcall_cb(struct dpif *dpif, upcall_callback *cb,
 static void
 dp_netdev_drop_packets(struct dpif_packet ** packets, int cnt, bool may_steal)
 {
-    int i;
-
     if (may_steal) {
+        int i;
+
         for (i = 0; i < cnt; i++) {
             dpif_packet_delete(packets[i]);
         }
     }
 }
 
+static int
+push_tnl_action(const struct dp_netdev *dp,
+                   const struct nlattr *attr,
+                   struct dpif_packet **packets, int cnt)
+{
+    struct dp_netdev_port *tun_port;
+    const struct ovs_action_push_tnl *data;
+
+    data = nl_attr_get(attr);
+
+    tun_port = dp_netdev_lookup_port(dp, u32_to_odp(data->tnl_port));
+    if (!tun_port) {
+        return -EINVAL;
+    }
+    netdev_push_header(tun_port->netdev, packets, cnt, data);
+
+    return 0;
+}
+
+static void
+dp_netdev_clone_pkt_batch(struct dpif_packet **tnl_pkt,
+                          struct dpif_packet **packets, int cnt)
+{
+    int i;
+
+    for (i = 0; i < cnt; i++) {
+        tnl_pkt[i] = dpif_packet_clone(packets[i]);
+    }
+}
+
 static void
 dp_execute_cb(void *aux_, struct dpif_packet **packets, int cnt,
               const struct nlattr *a, bool may_steal)
@@ -2956,11 +3043,66 @@ dp_execute_cb(void *aux_, struct dpif_packet **packets, int cnt,
         }
         break;
 
+    case OVS_ACTION_ATTR_TUNNEL_PUSH:
+        if (*depth < MAX_RECIRC_DEPTH) {
+            struct dpif_packet *tnl_pkt[NETDEV_MAX_RX_BATCH];
+            int err;
+
+            if (!may_steal) {
+                dp_netdev_clone_pkt_batch(tnl_pkt, packets, cnt);
+                packets = tnl_pkt;
+            }
+
+            err = push_tnl_action(dp, a, packets, cnt);
+            if (!err) {
+                (*depth)++;
+                dp_netdev_input(pmd, packets, cnt);
+                (*depth)--;
+            } else {
+                dp_netdev_drop_packets(tnl_pkt, cnt, !may_steal);
+            }
+            return;
+        }
+        break;
+
+    case OVS_ACTION_ATTR_TUNNEL_POP:
+        if (*depth < MAX_RECIRC_DEPTH) {
+            odp_port_t portno = u32_to_odp(nl_attr_get_u32(a));
+
+            p = dp_netdev_lookup_port(dp, portno);
+            if (p) {
+                struct dpif_packet *tnl_pkt[NETDEV_MAX_RX_BATCH];
+                int err;
+
+                if (!may_steal) {
+                   dp_netdev_clone_pkt_batch(tnl_pkt, packets, cnt);
+                   packets = tnl_pkt;
+                }
+
+                err = netdev_pop_header(p->netdev, packets, cnt);
+                if (!err) {
+
+                    for (i = 0; i < cnt; i++) {
+                        packets[i]->md.in_port.odp_port = portno;
+                    }
+
+                    (*depth)++;
+                    dp_netdev_input(pmd, packets, cnt);
+                    (*depth)--;
+                } else {
+                    dp_netdev_drop_packets(tnl_pkt, cnt, !may_steal);
+                }
+                return;
+            }
+        }
+        break;
+
     case OVS_ACTION_ATTR_USERSPACE:
         if (!fat_rwlock_tryrdlock(&dp->upcall_rwlock)) {
             const struct nlattr *userdata;
             struct ofpbuf actions;
             struct flow flow;
+            ovs_u128 ufid;
 
             userdata = nl_attr_find_nested(a, OVS_USERSPACE_ATTR_USERDATA);
             ofpbuf_init(&actions, 0);
@@ -2971,8 +3113,9 @@ dp_execute_cb(void *aux_, struct dpif_packet **packets, int cnt,
                 ofpbuf_clear(&actions);
 
                 flow_extract(&packets[i]->ofpbuf, &packets[i]->md, &flow);
-                error = dp_netdev_upcall(dp, packets[i], &flow, NULL,
-                                         DPIF_UC_ACTION, userdata, &actions,
+                dpif_flow_hash(dp->dpif, &flow, sizeof flow, &ufid);
+                error = dp_netdev_upcall(dp, packets[i], &flow, NULL, &ufid,
+                                         DPIF_UC_ACTION, userdata,&actions,
                                          NULL);
                 if (!error || error == ENOSPC) {
                     dp_netdev_execute_actions(pmd, &packets[i], 1, may_steal,
@@ -2989,34 +3132,6 @@ dp_execute_cb(void *aux_, struct dpif_packet **packets, int cnt,
         }
         break;
 
-    case OVS_ACTION_ATTR_HASH: {
-        const struct ovs_action_hash *hash_act;
-        uint32_t hash;
-
-        hash_act = nl_attr_get(a);
-
-        for (i = 0; i < cnt; i++) {
-
-            if (hash_act->hash_alg == OVS_HASH_ALG_L4) {
-                /* Hash need not be symmetric, nor does it need to include
-                 * L2 fields. */
-                hash = hash_2words(dpif_packet_get_dp_hash(packets[i]),
-                                   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 */
-            }
-
-            dpif_packet_set_dp_hash(packets[i], hash);
-        }
-        return;
-    }
-
     case OVS_ACTION_ATTR_RECIRC:
         if (*depth < MAX_RECIRC_DEPTH) {
 
@@ -3049,6 +3164,7 @@ dp_execute_cb(void *aux_, struct dpif_packet **packets, int cnt,
     case OVS_ACTION_ATTR_SET:
     case OVS_ACTION_ATTR_SET_MASKED:
     case OVS_ACTION_ATTR_SAMPLE:
+    case OVS_ACTION_ATTR_HASH:
     case OVS_ACTION_ATTR_UNSPEC:
     case __OVS_ACTION_ATTR_MAX:
         OVS_NOT_REACHED();
@@ -3290,6 +3406,7 @@ dpcls_create_subtable(struct dpcls *cls, const struct netdev_flow_key *mask)
     netdev_flow_key_clone(&subtable->mask, mask);
     cmap_insert(&cls->subtables_map, &subtable->cmap_node, mask->hash);
     pvector_insert(&cls->subtables, subtable, 0);
+    pvector_publish(&cls->subtables);
 
     return subtable;
 }
@@ -3332,6 +3449,7 @@ dpcls_remove(struct dpcls *cls, struct dpcls_rule *rule)
     if (cmap_remove(&subtable->rules, &rule->cmap_node, rule->flow.hash)
         == 0) {
         dpcls_destroy_subtable(cls, subtable);
+        pvector_publish(&cls->subtables);
     }
 }