]> git.proxmox.com Git - mirror_ovs.git/blobdiff - lib/dpif-netdev.c
cirrus: Use FreeBSD 12.2.
[mirror_ovs.git] / lib / dpif-netdev.c
index e023ece6f52a1a0f2c18a539961968b1d4e626ad..e3fd0a07fd5b79152135bdbc800a5c815784a2a1 100644 (file)
@@ -85,9 +85,9 @@
 VLOG_DEFINE_THIS_MODULE(dpif_netdev);
 
 /* Auto Load Balancing Defaults */
-#define ALB_ACCEPTABLE_IMPROVEMENT       25
-#define ALB_PMD_LOAD_THRESHOLD           95
-#define ALB_PMD_REBALANCE_POLL_INTERVAL  1 /* 1 Min */
+#define ALB_IMPROVEMENT_THRESHOLD    25
+#define ALB_LOAD_THRESHOLD           95
+#define ALB_REBALANCE_INTERVAL       1 /* 1 Min */
 #define MIN_TO_MSEC                  60000
 
 #define FLOW_DUMP_MAX_BATCH 50
@@ -261,6 +261,7 @@ struct dp_packet_flow_map {
 static void dpcls_init(struct dpcls *);
 static void dpcls_destroy(struct dpcls *);
 static void dpcls_sort_subtable_vector(struct dpcls *);
+static uint32_t dpcls_subtable_lookup_reprobe(struct dpcls *cls);
 static void dpcls_insert(struct dpcls *, struct dpcls_rule *,
                          const struct netdev_flow_key *mask);
 static void dpcls_remove(struct dpcls *, struct dpcls_rule *);
@@ -299,6 +300,8 @@ struct pmd_auto_lb {
     bool is_enabled;            /* Current status of Auto load balancing. */
     uint64_t rebalance_intvl;
     uint64_t rebalance_poll_timer;
+    uint8_t rebalance_improve_thresh;
+    atomic_uint8_t rebalance_load_thresh;
 };
 
 /* Datapath based on the network device interface from netdev.h.
@@ -491,6 +494,12 @@ struct dp_netdev_flow_stats {
     atomic_uint16_t tcp_flags;     /* Bitwise-OR of seen tcp_flags values. */
 };
 
+/* Contained by struct dp_netdev_flow's 'last_attrs' member.  */
+struct dp_netdev_flow_attrs {
+    atomic_bool offloaded;         /* True if flow is offloaded to HW. */
+    ATOMIC(const char *) dp_layer; /* DP layer the flow is handled in. */
+};
+
 /* A flow in 'dp_netdev_pmd_thread's 'flow_table'.
  *
  *
@@ -551,6 +560,11 @@ struct dp_netdev_flow {
     /* Statistics. */
     struct dp_netdev_flow_stats stats;
 
+    /* Statistics and attributes received from the netdev offload provider. */
+    atomic_int netdev_flow_get_result;
+    struct dp_netdev_flow_stats last_stats;
+    struct dp_netdev_flow_attrs last_attrs;
+
     /* Actions. */
     OVSRCU_TYPE(struct dp_netdev_actions *) actions;
 
@@ -617,9 +631,9 @@ struct tx_port {
     struct dp_netdev_rxq *output_pkts_rxqs[NETDEV_MAX_BURST];
 };
 
-/* Contained by struct tx_bond 'slave_buckets'. */
-struct slave_entry {
-    odp_port_t slave_id;
+/* Contained by struct tx_bond 'member_buckets'. */
+struct member_entry {
+    odp_port_t member_id;
     atomic_ullong n_packets;
     atomic_ullong n_bytes;
 };
@@ -628,7 +642,7 @@ struct slave_entry {
 struct tx_bond {
     struct cmap_node node;
     uint32_t bond_id;
-    struct slave_entry slave_buckets[BOND_BUCKETS];
+    struct member_entry member_buckets[BOND_BUCKETS];
 };
 
 /* A set of properties for the current processing loop that is not directly
@@ -896,6 +910,9 @@ dpif_netdev_xps_revalidate_pmd(const struct dp_netdev_pmd_thread *pmd,
                                bool purge);
 static int dpif_netdev_xps_get_tx_qid(const struct dp_netdev_pmd_thread *pmd,
                                       struct tx_port *tx);
+static inline struct dpcls *
+dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
+                           odp_port_t in_port);
 
 static inline bool emc_entry_alive(struct emc_entry *ce);
 static void emc_clear_entry(struct emc_entry *ce);
@@ -1296,6 +1313,121 @@ sorted_poll_thread_list(struct dp_netdev *dp,
     *n = k;
 }
 
+static void
+dpif_netdev_subtable_lookup_get(struct unixctl_conn *conn, int argc OVS_UNUSED,
+                                const char *argv[] OVS_UNUSED,
+                                void *aux OVS_UNUSED)
+{
+    /* Get a list of all lookup functions. */
+    struct dpcls_subtable_lookup_info_t *lookup_funcs = NULL;
+    int32_t count = dpcls_subtable_lookup_info_get(&lookup_funcs);
+    if (count < 0) {
+        unixctl_command_reply_error(conn, "error getting lookup names");
+        return;
+    }
+
+    /* Add all lookup functions to reply string. */
+    struct ds reply = DS_EMPTY_INITIALIZER;
+    ds_put_cstr(&reply, "Available lookup functions (priority : name)\n");
+    for (int i = 0; i < count; i++) {
+        ds_put_format(&reply, "  %d : %s\n", lookup_funcs[i].prio,
+                      lookup_funcs[i].name);
+    }
+    unixctl_command_reply(conn, ds_cstr(&reply));
+    ds_destroy(&reply);
+}
+
+static void
+dpif_netdev_subtable_lookup_set(struct unixctl_conn *conn, int argc,
+                                const char *argv[], void *aux OVS_UNUSED)
+{
+    /* This function requires 2 parameters (argv[1] and argv[2]) to execute.
+     *   argv[1] is subtable name
+     *   argv[2] is priority
+     *   argv[3] is the datapath name (optional if only 1 datapath exists)
+     */
+    const char *func_name = argv[1];
+
+    errno = 0;
+    char *err_char;
+    uint32_t new_prio = strtoul(argv[2], &err_char, 10);
+    if (errno != 0 || new_prio > UINT8_MAX) {
+        unixctl_command_reply_error(conn,
+            "error converting priority, use integer in range 0-255\n");
+        return;
+    }
+
+    int32_t err = dpcls_subtable_set_prio(func_name, new_prio);
+    if (err) {
+        unixctl_command_reply_error(conn,
+            "error, subtable lookup function not found\n");
+        return;
+    }
+
+    /* argv[3] is optional datapath instance. If no datapath name is provided
+     * and only one datapath exists, the one existing datapath is reprobed.
+     */
+    ovs_mutex_lock(&dp_netdev_mutex);
+    struct dp_netdev *dp = NULL;
+
+    if (argc == 4) {
+        dp = shash_find_data(&dp_netdevs, argv[3]);
+    } else if (shash_count(&dp_netdevs) == 1) {
+        dp = shash_first(&dp_netdevs)->data;
+    }
+
+    if (!dp) {
+        ovs_mutex_unlock(&dp_netdev_mutex);
+        unixctl_command_reply_error(conn,
+                                    "please specify an existing datapath");
+        return;
+    }
+
+    /* Get PMD threads list, required to get DPCLS instances. */
+    size_t n;
+    uint32_t lookup_dpcls_changed = 0;
+    uint32_t lookup_subtable_changed = 0;
+    struct dp_netdev_pmd_thread **pmd_list;
+    sorted_poll_thread_list(dp, &pmd_list, &n);
+
+    /* take port mutex as HMAP iters over them. */
+    ovs_mutex_lock(&dp->port_mutex);
+
+    for (size_t i = 0; i < n; i++) {
+        struct dp_netdev_pmd_thread *pmd = pmd_list[i];
+        if (pmd->core_id == NON_PMD_CORE_ID) {
+            continue;
+        }
+
+        struct dp_netdev_port *port = NULL;
+        HMAP_FOR_EACH (port, node, &dp->ports) {
+            odp_port_t in_port = port->port_no;
+            struct dpcls *cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
+            if (!cls) {
+                continue;
+            }
+            uint32_t subtbl_changes = dpcls_subtable_lookup_reprobe(cls);
+            if (subtbl_changes) {
+                lookup_dpcls_changed++;
+                lookup_subtable_changed += subtbl_changes;
+            }
+        }
+    }
+
+    /* release port mutex before netdev mutex. */
+    ovs_mutex_unlock(&dp->port_mutex);
+    ovs_mutex_unlock(&dp_netdev_mutex);
+
+    struct ds reply = DS_EMPTY_INITIALIZER;
+    ds_put_format(&reply,
+        "Lookup priority change affected %d dpcls ports and %d subtables.\n",
+        lookup_dpcls_changed, lookup_subtable_changed);
+    const char *reply_str = ds_cstr(&reply);
+    unixctl_command_reply(conn, reply_str);
+    VLOG_INFO("%s", reply_str);
+    ds_destroy(&reply);
+}
+
 static void
 dpif_netdev_pmd_rebalance(struct unixctl_conn *conn, int argc,
                           const char *argv[], void *aux OVS_UNUSED)
@@ -1458,17 +1590,17 @@ dpif_netdev_bond_show(struct unixctl_conn *conn, int argc,
 
     if (cmap_count(&dp->tx_bonds) > 0) {
         struct tx_bond *dp_bond_entry;
-        uint32_t slave_id;
 
         ds_put_cstr(&reply, "Bonds:\n");
         CMAP_FOR_EACH (dp_bond_entry, node, &dp->tx_bonds) {
             ds_put_format(&reply, "  bond-id %"PRIu32":\n",
                           dp_bond_entry->bond_id);
             for (int bucket = 0; bucket < BOND_BUCKETS; bucket++) {
-                slave_id =
-                    odp_to_u32(dp_bond_entry->slave_buckets[bucket].slave_id);
-                ds_put_format(&reply, "    bucket %d - slave %"PRIu32"\n",
-                              bucket, slave_id);
+                uint32_t member_id = odp_to_u32(
+                    dp_bond_entry->member_buckets[bucket].member_id);
+                ds_put_format(&reply,
+                              "    bucket %d - member %"PRIu32"\n",
+                              bucket, member_id);
             }
         }
     }
@@ -1511,6 +1643,13 @@ dpif_netdev_init(void)
     unixctl_command_register("dpif-netdev/bond-show", "[dp]",
                              0, 1, dpif_netdev_bond_show,
                              NULL);
+    unixctl_command_register("dpif-netdev/subtable-lookup-prio-set",
+                             "[lookup_func] [prio] [dp]",
+                             2, 3, dpif_netdev_subtable_lookup_set,
+                             NULL);
+    unixctl_command_register("dpif-netdev/subtable-lookup-prio-get", "",
+                             0, 0, dpif_netdev_subtable_lookup_get,
+                             NULL);
     return 0;
 }
 
@@ -2144,6 +2283,8 @@ static void
 do_del_port(struct dp_netdev *dp, struct dp_netdev_port *port)
     OVS_REQUIRES(dp->port_mutex)
 {
+    netdev_flow_flush(port->netdev);
+    netdev_uninit_flow_api(port->netdev);
     hmap_remove(&dp->ports, &port->node);
     seq_change(dp->port_seq);
 
@@ -3151,9 +3292,56 @@ dp_netdev_pmd_find_flow(const struct dp_netdev_pmd_thread *pmd,
     return NULL;
 }
 
+static void
+dp_netdev_flow_set_last_stats_attrs(struct dp_netdev_flow *netdev_flow,
+                                    const struct dpif_flow_stats *stats,
+                                    const struct dpif_flow_attrs *attrs,
+                                    int result)
+{
+    struct dp_netdev_flow_stats *last_stats = &netdev_flow->last_stats;
+    struct dp_netdev_flow_attrs *last_attrs = &netdev_flow->last_attrs;
+
+    atomic_store_relaxed(&netdev_flow->netdev_flow_get_result, result);
+    if (result) {
+        return;
+    }
+
+    atomic_store_relaxed(&last_stats->used,         stats->used);
+    atomic_store_relaxed(&last_stats->packet_count, stats->n_packets);
+    atomic_store_relaxed(&last_stats->byte_count,   stats->n_bytes);
+    atomic_store_relaxed(&last_stats->tcp_flags,    stats->tcp_flags);
+
+    atomic_store_relaxed(&last_attrs->offloaded,    attrs->offloaded);
+    atomic_store_relaxed(&last_attrs->dp_layer,     attrs->dp_layer);
+
+}
+
+static void
+dp_netdev_flow_get_last_stats_attrs(struct dp_netdev_flow *netdev_flow,
+                                    struct dpif_flow_stats *stats,
+                                    struct dpif_flow_attrs *attrs,
+                                    int *result)
+{
+    struct dp_netdev_flow_stats *last_stats = &netdev_flow->last_stats;
+    struct dp_netdev_flow_attrs *last_attrs = &netdev_flow->last_attrs;
+
+    atomic_read_relaxed(&netdev_flow->netdev_flow_get_result, result);
+    if (*result) {
+        return;
+    }
+
+    atomic_read_relaxed(&last_stats->used,         &stats->used);
+    atomic_read_relaxed(&last_stats->packet_count, &stats->n_packets);
+    atomic_read_relaxed(&last_stats->byte_count,   &stats->n_bytes);
+    atomic_read_relaxed(&last_stats->tcp_flags,    &stats->tcp_flags);
+
+    atomic_read_relaxed(&last_attrs->offloaded,    &attrs->offloaded);
+    atomic_read_relaxed(&last_attrs->dp_layer,     &attrs->dp_layer);
+}
+
 static bool
 dpif_netdev_get_flow_offload_status(const struct dp_netdev *dp,
-                                    const struct dp_netdev_flow *netdev_flow,
+                                    struct dp_netdev_flow *netdev_flow,
                                     struct dpif_flow_stats *stats,
                                     struct dpif_flow_attrs *attrs)
 {
@@ -3176,11 +3364,31 @@ dpif_netdev_get_flow_offload_status(const struct dp_netdev *dp,
     }
     ofpbuf_use_stack(&buf, &act_buf, sizeof act_buf);
     /* Taking a global 'port_mutex' to fulfill thread safety
-     * restrictions for the netdev-offload-dpdk module. */
-    ovs_mutex_lock(&dp->port_mutex);
-    ret = netdev_flow_get(netdev, &match, &actions, &netdev_flow->mega_ufid,
-                          stats, attrs, &buf);
-    ovs_mutex_unlock(&dp->port_mutex);
+     * restrictions for the netdev-offload-dpdk module.
+     *
+     * XXX: Main thread will try to pause/stop all revalidators during datapath
+     *      reconfiguration via datapath purge callback (dp_purge_cb) while
+     *      holding 'dp->port_mutex'.  So we're not waiting for mutex here.
+     *      Otherwise, deadlock is possible, bcause revalidators might sleep
+     *      waiting for the main thread to release the lock and main thread
+     *      will wait for them to stop processing.
+     *      This workaround might make statistics less accurate. Especially
+     *      for flow deletion case, since there will be no other attempt.  */
+    if (!ovs_mutex_trylock(&dp->port_mutex)) {
+        ret = netdev_flow_get(netdev, &match, &actions,
+                              &netdev_flow->mega_ufid, stats, attrs, &buf);
+        /* Storing statistics and attributes from the last request for
+         * later use on mutex contention. */
+        dp_netdev_flow_set_last_stats_attrs(netdev_flow, stats, attrs, ret);
+        ovs_mutex_unlock(&dp->port_mutex);
+    } else {
+        dp_netdev_flow_get_last_stats_attrs(netdev_flow, stats, attrs, &ret);
+        if (!ret && !attrs->dp_layer) {
+            /* Flow was never reported as 'offloaded' so it's harmless
+             * to continue to think so. */
+            ret = EAGAIN;
+        }
+    }
     netdev_close(netdev);
     if (ret) {
         return false;
@@ -3449,6 +3657,9 @@ dp_netdev_flow_add(struct dp_netdev_pmd_thread *pmd,
     /* Do not allocate extra space. */
     flow = xmalloc(sizeof *flow - sizeof flow->cr.flow.mf + mask.len);
     memset(&flow->stats, 0, sizeof flow->stats);
+    atomic_init(&flow->netdev_flow_get_result, 0);
+    memset(&flow->last_stats, 0, sizeof flow->last_stats);
+    memset(&flow->last_attrs, 0, sizeof flow->last_attrs);
     flow->dead = false;
     flow->batch = NULL;
     flow->mark = INVALID_FLOW_MARK;
@@ -3992,11 +4203,12 @@ dpif_netdev_operate(struct dpif *dpif, struct dpif_op **ops, size_t n_ops,
 
 /* Enable or Disable PMD auto load balancing. */
 static void
-set_pmd_auto_lb(struct dp_netdev *dp)
+set_pmd_auto_lb(struct dp_netdev *dp, bool always_log)
 {
     unsigned int cnt = 0;
     struct dp_netdev_pmd_thread *pmd;
     struct pmd_auto_lb *pmd_alb = &dp->pmd_alb;
+    uint8_t rebalance_load_thresh;
 
     bool enable_alb = false;
     bool multi_rxq = false;
@@ -4023,18 +4235,24 @@ set_pmd_auto_lb(struct dp_netdev *dp)
     enable_alb = enable_alb && pmd_rxq_assign_cyc &&
                     pmd_alb->auto_lb_requested;
 
-    if (pmd_alb->is_enabled != enable_alb) {
+    if (pmd_alb->is_enabled != enable_alb || always_log) {
         pmd_alb->is_enabled = enable_alb;
         if (pmd_alb->is_enabled) {
+            atomic_read_relaxed(&pmd_alb->rebalance_load_thresh,
+                                &rebalance_load_thresh);
             VLOG_INFO("PMD auto load balance is enabled "
-                      "(with rebalance interval:%"PRIu64" msec)",
-                       pmd_alb->rebalance_intvl);
+                      "interval %"PRIu64" mins, "
+                      "pmd load threshold %"PRIu8"%%, "
+                      "improvement threshold %"PRIu8"%%",
+                       pmd_alb->rebalance_intvl / MIN_TO_MSEC,
+                       rebalance_load_thresh,
+                       pmd_alb->rebalance_improve_thresh);
+
         } else {
             pmd_alb->rebalance_poll_timer = 0;
             VLOG_INFO("PMD auto load balance is disabled");
         }
     }
-
 }
 
 /* Applies datapath configuration from the database. Some of the changes are
@@ -4052,6 +4270,9 @@ dpif_netdev_set_config(struct dpif *dpif, const struct smap *other_config)
     uint32_t insert_min, cur_min;
     uint32_t tx_flush_interval, cur_tx_flush_interval;
     uint64_t rebalance_intvl;
+    uint8_t rebalance_load, cur_rebalance_load;
+    uint8_t rebalance_improve;
+    bool log_autolb = false;
 
     tx_flush_interval = smap_get_int(other_config, "tx-flush-interval",
                                      DEFAULT_TX_FLUSH_INTERVAL);
@@ -4129,7 +4350,7 @@ dpif_netdev_set_config(struct dpif *dpif, const struct smap *other_config)
                               false);
 
     rebalance_intvl = smap_get_int(other_config, "pmd-auto-lb-rebal-interval",
-                              ALB_PMD_REBALANCE_POLL_INTERVAL);
+                                   ALB_REBALANCE_INTERVAL);
 
     /* Input is in min, convert it to msec. */
     rebalance_intvl =
@@ -4137,9 +4358,38 @@ dpif_netdev_set_config(struct dpif *dpif, const struct smap *other_config)
 
     if (pmd_alb->rebalance_intvl != rebalance_intvl) {
         pmd_alb->rebalance_intvl = rebalance_intvl;
-    }
-
-    set_pmd_auto_lb(dp);
+        VLOG_INFO("PMD auto load balance interval set to "
+                  "%"PRIu64" mins\n", rebalance_intvl / MIN_TO_MSEC);
+        log_autolb = true;
+    }
+
+    rebalance_improve = smap_get_int(other_config,
+                                     "pmd-auto-lb-improvement-threshold",
+                                     ALB_IMPROVEMENT_THRESHOLD);
+    if (rebalance_improve > 100) {
+        rebalance_improve = ALB_IMPROVEMENT_THRESHOLD;
+    }
+    if (rebalance_improve != pmd_alb->rebalance_improve_thresh) {
+        pmd_alb->rebalance_improve_thresh = rebalance_improve;
+        VLOG_INFO("PMD auto load balance improvement threshold set to "
+                  "%"PRIu8"%%", rebalance_improve);
+        log_autolb = true;
+    }
+
+    rebalance_load = smap_get_int(other_config, "pmd-auto-lb-load-threshold",
+                                  ALB_LOAD_THRESHOLD);
+    if (rebalance_load > 100) {
+        rebalance_load = ALB_LOAD_THRESHOLD;
+    }
+    atomic_read_relaxed(&pmd_alb->rebalance_load_thresh, &cur_rebalance_load);
+    if (rebalance_load != cur_rebalance_load) {
+        atomic_store_relaxed(&pmd_alb->rebalance_load_thresh,
+                             rebalance_load);
+        VLOG_INFO("PMD auto load balance load threshold set to %"PRIu8"%%",
+                  rebalance_load);
+        log_autolb = true;
+    }
+    set_pmd_auto_lb(dp, log_autolb);
     return 0;
 }
 
@@ -5223,7 +5473,7 @@ reconfigure_datapath(struct dp_netdev *dp)
     reload_affected_pmds(dp);
 
     /* Check if PMD Auto LB is to be enabled */
-    set_pmd_auto_lb(dp);
+    set_pmd_auto_lb(dp, false);
 }
 
 /* Returns true if one of the netdevs in 'dp' requires a reconfiguration */
@@ -5467,7 +5717,7 @@ pmd_rebalance_dry_run(struct dp_netdev *dp)
             improvement =
                 ((curr_variance - new_variance) * 100) / curr_variance;
         }
-        if (improvement < ALB_ACCEPTABLE_IMPROVEMENT) {
+        if (improvement < dp->pmd_alb.rebalance_improve_thresh) {
             ret = false;
         }
     }
@@ -5888,7 +6138,6 @@ dp_netdev_run_meter(struct dp_netdev *dp, struct dp_packet_batch *packets_,
     struct dp_packet *packet;
     long long int long_delta_t; /* msec */
     uint32_t delta_t; /* msec */
-    uint32_t delta_in_us; /* usec */
     const size_t cnt = dp_packet_batch_size(packets_);
     uint32_t bytes, volume;
     int exceeded_band[NETDEV_MAX_BURST];
@@ -5919,9 +6168,6 @@ dp_netdev_run_meter(struct dp_netdev *dp, struct dp_packet_batch *packets_,
            Assuming that all racing threads received packets at the same time
            to avoid overflow. */
         long_delta_t = 0;
-        delta_in_us  = 0;
-    } else {
-        delta_in_us  = (now - meter->used) % 1000;
     }
 
     /* Make sure delta_t will not be too large, so that bucket will not
@@ -5957,7 +6203,6 @@ dp_netdev_run_meter(struct dp_netdev *dp, struct dp_packet_batch *packets_,
 
         /* Update band's bucket. */
         band->bucket += delta_t * band->up.rate;
-        band->bucket += delta_in_us * band->up.rate / 1000;
         if (band->bucket > band->up.burst_size) {
             band->bucket = band->up.burst_size;
         }
@@ -6508,10 +6753,10 @@ dp_netdev_add_bond_tx_to_pmd(struct dp_netdev_pmd_thread *pmd,
         for (int i = 0; i < BOND_BUCKETS; i++) {
             uint64_t n_packets, n_bytes;
 
-            atomic_read_relaxed(&tx->slave_buckets[i].n_packets, &n_packets);
-            atomic_read_relaxed(&tx->slave_buckets[i].n_bytes, &n_bytes);
-            atomic_init(&new_tx->slave_buckets[i].n_packets, n_packets);
-            atomic_init(&new_tx->slave_buckets[i].n_bytes, n_bytes);
+            atomic_read_relaxed(&tx->member_buckets[i].n_packets, &n_packets);
+            atomic_read_relaxed(&tx->member_buckets[i].n_bytes, &n_bytes);
+            atomic_init(&new_tx->member_buckets[i].n_packets, n_packets);
+            atomic_init(&new_tx->member_buckets[i].n_bytes, n_bytes);
         }
         cmap_replace(&pmd->tx_bonds, &tx->node, &new_tx->node,
                      hash_bond_id(bond->bond_id));
@@ -7437,18 +7682,19 @@ dp_execute_lb_output_action(struct dp_netdev_pmd_thread *pmd,
 
     DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
         /*
-         * Lookup the bond-hash table using hash to get the slave.
+         * Lookup the bond-hash table using hash to get the member.
          */
         uint32_t hash = dp_packet_get_rss_hash(packet);
-        struct slave_entry *s_entry = &p_bond->slave_buckets[hash & BOND_MASK];
-        odp_port_t bond_member = s_entry->slave_id;
+        struct member_entry *s_entry
+            = &p_bond->member_buckets[hash & BOND_MASK];
+        odp_port_t bond_member = s_entry->member_id;
         uint32_t size = dp_packet_size(packet);
         struct dp_packet_batch output_pkt;
 
         dp_packet_batch_init_packet(&output_pkt, packet);
         if (OVS_LIKELY(dp_execute_output_action(pmd, &output_pkt, true,
                                                 bond_member))) {
-            /* Update slave stats. */
+            /* Update member stats. */
             non_atomic_ullong_add(&s_entry->n_packets, 1);
             non_atomic_ullong_add(&s_entry->n_bytes, size);
         }
@@ -8091,7 +8337,7 @@ dpif_netdev_ipf_dump_done(struct dpif *dpif OVS_UNUSED, void *ipf_dump_ctx)
 
 static int
 dpif_netdev_bond_add(struct dpif *dpif, uint32_t bond_id,
-                     odp_port_t *slave_map)
+                     odp_port_t *member_map)
 {
     struct tx_bond *new_tx = xzalloc(sizeof *new_tx);
     struct dp_netdev *dp = get_dp_netdev(dpif);
@@ -8100,7 +8346,7 @@ dpif_netdev_bond_add(struct dpif *dpif, uint32_t bond_id,
     /* Prepare new bond mapping. */
     new_tx->bond_id = bond_id;
     for (int bucket = 0; bucket < BOND_BUCKETS; bucket++) {
-        new_tx->slave_buckets[bucket].slave_id = slave_map[bucket];
+        new_tx->member_buckets[bucket].member_id = member_map[bucket];
     }
 
     ovs_mutex_lock(&dp->bond_mutex);
@@ -8173,7 +8419,7 @@ dpif_netdev_bond_stats_get(struct dpif *dpif, uint32_t bond_id,
         for (int i = 0; i < BOND_BUCKETS; i++) {
             uint64_t pmd_n_bytes;
 
-            atomic_read_relaxed(&pmd_bond_entry->slave_buckets[i].n_bytes,
+            atomic_read_relaxed(&pmd_bond_entry->member_buckets[i].n_bytes,
                                 &pmd_n_bytes);
             n_bytes[i] += pmd_n_bytes;
         }
@@ -8459,6 +8705,28 @@ dpcls_find_subtable(struct dpcls *cls, const struct netdev_flow_key *mask)
     return dpcls_create_subtable(cls, mask);
 }
 
+/* Checks for the best available implementation for each subtable lookup
+ * function, and assigns it as the lookup function pointer for each subtable.
+ * Returns the number of subtables that have changed lookup implementation.
+ */
+static uint32_t
+dpcls_subtable_lookup_reprobe(struct dpcls *cls)
+{
+    struct pvector *pvec = &cls->subtables;
+    uint32_t subtables_changed = 0;
+    struct dpcls_subtable *subtable = NULL;
+
+    PVECTOR_FOR_EACH (subtable, pvec) {
+        uint32_t u0_bits = subtable->mf_bits_set_unit0;
+        uint32_t u1_bits = subtable->mf_bits_set_unit1;
+        void *old_func = subtable->lookup_func;
+        subtable->lookup_func = dpcls_subtable_get_best_impl(u0_bits, u1_bits);
+        subtables_changed += (old_func != subtable->lookup_func);
+    }
+    pvector_publish(pvec);
+
+    return subtables_changed;
+}
 
 /* Periodically sort the dpcls subtable vectors according to hit counts */
 static void
@@ -8484,6 +8752,7 @@ dp_netdev_pmd_try_optimize(struct dp_netdev_pmd_thread *pmd,
 
     if (pmd->ctx.now > pmd->rxq_next_cycle_store) {
         uint64_t curr_tsc;
+        uint8_t rebalance_load_trigger;
         struct pmd_auto_lb *pmd_alb = &pmd->dp->pmd_alb;
         if (pmd_alb->is_enabled && !pmd->isolated
             && (pmd->perf_stats.counters.n[PMD_CYCLES_ITER_IDLE] >=
@@ -8500,7 +8769,9 @@ dp_netdev_pmd_try_optimize(struct dp_netdev_pmd_thread *pmd,
                 pmd_load = ((tot_proc * 100) / (tot_idle + tot_proc));
             }
 
-            if (pmd_load >= ALB_PMD_LOAD_THRESHOLD) {
+            atomic_read_relaxed(&pmd_alb->rebalance_load_thresh,
+                                &rebalance_load_trigger);
+            if (pmd_load >= rebalance_load_trigger) {
                 atomic_count_inc(&pmd->pmd_overloaded);
             } else {
                 atomic_count_set(&pmd->pmd_overloaded, 0);