2
0
mirror of https://github.com/openvswitch/ovs synced 2025-08-31 14:25:26 +00:00

dpif-netdev: Add SMC cache after EMC cache

This patch adds a signature match cache (SMC) after exact match
cache (EMC). The difference between SMC and EMC is SMC only stores
a signature of a flow thus it is much more memory efficient. With
same memory space, EMC can store 8k flows while SMC can store 1M
flows. It is generally beneficial to turn on SMC but turn off EMC
when traffic flow count is much larger than EMC size.

SMC cache will map a signature to an dp_netdev_flow index in
flow_table. Thus, we add two new APIs in cmap for lookup key by
index and lookup index by key.

For now, SMC is an experimental feature that it is turned off by
default. One can turn it on using ovsdb options.

Signed-off-by: Yipeng Wang <yipeng1.wang@intel.com>
Co-authored-by: Jan Scheurich <jan.scheurich@ericsson.com>
Signed-off-by: Jan Scheurich <jan.scheurich@ericsson.com>
Acked-by: Billy O'Mahony <billy.o.mahony@intel.com>
Signed-off-by: Ian Stokes <ian.stokes@intel.com>
This commit is contained in:
Yipeng Wang
2018-07-10 03:14:06 -07:00
committed by Ian Stokes
parent 1ac6908995
commit 60d8ccae13
8 changed files with 413 additions and 39 deletions

View File

@@ -130,7 +130,9 @@ struct netdev_flow_key {
uint64_t buf[FLOW_MAX_PACKET_U64S];
};
/* Exact match cache for frequently used flows
/* EMC cache and SMC cache compose the datapath flow cache (DFC)
*
* Exact match cache for frequently used flows
*
* The cache uses a 32-bit hash of the packet (which can be the RSS hash) to
* search its entries for a miniflow that matches exactly the miniflow of the
@@ -144,6 +146,17 @@ struct netdev_flow_key {
* value is the index of a cache entry where the miniflow could be.
*
*
* Signature match cache (SMC)
*
* This cache stores a 16-bit signature for each flow without storing keys, and
* stores the corresponding 16-bit flow_table index to the 'dp_netdev_flow'.
* Each flow thus occupies 32bit which is much more memory efficient than EMC.
* SMC uses a set-associative design that each bucket contains
* SMC_ENTRY_PER_BUCKET number of entries.
* Since 16-bit flow_table index is used, if there are more than 2^16
* dp_netdev_flow, SMC will miss them that cannot be indexed by a 16-bit value.
*
*
* Thread-safety
* =============
*
@@ -156,6 +169,14 @@ struct netdev_flow_key {
#define EM_FLOW_HASH_MASK (EM_FLOW_HASH_ENTRIES - 1)
#define EM_FLOW_HASH_SEGS 2
/* SMC uses a set-associative design. A bucket contains a set of entries that
* a flow item can occupy. For now, it uses one hash function rather than two
* as for the EMC design. */
#define SMC_ENTRY_PER_BUCKET 4
#define SMC_ENTRIES (1u << 20)
#define SMC_BUCKET_CNT (SMC_ENTRIES / SMC_ENTRY_PER_BUCKET)
#define SMC_MASK (SMC_BUCKET_CNT - 1)
/* Default EMC insert probability is 1 / DEFAULT_EM_FLOW_INSERT_INV_PROB */
#define DEFAULT_EM_FLOW_INSERT_INV_PROB 100
#define DEFAULT_EM_FLOW_INSERT_MIN (UINT32_MAX / \
@@ -171,6 +192,21 @@ struct emc_cache {
int sweep_idx; /* For emc_cache_slow_sweep(). */
};
struct smc_bucket {
uint16_t sig[SMC_ENTRY_PER_BUCKET];
uint16_t flow_idx[SMC_ENTRY_PER_BUCKET];
};
/* Signature match cache, differentiate from EMC cache */
struct smc_cache {
struct smc_bucket buckets[SMC_BUCKET_CNT];
};
struct dfc_cache {
struct emc_cache emc_cache;
struct smc_cache smc_cache;
};
/* Iterate in the exact match cache through every entry that might contain a
* miniflow with hash 'HASH'. */
#define EMC_FOR_EACH_POS_WITH_HASH(EMC, CURRENT_ENTRY, HASH) \
@@ -215,10 +251,11 @@ static void dpcls_insert(struct dpcls *, struct dpcls_rule *,
const struct netdev_flow_key *mask);
static void dpcls_remove(struct dpcls *, struct dpcls_rule *);
static bool dpcls_lookup(struct dpcls *cls,
const struct netdev_flow_key keys[],
const struct netdev_flow_key *keys[],
struct dpcls_rule **rules, size_t cnt,
int *num_lookups_p);
static bool dpcls_rule_matches_key(const struct dpcls_rule *rule,
const struct netdev_flow_key *target);
/* Set of supported meter flags */
#define DP_SUPPORTED_METER_FLAGS_MASK \
(OFPMF13_STATS | OFPMF13_PKTPS | OFPMF13_KBPS | OFPMF13_BURST)
@@ -285,6 +322,8 @@ struct dp_netdev {
OVS_ALIGNED_VAR(CACHE_LINE_SIZE) atomic_uint32_t emc_insert_min;
/* Enable collection of PMD performance metrics. */
atomic_bool pmd_perf_metrics;
/* Enable the SMC cache from ovsdb config */
atomic_bool smc_enable_db;
/* Protects access to ofproto-dpif-upcall interface during revalidator
* thread synchronization. */
@@ -587,7 +626,7 @@ struct dp_netdev_pmd_thread {
* NON_PMD_CORE_ID can be accessed by multiple threads, and thusly
* need to be protected by 'non_pmd_mutex'. Every other instance
* will only be accessed by its own pmd thread. */
struct emc_cache flow_cache;
OVS_ALIGNED_VAR(CACHE_LINE_SIZE) struct dfc_cache flow_cache;
/* Flow-Table and classifiers
*
@@ -755,6 +794,7 @@ static int dpif_netdev_xps_get_tx_qid(const struct dp_netdev_pmd_thread *pmd,
static inline bool emc_entry_alive(struct emc_entry *ce);
static void emc_clear_entry(struct emc_entry *ce);
static void smc_clear_entry(struct smc_bucket *b, int idx);
static void dp_netdev_request_reconfigure(struct dp_netdev *dp);
static inline bool
@@ -776,6 +816,24 @@ emc_cache_init(struct emc_cache *flow_cache)
}
}
static void
smc_cache_init(struct smc_cache *smc_cache)
{
int i, j;
for (i = 0; i < SMC_BUCKET_CNT; i++) {
for (j = 0; j < SMC_ENTRY_PER_BUCKET; j++) {
smc_cache->buckets[i].flow_idx[j] = UINT16_MAX;
}
}
}
static void
dfc_cache_init(struct dfc_cache *flow_cache)
{
emc_cache_init(&flow_cache->emc_cache);
smc_cache_init(&flow_cache->smc_cache);
}
static void
emc_cache_uninit(struct emc_cache *flow_cache)
{
@@ -786,6 +844,25 @@ emc_cache_uninit(struct emc_cache *flow_cache)
}
}
static void
smc_cache_uninit(struct smc_cache *smc)
{
int i, j;
for (i = 0; i < SMC_BUCKET_CNT; i++) {
for (j = 0; j < SMC_ENTRY_PER_BUCKET; j++) {
smc_clear_entry(&(smc->buckets[i]), j);
}
}
}
static void
dfc_cache_uninit(struct dfc_cache *flow_cache)
{
smc_cache_uninit(&flow_cache->smc_cache);
emc_cache_uninit(&flow_cache->emc_cache);
}
/* Check and clear dead flow references slowly (one entry at each
* invocation). */
static void
@@ -897,6 +974,7 @@ pmd_info_show_stats(struct ds *reply,
" packet recirculations: %"PRIu64"\n"
" avg. datapath passes per packet: %.02f\n"
" emc hits: %"PRIu64"\n"
" smc hits: %"PRIu64"\n"
" megaflow hits: %"PRIu64"\n"
" avg. subtable lookups per megaflow hit: %.02f\n"
" miss with success upcall: %"PRIu64"\n"
@@ -904,6 +982,7 @@ pmd_info_show_stats(struct ds *reply,
" avg. packets per output batch: %.02f\n",
total_packets, stats[PMD_STAT_RECIRC],
passes_per_pkt, stats[PMD_STAT_EXACT_HIT],
stats[PMD_STAT_SMC_HIT],
stats[PMD_STAT_MASKED_HIT], lookups_per_hit,
stats[PMD_STAT_MISS], stats[PMD_STAT_LOST],
packets_per_batch);
@@ -1617,6 +1696,7 @@ dpif_netdev_get_stats(const struct dpif *dpif, struct dpif_dp_stats *stats)
stats->n_flows += cmap_count(&pmd->flow_table);
pmd_perf_read_counters(&pmd->perf_stats, pmd_stats);
stats->n_hit += pmd_stats[PMD_STAT_EXACT_HIT];
stats->n_hit += pmd_stats[PMD_STAT_SMC_HIT];
stats->n_hit += pmd_stats[PMD_STAT_MASKED_HIT];
stats->n_missed += pmd_stats[PMD_STAT_MISS];
stats->n_lost += pmd_stats[PMD_STAT_LOST];
@@ -2721,10 +2801,11 @@ emc_probabilistic_insert(struct dp_netdev_pmd_thread *pmd,
* probability of 1/100 ie. 1% */
uint32_t min;
atomic_read_relaxed(&pmd->dp->emc_insert_min, &min);
if (min && random_uint32() <= min) {
emc_insert(&pmd->flow_cache, key, flow);
emc_insert(&(pmd->flow_cache).emc_cache, key, flow);
}
}
@@ -2746,6 +2827,86 @@ emc_lookup(struct emc_cache *cache, const struct netdev_flow_key *key)
return NULL;
}
static inline const struct cmap_node *
smc_entry_get(struct dp_netdev_pmd_thread *pmd, const uint32_t hash)
{
struct smc_cache *cache = &(pmd->flow_cache).smc_cache;
struct smc_bucket *bucket = &cache->buckets[hash & SMC_MASK];
uint16_t sig = hash >> 16;
uint16_t index = UINT16_MAX;
for (int i = 0; i < SMC_ENTRY_PER_BUCKET; i++) {
if (bucket->sig[i] == sig) {
index = bucket->flow_idx[i];
break;
}
}
if (index != UINT16_MAX) {
return cmap_find_by_index(&pmd->flow_table, index);
}
return NULL;
}
static void
smc_clear_entry(struct smc_bucket *b, int idx)
{
b->flow_idx[idx] = UINT16_MAX;
}
/* Insert the flow_table index into SMC. Insertion may fail when 1) SMC is
* turned off, 2) the flow_table index is larger than uint16_t can handle.
* If there is already an SMC entry having same signature, the index will be
* updated. If there is no existing entry, but an empty entry is available,
* the empty entry will be taken. If no empty entry or existing same signature,
* a random entry from the hashed bucket will be picked. */
static inline void
smc_insert(struct dp_netdev_pmd_thread *pmd,
const struct netdev_flow_key *key,
uint32_t hash)
{
struct smc_cache *smc_cache = &(pmd->flow_cache).smc_cache;
struct smc_bucket *bucket = &smc_cache->buckets[key->hash & SMC_MASK];
uint16_t index;
uint32_t cmap_index;
bool smc_enable_db;
int i;
atomic_read_relaxed(&pmd->dp->smc_enable_db, &smc_enable_db);
if (!smc_enable_db) {
return;
}
cmap_index = cmap_find_index(&pmd->flow_table, hash);
index = (cmap_index >= UINT16_MAX) ? UINT16_MAX : (uint16_t)cmap_index;
/* If the index is larger than SMC can handle (uint16_t), we don't
* insert */
if (index == UINT16_MAX) {
return;
}
/* If an entry with same signature already exists, update the index */
uint16_t sig = key->hash >> 16;
for (i = 0; i < SMC_ENTRY_PER_BUCKET; i++) {
if (bucket->sig[i] == sig) {
bucket->flow_idx[i] = index;
return;
}
}
/* If there is an empty entry, occupy it. */
for (i = 0; i < SMC_ENTRY_PER_BUCKET; i++) {
if (bucket->flow_idx[i] == UINT16_MAX) {
bucket->sig[i] = sig;
bucket->flow_idx[i] = index;
return;
}
}
/* Otherwise, pick a random entry. */
i = random_uint32() % SMC_ENTRY_PER_BUCKET;
bucket->sig[i] = sig;
bucket->flow_idx[i] = index;
}
static struct dp_netdev_flow *
dp_netdev_pmd_lookup_flow(struct dp_netdev_pmd_thread *pmd,
const struct netdev_flow_key *key,
@@ -2759,7 +2920,7 @@ dp_netdev_pmd_lookup_flow(struct dp_netdev_pmd_thread *pmd,
cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
if (OVS_LIKELY(cls)) {
dpcls_lookup(cls, key, &rule, 1, lookup_num_p);
dpcls_lookup(cls, &key, &rule, 1, lookup_num_p);
netdev_flow = dp_netdev_flow_cast(rule);
}
return netdev_flow;
@@ -3606,6 +3767,17 @@ dpif_netdev_set_config(struct dpif *dpif, const struct smap *other_config)
}
}
bool smc_enable = smap_get_bool(other_config, "smc-enable", false);
bool cur_smc;
atomic_read_relaxed(&dp->smc_enable_db, &cur_smc);
if (smc_enable != cur_smc) {
atomic_store_relaxed(&dp->smc_enable_db, smc_enable);
if (smc_enable) {
VLOG_INFO("SMC cache is enabled");
} else {
VLOG_INFO("SMC cache is disabled");
}
}
return 0;
}
@@ -4740,7 +4912,7 @@ pmd_thread_main(void *f_)
ovs_numa_thread_setaffinity_core(pmd->core_id);
dpdk_set_lcore_id(pmd->core_id);
poll_cnt = pmd_load_queues_and_ports(pmd, &poll_list);
emc_cache_init(&pmd->flow_cache);
dfc_cache_init(&pmd->flow_cache);
reload:
pmd_alloc_static_tx_qid(pmd);
@@ -4794,7 +4966,7 @@ reload:
coverage_try_clear();
dp_netdev_pmd_try_optimize(pmd, poll_list, poll_cnt);
if (!ovsrcu_try_quiesce()) {
emc_cache_slow_sweep(&pmd->flow_cache);
emc_cache_slow_sweep(&((pmd->flow_cache).emc_cache));
}
atomic_read_relaxed(&pmd->reload, &reload);
@@ -4819,7 +4991,7 @@ reload:
goto reload;
}
emc_cache_uninit(&pmd->flow_cache);
dfc_cache_uninit(&pmd->flow_cache);
free(poll_list);
pmd_free_cached_ports(pmd);
return NULL;
@@ -5255,7 +5427,7 @@ dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd, struct dp_netdev *dp,
/* init the 'flow_cache' since there is no
* actual thread created for NON_PMD_CORE_ID. */
if (core_id == NON_PMD_CORE_ID) {
emc_cache_init(&pmd->flow_cache);
dfc_cache_init(&pmd->flow_cache);
pmd_alloc_static_tx_qid(pmd);
}
pmd_perf_stats_init(&pmd->perf_stats);
@@ -5298,7 +5470,7 @@ dp_netdev_del_pmd(struct dp_netdev *dp, struct dp_netdev_pmd_thread *pmd)
* but extra cleanup is necessary */
if (pmd->core_id == NON_PMD_CORE_ID) {
ovs_mutex_lock(&dp->non_pmd_mutex);
emc_cache_uninit(&pmd->flow_cache);
dfc_cache_uninit(&pmd->flow_cache);
pmd_free_cached_ports(pmd);
pmd_free_static_tx_qid(pmd);
ovs_mutex_unlock(&dp->non_pmd_mutex);
@@ -5602,10 +5774,72 @@ dp_netdev_queue_batches(struct dp_packet *pkt,
packet_batch_per_flow_update(batch, pkt, tcp_flags);
}
/* Try to process all ('cnt') the 'packets' using only the exact match cache
/* SMC lookup function for a batch of packets.
* By doing batching SMC lookup, we can use prefetch
* to hide memory access latency.
*/
static inline void
smc_lookup_batch(struct dp_netdev_pmd_thread *pmd,
struct netdev_flow_key *keys,
struct netdev_flow_key **missed_keys,
struct dp_packet_batch *packets_,
struct packet_batch_per_flow batches[],
size_t *n_batches, const int cnt)
{
int i;
struct dp_packet *packet;
size_t n_smc_hit = 0, n_missed = 0;
struct dfc_cache *cache = &pmd->flow_cache;
struct smc_cache *smc_cache = &cache->smc_cache;
const struct cmap_node *flow_node;
/* Prefetch buckets for all packets */
for (i = 0; i < cnt; i++) {
OVS_PREFETCH(&smc_cache->buckets[keys[i].hash & SMC_MASK]);
}
DP_PACKET_BATCH_REFILL_FOR_EACH (i, cnt, packet, packets_) {
struct dp_netdev_flow *flow = NULL;
flow_node = smc_entry_get(pmd, keys[i].hash);
bool hit = false;
if (OVS_LIKELY(flow_node != NULL)) {
CMAP_NODE_FOR_EACH (flow, node, flow_node) {
/* Since we dont have per-port megaflow to check the port
* number, we need to verify that the input ports match. */
if (OVS_LIKELY(dpcls_rule_matches_key(&flow->cr, &keys[i]) &&
flow->flow.in_port.odp_port == packet->md.in_port.odp_port)) {
/* SMC hit and emc miss, we insert into EMC */
emc_probabilistic_insert(pmd, &keys[i], flow);
keys[i].len =
netdev_flow_key_size(miniflow_n_values(&keys[i].mf));
dp_netdev_queue_batches(packet, flow,
miniflow_get_tcp_flags(&keys[i].mf), batches, n_batches);
n_smc_hit++;
hit = true;
break;
}
}
if (hit) {
continue;
}
}
/* SMC missed. Group missed packets together at
* the beginning of the 'packets' array. */
dp_packet_batch_refill(packets_, packet, i);
/* Put missed keys to the pointer arrays return to the caller */
missed_keys[n_missed++] = &keys[i];
}
pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_SMC_HIT, n_smc_hit);
}
/* Try to process all ('cnt') the 'packets' using only the datapath flow cache
* 'pmd->flow_cache'. If a flow is not found for a packet 'packets[i]', the
* miniflow is copied into 'keys' and the packet pointer is moved at the
* beginning of the 'packets' array.
* beginning of the 'packets' array. The pointers of missed keys are put in the
* missed_keys pointer array for future processing.
*
* The function returns the number of packets that needs to be processed in the
* 'packets' array (they have been moved to the beginning of the vector).
@@ -5617,21 +5851,24 @@ dp_netdev_queue_batches(struct dp_packet *pkt,
* will be ignored.
*/
static inline size_t
emc_processing(struct dp_netdev_pmd_thread *pmd,
dfc_processing(struct dp_netdev_pmd_thread *pmd,
struct dp_packet_batch *packets_,
struct netdev_flow_key *keys,
struct netdev_flow_key **missed_keys,
struct packet_batch_per_flow batches[], size_t *n_batches,
bool md_is_valid, odp_port_t port_no)
{
struct emc_cache *flow_cache = &pmd->flow_cache;
struct netdev_flow_key *key = &keys[0];
size_t n_missed = 0, n_dropped = 0;
size_t n_missed = 0, n_emc_hit = 0;
struct dfc_cache *cache = &pmd->flow_cache;
struct dp_packet *packet;
const size_t cnt = dp_packet_batch_size(packets_);
uint32_t cur_min;
int i;
uint16_t tcp_flags;
bool smc_enable_db;
atomic_read_relaxed(&pmd->dp->smc_enable_db, &smc_enable_db);
atomic_read_relaxed(&pmd->dp->emc_insert_min, &cur_min);
pmd_perf_update_counter(&pmd->perf_stats,
md_is_valid ? PMD_STAT_RECIRC : PMD_STAT_RECV,
@@ -5643,7 +5880,6 @@ emc_processing(struct dp_netdev_pmd_thread *pmd,
if (OVS_UNLIKELY(dp_packet_size(packet) < ETH_HEADER_LEN)) {
dp_packet_delete(packet);
n_dropped++;
continue;
}
@@ -5671,15 +5907,17 @@ emc_processing(struct dp_netdev_pmd_thread *pmd,
miniflow_extract(packet, &key->mf);
key->len = 0; /* Not computed yet. */
/* If EMC is disabled skip hash computation and emc_lookup */
if (cur_min) {
/* If EMC and SMC disabled skip hash computation */
if (smc_enable_db == true || cur_min != 0) {
if (!md_is_valid) {
key->hash = dpif_netdev_packet_get_rss_hash_orig_pkt(packet,
&key->mf);
} else {
key->hash = dpif_netdev_packet_get_rss_hash(packet, &key->mf);
}
flow = emc_lookup(flow_cache, key);
}
if (cur_min) {
flow = emc_lookup(&cache->emc_cache, key);
} else {
flow = NULL;
}
@@ -5687,19 +5925,30 @@ emc_processing(struct dp_netdev_pmd_thread *pmd,
tcp_flags = miniflow_get_tcp_flags(&key->mf);
dp_netdev_queue_batches(packet, flow, tcp_flags, batches,
n_batches);
n_emc_hit++;
} else {
/* Exact match cache missed. Group missed packets together at
* the beginning of the 'packets' array. */
dp_packet_batch_refill(packets_, packet, i);
/* 'key[n_missed]' contains the key of the current packet and it
* must be returned to the caller. The next key should be extracted
* to 'keys[n_missed + 1]'. */
* will be passed to SMC lookup. The next key should be extracted
* to 'keys[n_missed + 1]'.
* We also maintain a pointer array to keys missed both SMC and EMC
* which will be returned to the caller for future processing. */
missed_keys[n_missed] = key;
key = &keys[++n_missed];
}
}
pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_EXACT_HIT,
cnt - n_dropped - n_missed);
pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_EXACT_HIT, n_emc_hit);
if (!smc_enable_db) {
return dp_packet_batch_size(packets_);
}
/* Packets miss EMC will do a batch lookup in SMC if enabled */
smc_lookup_batch(pmd, keys, missed_keys, packets_, batches,
n_batches, n_missed);
return dp_packet_batch_size(packets_);
}
@@ -5767,6 +6016,8 @@ handle_packet_upcall(struct dp_netdev_pmd_thread *pmd,
add_actions->size);
}
ovs_mutex_unlock(&pmd->flow_mutex);
uint32_t hash = dp_netdev_flow_hash(&netdev_flow->ufid);
smc_insert(pmd, key, hash);
emc_probabilistic_insert(pmd, key, netdev_flow);
}
if (pmd_perf_metrics_enabled(pmd)) {
@@ -5783,7 +6034,7 @@ handle_packet_upcall(struct dp_netdev_pmd_thread *pmd,
static inline void
fast_path_processing(struct dp_netdev_pmd_thread *pmd,
struct dp_packet_batch *packets_,
struct netdev_flow_key *keys,
struct netdev_flow_key **keys,
struct packet_batch_per_flow batches[],
size_t *n_batches,
odp_port_t in_port)
@@ -5805,12 +6056,13 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
for (size_t i = 0; i < cnt; i++) {
/* Key length is needed in all the cases, hash computed on demand. */
keys[i].len = netdev_flow_key_size(miniflow_n_values(&keys[i].mf));
keys[i]->len = netdev_flow_key_size(miniflow_n_values(&keys[i]->mf));
}
/* Get the classifier for the in_port */
cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
if (OVS_LIKELY(cls)) {
any_miss = !dpcls_lookup(cls, keys, rules, cnt, &lookup_cnt);
any_miss = !dpcls_lookup(cls, (const struct netdev_flow_key **)keys,
rules, cnt, &lookup_cnt);
} else {
any_miss = true;
memset(rules, 0, sizeof(rules));
@@ -5832,7 +6084,7 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
/* It's possible that an earlier slow path execution installed
* a rule covering this flow. In this case, it's a lot cheaper
* to catch it here than execute a miss. */
netdev_flow = dp_netdev_pmd_lookup_flow(pmd, &keys[i],
netdev_flow = dp_netdev_pmd_lookup_flow(pmd, keys[i],
&add_lookup_cnt);
if (netdev_flow) {
lookup_cnt += add_lookup_cnt;
@@ -5840,7 +6092,7 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
continue;
}
int error = handle_packet_upcall(pmd, packet, &keys[i],
int error = handle_packet_upcall(pmd, packet, keys[i],
&actions, &put_actions);
if (OVS_UNLIKELY(error)) {
@@ -5870,10 +6122,12 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
}
flow = dp_netdev_flow_cast(rules[i]);
uint32_t hash = dp_netdev_flow_hash(&flow->ufid);
smc_insert(pmd, keys[i], hash);
emc_probabilistic_insert(pmd, &keys[i], flow);
emc_probabilistic_insert(pmd, keys[i], flow);
dp_netdev_queue_batches(packet, flow,
miniflow_get_tcp_flags(&keys[i].mf),
miniflow_get_tcp_flags(&keys[i]->mf),
batches, n_batches);
}
@@ -5904,17 +6158,18 @@ dp_netdev_input__(struct dp_netdev_pmd_thread *pmd,
#endif
OVS_ALIGNED_VAR(CACHE_LINE_SIZE)
struct netdev_flow_key keys[PKT_ARRAY_SIZE];
struct netdev_flow_key *missed_keys[PKT_ARRAY_SIZE];
struct packet_batch_per_flow batches[PKT_ARRAY_SIZE];
size_t n_batches;
odp_port_t in_port;
n_batches = 0;
emc_processing(pmd, packets, keys, batches, &n_batches,
dfc_processing(pmd, packets, keys, missed_keys, batches, &n_batches,
md_is_valid, port_no);
if (!dp_packet_batch_is_empty(packets)) {
/* Get ingress port from first packet's metadata. */
in_port = packets->packets[0]->md.in_port.odp_port;
fast_path_processing(pmd, packets, keys,
fast_path_processing(pmd, packets, missed_keys,
batches, &n_batches, in_port);
}
@@ -6864,7 +7119,7 @@ dpcls_remove(struct dpcls *cls, struct dpcls_rule *rule)
/* Returns true if 'target' satisfies 'key' in 'mask', that is, if each 1-bit
* in 'mask' the values in 'key' and 'target' are the same. */
static inline bool
static bool
dpcls_rule_matches_key(const struct dpcls_rule *rule,
const struct netdev_flow_key *target)
{
@@ -6891,7 +7146,7 @@ dpcls_rule_matches_key(const struct dpcls_rule *rule,
*
* Returns true if all miniflows found a corresponding rule. */
static bool
dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key keys[],
dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key *keys[],
struct dpcls_rule **rules, const size_t cnt,
int *num_lookups_p)
{
@@ -6930,7 +7185,7 @@ dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key keys[],
* masked with the subtable's mask to avoid hashing the wildcarded
* bits. */
ULLONG_FOR_EACH_1(i, keys_map) {
hashes[i] = netdev_flow_key_hash_in_mask(&keys[i],
hashes[i] = netdev_flow_key_hash_in_mask(keys[i],
&subtable->mask);
}
/* Lookup. */
@@ -6944,7 +7199,7 @@ dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key keys[],
struct dpcls_rule *rule;
CMAP_NODE_FOR_EACH (rule, cmap_node, nodes[i]) {
if (OVS_LIKELY(dpcls_rule_matches_key(rule, &keys[i]))) {
if (OVS_LIKELY(dpcls_rule_matches_key(rule, keys[i]))) {
rules[i] = rule;
/* Even at 20 Mpps the 32-bit hit_cnt cannot wrap
* within one second optimization interval. */