diff --git a/Documentation/topics/dpdk/bridge.rst b/Documentation/topics/dpdk/bridge.rst index 63f8a62dee2..df74c02adf9 100644 --- a/Documentation/topics/dpdk/bridge.rst +++ b/Documentation/topics/dpdk/bridge.rst @@ -102,3 +102,18 @@ For certain traffic profiles with many parallel flows, it's recommended to set ``N`` to '0' to achieve higher forwarding performance. For more information on the EMC refer to :doc:`/intro/install/dpdk` . + + +SMC cache (experimental) +------------------------- + +SMC cache or signature match cache is a new cache level after EMC cache. +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. When traffic flow count is much larger than +EMC size, it is generally beneficial to turn off EMC and turn on SMC. It is +currently turned off by default and an experimental feature. + +To turn on SMC:: + + $ ovs-vsctl --no-wait set Open_vSwitch . other_config:smc-enable=true diff --git a/NEWS b/NEWS index 2414b9199ae..2f04da6cda9 100644 --- a/NEWS +++ b/NEWS @@ -48,6 +48,8 @@ Post-v2.9.0 ovs-appctl dpif-netdev/pmd-perf-show * Supervision of PMD performance metrics and logging of suspicious iterations + * Add signature match cache (SMC) as experimental feature. When turned on, + it improves throughput when traffic has many more flows than EMC size. - ERSPAN: * Implemented ERSPAN protocol (draft-foschiano-erspan-00.txt) for both kernel datapath and userspace datapath. diff --git a/lib/cmap.c b/lib/cmap.c index 07719a8fd28..cb9cd32ab3d 100644 --- a/lib/cmap.c +++ b/lib/cmap.c @@ -373,6 +373,80 @@ cmap_find(const struct cmap *cmap, uint32_t hash) hash); } +/* Find a node by the index of the entry of cmap. Index N means the N/CMAP_K + * bucket and N%CMAP_K entry in that bucket. + * Notice that it is not protected by the optimistic lock (versioning) because + * it does not compare the hashes. Currently it is only used by the datapath + * SMC cache. + * + * Return node for the entry of index or NULL if the index beyond boundary */ +const struct cmap_node * +cmap_find_by_index(const struct cmap *cmap, uint32_t index) +{ + const struct cmap_impl *impl = cmap_get_impl(cmap); + + uint32_t b = index / CMAP_K; + uint32_t e = index % CMAP_K; + + if (b > impl->mask) { + return NULL; + } + + const struct cmap_bucket *bucket = &impl->buckets[b]; + + return cmap_node_next(&bucket->nodes[e]); +} + +/* Find the index of certain hash value. Currently only used by the datapath + * SMC cache. + * + * Return the index of the entry if found, or UINT32_MAX if not found. The + * function assumes entry index cannot be larger than UINT32_MAX. */ +uint32_t +cmap_find_index(const struct cmap *cmap, uint32_t hash) +{ + const struct cmap_impl *impl = cmap_get_impl(cmap); + uint32_t h1 = rehash(impl, hash); + uint32_t h2 = other_hash(h1); + + uint32_t b_index1 = h1 & impl->mask; + uint32_t b_index2 = h2 & impl->mask; + + uint32_t c1, c2; + uint32_t index = UINT32_MAX; + + const struct cmap_bucket *b1 = &impl->buckets[b_index1]; + const struct cmap_bucket *b2 = &impl->buckets[b_index2]; + + do { + do { + c1 = read_even_counter(b1); + for (int i = 0; i < CMAP_K; i++) { + if (b1->hashes[i] == hash) { + index = b_index1 * CMAP_K + i; + } + } + } while (OVS_UNLIKELY(counter_changed(b1, c1))); + if (index != UINT32_MAX) { + break; + } + do { + c2 = read_even_counter(b2); + for (int i = 0; i < CMAP_K; i++) { + if (b2->hashes[i] == hash) { + index = b_index2 * CMAP_K + i; + } + } + } while (OVS_UNLIKELY(counter_changed(b2, c2))); + + if (index != UINT32_MAX) { + break; + } + } while (OVS_UNLIKELY(counter_changed(b1, c1))); + + return index; +} + /* Looks up multiple 'hashes', when the corresponding bit in 'map' is 1, * and sets the corresponding pointer in 'nodes', if the hash value was * found from the 'cmap'. In other cases the 'nodes' values are not changed, diff --git a/lib/cmap.h b/lib/cmap.h index 8bfb6c09434..d9db3c915cb 100644 --- a/lib/cmap.h +++ b/lib/cmap.h @@ -145,6 +145,17 @@ size_t cmap_replace(struct cmap *, struct cmap_node *old_node, const struct cmap_node *cmap_find(const struct cmap *, uint32_t hash); struct cmap_node *cmap_find_protected(const struct cmap *, uint32_t hash); +/* Find node by index or find index by hash. The 'index' of a cmap entry is a + * way to combine the specific bucket and the entry of the bucket into a + * convenient single integer value. In other words, it is the index of the + * entry and each entry has an unique index. It is not used internally by + * cmap. + * Currently the functions assume index will not be larger than uint32_t. In + * OvS table size is usually much smaller than this size.*/ +const struct cmap_node * cmap_find_by_index(const struct cmap *, + uint32_t index); +uint32_t cmap_find_index(const struct cmap *, uint32_t hash); + /* Looks up multiple 'hashes', when the corresponding bit in 'map' is 1, * and sets the corresponding pointer in 'nodes', if the hash value was * found from the 'cmap'. In other cases the 'nodes' values are not changed, diff --git a/lib/dpif-netdev-perf.h b/lib/dpif-netdev-perf.h index b8aa4e34483..299d52a9808 100644 --- a/lib/dpif-netdev-perf.h +++ b/lib/dpif-netdev-perf.h @@ -56,6 +56,7 @@ extern "C" { enum pmd_stat_type { PMD_STAT_EXACT_HIT, /* Packets that had an exact match (emc). */ + PMD_STAT_SMC_HIT, /* Packets that had a sig match hit (SMC). */ PMD_STAT_MASKED_HIT, /* Packets that matched in the flow table. */ PMD_STAT_MISS, /* Packets that did not match and upcall was ok. */ PMD_STAT_LOST, /* Packets that did not match and upcall failed. */ diff --git a/lib/dpif-netdev.c b/lib/dpif-netdev.c index 8b3556d84d3..13a20f02355 100644 --- a/lib/dpif-netdev.c +++ b/lib/dpif-netdev.c @@ -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 @@ pmd_thread_main(void *f_) 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 @@ pmd_thread_main(void *f_) 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. */ diff --git a/tests/pmd.at b/tests/pmd.at index f3fac6385b6..4cae6c8c570 100644 --- a/tests/pmd.at +++ b/tests/pmd.at @@ -185,6 +185,7 @@ CHECK_PMD_THREADS_CREATED() AT_CHECK([ovs-appctl vlog/set dpif_netdev:dbg]) AT_CHECK([ovs-ofctl add-flow br0 action=normal]) AT_CHECK([ovs-vsctl set Open_vSwitch . other_config:emc-insert-inv-prob=1]) +AT_CHECK([ovs-vsctl set Open_vSwitch . other_config:smc-enable=true]) sleep 1 @@ -195,12 +196,13 @@ dummy@ovs-dummy: hit:0 missed:0 p0 7/1: (dummy-pmd: configured_rx_queues=4, configured_tx_queues=, requested_rx_queues=4, requested_tx_queues=) ]) -AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed SED_NUMA_CORE_PATTERN | sed '/cycles/d' | grep pmd -A 8], [0], [dnl +AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed SED_NUMA_CORE_PATTERN | sed '/cycles/d' | grep pmd -A 9], [0], [dnl pmd thread numa_id core_id : packets received: 0 packet recirculations: 0 avg. datapath passes per packet: 0.00 emc hits: 0 + smc hits: 0 megaflow hits: 0 avg. subtable lookups per megaflow hit: 0.00 miss with success upcall: 0 @@ -225,12 +227,13 @@ AT_CHECK([cat ovs-vswitchd.log | filter_flow_install | strip_xout], [0], [dnl recirc_id(0),in_port(1),packet_type(ns=0,id=0),eth(src=50:54:00:00:00:77,dst=50:54:00:00:01:78),eth_type(0x0800),ipv4(frag=no), actions: ]) -AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed SED_NUMA_CORE_PATTERN | sed '/cycles/d' | grep pmd -A 8], [0], [dnl +AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed SED_NUMA_CORE_PATTERN | sed '/cycles/d' | grep pmd -A 9], [0], [dnl pmd thread numa_id core_id : packets received: 20 packet recirculations: 0 avg. datapath passes per packet: 1.00 emc hits: 19 + smc hits: 0 megaflow hits: 0 avg. subtable lookups per megaflow hit: 0.00 miss with success upcall: 1 diff --git a/vswitchd/vswitch.xml b/vswitchd/vswitch.xml index 63a3a2ed166..63429494479 100644 --- a/vswitchd/vswitch.xml +++ b/vswitchd/vswitch.xml @@ -405,6 +405,19 @@

+ +

+ Signature match cache or SMC is a cache between EMC and megaflow + cache. It does not store the full key of the flow, so it is more + memory efficient comparing to EMC cache. SMC is especially useful + when flow count is larger than EMC capacity. +

+

+ Defaults to false but can be changed at any time. +

+
+