From 0de8783a9d3a339032b4213b91d66d53f1c0643f Mon Sep 17 00:00:00 2001 From: Jarno Rajahalme Date: Fri, 17 Oct 2014 09:37:11 -0700 Subject: [PATCH] lib/dpif-netdev: Integrate megaflow classifier. Megaflow inserts and removals are simplified: - No need for classifier internal mutex, as dpif-netdev already has a 'flow_mutex'. - Number of memory allocations/frees can be halved. - Lookup code path can rely on netdev_flow_key always having inline data. This will also be easier to simplify further when moving to per-thread megaflow classifiers in the future. Signed-off-by: Jarno Rajahalme Acked-by: Alex Wang --- lib/classifier.c | 109 --------- lib/classifier.h | 5 - lib/dpif-netdev.c | 604 ++++++++++++++++++++++++++++++++++++---------- lib/flow.c | 61 +++++ lib/flow.h | 3 +- 5 files changed, 535 insertions(+), 247 deletions(-) diff --git a/lib/classifier.c b/lib/classifier.c index f30ba9579..8dc89d963 100644 --- a/lib/classifier.c +++ b/lib/classifier.c @@ -970,115 +970,6 @@ classifier_lookup(const struct classifier *cls, const struct flow *flow, return best ? best->cls_rule : NULL; } -/* Returns true if 'target' satisifies 'match', that is, if each bit for which - * 'match' specifies a particular value has the correct value in 'target'. - * - * 'flow' and 'mask' have the same mask! */ -static bool -miniflow_and_mask_matches_miniflow(const struct miniflow *flow, - const struct minimask *mask, - const struct miniflow *target) -{ - const uint32_t *flowp = miniflow_get_u32_values(flow); - const uint32_t *maskp = miniflow_get_u32_values(&mask->masks); - uint32_t target_u32; - - MINIFLOW_FOR_EACH_IN_MAP(target_u32, target, mask->masks.map) { - if ((*flowp++ ^ target_u32) & *maskp++) { - return false; - } - } - - return true; -} - -/* For each miniflow in 'flows' performs a classifier lookup writing the result - * into the corresponding slot in 'rules'. - * - * This function is optimized for use in the userspace datapath and therefore - * does not implement a lot of features available in the standard - * classifier_lookup() function. Specifically, it does not implement - * priorities, instead returning any rule which matches the flow. - * - * Returns true if all flows found a corresponding rule. */ -bool -classifier_lookup_miniflow_batch(const struct classifier *cls, - const struct miniflow *flows[], - struct cls_rule *rules[], const size_t cnt) -{ - /* The batch size 16 was experimentally found faster than 8 or 32. */ - typedef uint16_t map_type; -#define MAP_BITS (sizeof(map_type) * CHAR_BIT) - - struct cls_subtable *subtable; - const int n_maps = DIV_ROUND_UP(cnt, MAP_BITS); - -#if !defined(__CHECKER__) && !defined(_WIN32) - map_type maps[n_maps]; -#else - map_type maps[DIV_ROUND_UP(CLASSIFIER_MAX_BATCH, MAP_BITS)]; - ovs_assert(n_maps <= CLASSIFIER_MAX_BATCH); -#endif - BUILD_ASSERT_DECL(sizeof *maps * CHAR_BIT == MAP_BITS); - - memset(maps, 0xff, sizeof maps); - if (cnt % MAP_BITS) { - maps[n_maps - 1] >>= MAP_BITS - cnt % MAP_BITS; /* Clear extra bits. */ - } - memset(rules, 0, cnt * sizeof *rules); - - PVECTOR_FOR_EACH (subtable, &cls->subtables) { - const struct miniflow **mfs = flows; - struct cls_rule **results = rules; - map_type remains = 0; - int m; - - BUILD_ASSERT_DECL(sizeof remains == sizeof *maps); - - for (m = 0; m < n_maps; m++, mfs += MAP_BITS, results += MAP_BITS) { - uint32_t hashes[MAP_BITS]; - const struct cmap_node *nodes[MAP_BITS]; - unsigned long map = maps[m]; - int i; - - if (!map) { - continue; /* Skip empty ones. */ - } - - /* Compute hashes for the unfound flows. */ - ULONG_FOR_EACH_1(i, map) { - hashes[i] = miniflow_hash_in_minimask(mfs[i], &subtable->mask, - 0); - } - /* Lookup. */ - map = cmap_find_batch(&subtable->rules, map, hashes, nodes); - /* Check results. */ - ULONG_FOR_EACH_1(i, map) { - struct cls_match *rule; - - CMAP_NODE_FOR_EACH (rule, cmap_node, nodes[i]) { - if (OVS_LIKELY(miniflow_and_mask_matches_miniflow( - &rule->flow, &subtable->mask, - mfs[i]))) { - results[i] = rule->cls_rule; - goto next; - } - } - ULONG_SET0(map, i); /* Did not match. */ - next: - ; /* Keep Sparse happy. */ - } - maps[m] &= ~map; /* Clear the found rules. */ - remains |= maps[m]; - } - if (!remains) { - return true; /* All found. */ - } - } - /* Some misses. */ - return false; -} - /* Finds and returns a rule in 'cls' with exactly the same priority and * matching criteria as 'target'. Returns a null pointer if 'cls' doesn't * contain an exact match. */ diff --git a/lib/classifier.h b/lib/classifier.h index d1f4e866b..c910ac4bf 100644 --- a/lib/classifier.h +++ b/lib/classifier.h @@ -295,11 +295,6 @@ struct cls_rule *classifier_remove(struct classifier *, struct cls_rule *); struct cls_rule *classifier_lookup(const struct classifier *, const struct flow *, struct flow_wildcards *); -bool classifier_lookup_miniflow_batch(const struct classifier *cls, - const struct miniflow **flows, - struct cls_rule **rules, - const size_t cnt); -enum { CLASSIFIER_MAX_BATCH = 256 }; bool classifier_rule_overlaps(const struct classifier *, const struct cls_rule *); diff --git a/lib/dpif-netdev.c b/lib/dpif-netdev.c index cea7c888f..f6a0c48ca 100644 --- a/lib/dpif-netdev.c +++ b/lib/dpif-netdev.c @@ -31,7 +31,6 @@ #include #include -#include "classifier.h" #include "cmap.h" #include "csum.h" #include "dpif.h" @@ -43,6 +42,7 @@ #include "cmap.h" #include "latch.h" #include "list.h" +#include "match.h" #include "meta-flow.h" #include "netdev.h" #include "netdev-dpdk.h" @@ -57,6 +57,7 @@ #include "packet-dpif.h" #include "packets.h" #include "poll-loop.h" +#include "pvector.h" #include "random.h" #include "seq.h" #include "shash.h" @@ -68,9 +69,6 @@ VLOG_DEFINE_THIS_MODULE(dpif_netdev); -/* By default, choose a priority in the middle. */ -#define NETDEV_RULE_PRIORITY 0x8000 - #define FLOW_DUMP_MAX_BATCH 50 /* Use per thread recirc_depth to prevent recirculation loop. */ #define MAX_RECIRC_DEPTH 5 @@ -91,7 +89,9 @@ static struct vlog_rate_limit upcall_rl = VLOG_RATE_LIMIT_INIT(600, 600); /* Stores a miniflow with inline values */ struct netdev_flow_key { - struct miniflow flow; + uint32_t hash; /* hash function differs for different users. */ + uint32_t len; + struct miniflow mf; uint32_t buf[FLOW_MAX_PACKET_U32S - MINI_N_INLINE]; }; @@ -99,7 +99,7 @@ struct netdev_flow_key { * * 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 - * packet. It stores the 'cls_rule'(rule) that matches the miniflow. + * packet. It stores the 'dpcls_rule' (rule) that matches the miniflow. * * A cache entry holds a reference to its 'dp_netdev_flow'. * @@ -122,10 +122,8 @@ struct netdev_flow_key { #define EM_FLOW_HASH_SEGS 2 struct emc_entry { - uint32_t hash; - uint32_t mf_len; - struct netdev_flow_key mf; struct dp_netdev_flow *flow; + struct netdev_flow_key key; /* key.hash used for emc hash value. */ }; struct emc_cache { @@ -139,7 +137,31 @@ struct emc_cache { (CURRENT_ENTRY) = &(EMC)->entries[srch_hash__ & EM_FLOW_HASH_MASK], \ i__ < EM_FLOW_HASH_SEGS; \ i__++, srch_hash__ >>= EM_FLOW_HASH_SHIFT) + +/* Simple non-wildcarding single-priority classifier. */ + +struct dpcls { + struct cmap subtables_map; + struct pvector subtables; +}; +/* A rule to be inserted to the classifier. */ +struct dpcls_rule { + struct cmap_node cmap_node; /* Within struct dpcls_subtable 'rules'. */ + struct netdev_flow_key *mask; /* Subtable's mask. */ + struct netdev_flow_key flow; /* Matching key. */ + /* 'flow' must be the last field, additional space is allocated here. */ +}; + +static void dpcls_init(struct dpcls *); +static void dpcls_destroy(struct dpcls *); +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(const struct dpcls *cls, + const struct netdev_flow_key keys[], + struct dpcls_rule **rules, size_t cnt); + /* Datapath based on the network device interface from netdev.h. * * @@ -168,7 +190,7 @@ struct dp_netdev { * changes to 'cls' must be made while still holding the 'flow_mutex'. */ struct ovs_mutex flow_mutex; - struct classifier cls; + struct dpcls cls; struct cmap flow_table OVS_GUARDED; /* Flow table. */ /* Statistics. @@ -236,6 +258,7 @@ struct dp_netdev_port { char *type; /* Port type as requested by user. */ }; + /* A flow in dp_netdev's 'flow_table'. * * @@ -275,12 +298,10 @@ struct dp_netdev_port { */ struct dp_netdev_flow { bool dead; - /* Packet classification. */ - const struct cls_rule cr; /* In owning dp_netdev's 'cls'. */ /* Hash table index by unmasked flow. */ const struct cmap_node node; /* In owning dp_netdev's 'flow_table'. */ - const struct flow flow; /* The flow that created this entry. */ + const struct flow flow; /* Unmasked flow that created this entry. */ /* Number of references. * The classifier owns one reference. @@ -295,6 +316,10 @@ struct dp_netdev_flow { /* Actions. */ OVSRCU_TYPE(struct dp_netdev_actions *) actions; + + /* Packet classification. */ + struct dpcls_rule cr; /* In owning dp_netdev's 'cls'. */ + /* 'cr' must be the last member. */ }; static void dp_netdev_flow_unref(struct dp_netdev_flow *); @@ -410,10 +435,10 @@ emc_cache_init(struct emc_cache *flow_cache) for (i = 0; i < ARRAY_SIZE(flow_cache->entries); i++) { flow_cache->entries[i].flow = NULL; - flow_cache->entries[i].hash = 0; - flow_cache->entries[i].mf_len = 0; - miniflow_initialize(&flow_cache->entries[i].mf.flow, - flow_cache->entries[i].mf.buf); + flow_cache->entries[i].key.hash = 0; + flow_cache->entries[i].key.len = 0; + miniflow_initialize(&flow_cache->entries[i].key.mf, + flow_cache->entries[i].key.buf); } } @@ -550,7 +575,7 @@ create_dp_netdev(const char *name, const struct dpif_class *class, atomic_flag_clear(&dp->destroyed); ovs_mutex_init(&dp->flow_mutex); - classifier_init(&dp->cls, NULL); + dpcls_init(&dp->cls); cmap_init(&dp->flow_table); ovsthread_stats_init(&dp->stats); @@ -653,7 +678,7 @@ dp_netdev_free(struct dp_netdev *dp) } ovsthread_stats_destroy(&dp->stats); - classifier_destroy(&dp->cls); + dpcls_destroy(&dp->cls); cmap_destroy(&dp->flow_table); ovs_mutex_destroy(&dp->flow_mutex); seq_destroy(dp->port_seq); @@ -1092,7 +1117,6 @@ dp_netdev_flow_free(struct dp_netdev_flow *flow) } ovsthread_stats_destroy(&flow->stats); - cls_rule_destroy(CONST_CAST(struct cls_rule *, &flow->cr)); dp_netdev_actions_free(dp_netdev_flow_get_actions(flow)); free(flow); } @@ -1108,10 +1132,9 @@ static void dp_netdev_remove_flow(struct dp_netdev *dp, struct dp_netdev_flow *flow) OVS_REQUIRES(dp->flow_mutex) { - struct cls_rule *cr = CONST_CAST(struct cls_rule *, &flow->cr); struct cmap_node *node = CONST_CAST(struct cmap_node *, &flow->node); - classifier_remove(&dp->cls, cr); + dpcls_remove(&dp->cls, &flow->cr); cmap_remove(&dp->flow_table, node, flow_hash(&flow->flow, 0)); flow->dead = true; @@ -1216,7 +1239,7 @@ dpif_netdev_port_poll_wait(const struct dpif *dpif_) } static struct dp_netdev_flow * -dp_netdev_flow_cast(const struct cls_rule *cr) +dp_netdev_flow_cast(const struct dpcls_rule *cr) { return cr ? CONTAINER_OF(cr, struct dp_netdev_flow, cr) : NULL; } @@ -1244,38 +1267,144 @@ static bool dp_netdev_flow_ref(struct dp_netdev_flow *flow) */ BUILD_ASSERT_DECL(offsetof(struct miniflow, inline_values) == sizeof(uint64_t)); -BUILD_ASSERT_DECL(offsetof(struct netdev_flow_key, flow) == 0); - -static inline struct netdev_flow_key * -miniflow_to_netdev_flow_key(const struct miniflow *mf) -{ - return (struct netdev_flow_key *) CONST_CAST(struct miniflow *, mf); -} /* Given the number of bits set in the miniflow map, returns the size of the * netdev_flow key */ static inline uint32_t netdev_flow_key_size(uint32_t flow_u32s) { - return MINIFLOW_VALUES_SIZE(flow_u32s) - + offsetof(struct miniflow, inline_values); + return offsetof(struct netdev_flow_key, mf.inline_values) + + MINIFLOW_VALUES_SIZE(flow_u32s); } -/* Used to compare 'netdev_flow_key's (miniflows) in the exact match cache. */ static inline bool netdev_flow_key_equal(const struct netdev_flow_key *a, - const struct netdev_flow_key *b, - uint32_t size) + const struct netdev_flow_key *b) +{ + /* 'b's size and hash may be not set yet. */ + return !memcmp(a, b, a->len); +} + +/* Used to compare 'netdev_flow_key' in the exact match cache to a miniflow. + * The maps are compared bitwise, so both 'key->mf' 'mf' must have been + * generated by miniflow_extract. */ +static inline bool +netdev_flow_key_equal_mf(const struct netdev_flow_key *key, + const struct miniflow *mf) { - return !memcmp(a, b, size); + return !memcmp(&key->mf, mf, + key->len - offsetof(struct netdev_flow_key, mf)); } static inline void netdev_flow_key_clone(struct netdev_flow_key *dst, - const struct netdev_flow_key *src, - uint32_t size) + const struct netdev_flow_key *src) +{ + memcpy(dst, src, src->len); +} + +/* Slow. */ +static void +netdev_flow_key_from_flow(struct netdev_flow_key *dst, + const struct flow *src) +{ + struct ofpbuf packet; + uint64_t buf_stub[512 / 8]; + struct pkt_metadata md = pkt_metadata_from_flow(src); + + miniflow_initialize(&dst->mf, dst->buf); + + ofpbuf_use_stub(&packet, buf_stub, sizeof buf_stub); + flow_compose(&packet, src); + miniflow_extract(&packet, &md, &dst->mf); + ofpbuf_uninit(&packet); + + dst->len = netdev_flow_key_size(count_1bits(dst->mf.map)); + dst->hash = 0; /* Not computed yet. */ +} + +/* Initialize a netdev_flow_key 'mask' from 'match'. */ +static inline void +netdev_flow_mask_init(struct netdev_flow_key *mask, + const struct match *match) +{ + const uint32_t *mask_u32 = (const uint32_t *) &match->wc.masks; + uint32_t *dst = mask->mf.inline_values; + uint64_t map, mask_map = 0; + uint32_t hash = 0; + int n; + + /* Only check masks that make sense for the flow. */ + map = flow_wc_map(&match->flow); + + while (map) { + uint64_t rm1bit = rightmost_1bit(map); + int i = raw_ctz(map); + + if (mask_u32[i]) { + mask_map |= rm1bit; + *dst++ = mask_u32[i]; + hash = hash_add(hash, mask_u32[i]); + } + map -= rm1bit; + } + + mask->mf.values_inline = true; + mask->mf.map = mask_map; + + hash = hash_add(hash, mask_map); + hash = hash_add(hash, mask_map >> 32); + + n = dst - mask->mf.inline_values; + + mask->hash = hash_finish(hash, n * 4); + mask->len = netdev_flow_key_size(n); +} + +/* Initializes 'dst' as a copy of 'src' masked with 'mask'. */ +static inline void +netdev_flow_key_init_masked(struct netdev_flow_key *dst, + const struct flow *flow, + const struct netdev_flow_key *mask) { - memcpy(dst, src, size); + uint32_t *dst_u32 = dst->mf.inline_values; + const uint32_t *mask_u32 = mask->mf.inline_values; + uint32_t hash = 0; + uint32_t value; + + dst->len = mask->len; + dst->mf.values_inline = true; + dst->mf.map = mask->mf.map; + + FLOW_FOR_EACH_IN_MAP(value, flow, mask->mf.map) { + *dst_u32 = value & *mask_u32++; + hash = hash_add(hash, *dst_u32++); + } + dst->hash = hash_finish(hash, (dst_u32 - dst->mf.inline_values) * 4); +} + +/* Iterate through all netdev_flow_key u32 values specified by 'MAP' */ +#define NETDEV_FLOW_KEY_FOR_EACH_IN_MAP(VALUE, KEY, MAP) \ + for (struct mf_for_each_in_map_aux aux__ \ + = { (KEY)->mf.inline_values, (KEY)->mf.map, MAP }; \ + mf_get_next_in_map(&aux__, &(VALUE)); \ + ) + +/* Returns a hash value for the bits of 'key' where there are 1-bits in + * 'mask'. */ +static inline uint32_t +netdev_flow_key_hash_in_mask(const struct netdev_flow_key *key, + const struct netdev_flow_key *mask) +{ + const uint32_t *p = mask->mf.inline_values; + uint32_t hash = 0; + uint32_t key_u32; + + NETDEV_FLOW_KEY_FOR_EACH_IN_MAP(key_u32, key, mask->mf.map) { + hash = hash_add(hash, key_u32 & *p++); + } + + return hash_finish(hash, (p - mask->mf.inline_values) * 4); } static inline bool @@ -1295,7 +1424,7 @@ emc_clear_entry(struct emc_entry *ce) static inline void emc_change_entry(struct emc_entry *ce, struct dp_netdev_flow *flow, - const struct netdev_flow_key *mf, uint32_t hash) + const struct netdev_flow_key *key) { if (ce->flow != flow) { if (ce->flow) { @@ -1308,30 +1437,22 @@ emc_change_entry(struct emc_entry *ce, struct dp_netdev_flow *flow, ce->flow = NULL; } } - if (mf) { - uint32_t mf_len = netdev_flow_key_size(count_1bits(mf->flow.map)); - - netdev_flow_key_clone(&ce->mf, mf, mf_len); - ce->hash = hash; - ce->mf_len = mf_len; + if (key) { + netdev_flow_key_clone(&ce->key, key); } } static inline void -emc_insert(struct emc_cache *cache, const struct miniflow *mf, uint32_t hash, +emc_insert(struct emc_cache *cache, const struct netdev_flow_key *key, struct dp_netdev_flow *flow) { struct emc_entry *to_be_replaced = NULL; struct emc_entry *current_entry; - EMC_FOR_EACH_POS_WITH_HASH(cache, current_entry, hash) { - if (current_entry->hash == hash - && netdev_flow_key_equal(¤t_entry->mf, - miniflow_to_netdev_flow_key(mf), - current_entry->mf_len)) { - + EMC_FOR_EACH_POS_WITH_HASH(cache, current_entry, key->hash) { + if (netdev_flow_key_equal(¤t_entry->key, key)) { /* We found the entry with the 'mf' miniflow */ - emc_change_entry(current_entry, flow, NULL, 0); + emc_change_entry(current_entry, flow, NULL); return; } @@ -1340,29 +1461,27 @@ emc_insert(struct emc_cache *cache, const struct miniflow *mf, uint32_t hash, if (!to_be_replaced || (emc_entry_alive(to_be_replaced) && !emc_entry_alive(current_entry)) - || current_entry->hash < to_be_replaced->hash) { + || current_entry->key.hash < to_be_replaced->key.hash) { to_be_replaced = current_entry; } } /* We didn't find the miniflow in the cache. * The 'to_be_replaced' entry is where the new flow will be stored */ - emc_change_entry(to_be_replaced, flow, miniflow_to_netdev_flow_key(mf), - hash); + emc_change_entry(to_be_replaced, flow, key); } static inline struct dp_netdev_flow * -emc_lookup(struct emc_cache *cache, const struct miniflow *mf, uint32_t hash) +emc_lookup(struct emc_cache *cache, const struct netdev_flow_key *key) { struct emc_entry *current_entry; - EMC_FOR_EACH_POS_WITH_HASH(cache, current_entry, hash) { - if (current_entry->hash == hash && emc_entry_alive(current_entry) - && netdev_flow_key_equal(¤t_entry->mf, - miniflow_to_netdev_flow_key(mf), - current_entry->mf_len)) { + EMC_FOR_EACH_POS_WITH_HASH(cache, current_entry, key->hash) { + if (current_entry->key.hash == key->hash + && emc_entry_alive(current_entry) + && netdev_flow_key_equal_mf(¤t_entry->key, &key->mf)) { - /* We found the entry with the 'mf' miniflow */ + /* We found the entry with the 'key->mf' miniflow */ return current_entry->flow; } } @@ -1371,12 +1490,13 @@ emc_lookup(struct emc_cache *cache, const struct miniflow *mf, uint32_t hash) } static struct dp_netdev_flow * -dp_netdev_lookup_flow(const struct dp_netdev *dp, const struct miniflow *key) +dp_netdev_lookup_flow(const struct dp_netdev *dp, + const struct netdev_flow_key *key) { struct dp_netdev_flow *netdev_flow; - struct cls_rule *rule; + struct dpcls_rule *rule; - classifier_lookup_miniflow_batch(&dp->cls, &key, &rule, 1); + dpcls_lookup(&dp->cls, key, &rule, 1); netdev_flow = dp_netdev_flow_cast(rule); return netdev_flow; @@ -1422,7 +1542,7 @@ dp_netdev_flow_to_dpif_flow(const struct dp_netdev_flow *netdev_flow, struct flow_wildcards wc; struct dp_netdev_actions *actions; - minimask_expand(&netdev_flow->cr.match.mask, &wc); + miniflow_expand(&netdev_flow->cr.mask->mf, &wc.masks); odp_flow_key_from_mask(buffer, &wc.masks, &netdev_flow->flow, odp_to_u32(wc.masks.in_port.odp_port), SIZE_MAX, true); @@ -1552,36 +1672,41 @@ dpif_netdev_flow_get(const struct dpif *dpif, const struct dpif_flow_get *get) return error; } -static int +static struct dp_netdev_flow * dp_netdev_flow_add(struct dp_netdev *dp, struct match *match, const struct nlattr *actions, size_t actions_len) OVS_REQUIRES(dp->flow_mutex) { - struct dp_netdev_flow *netdev_flow; - - netdev_flow = xzalloc(sizeof *netdev_flow); - *CONST_CAST(struct flow *, &netdev_flow->flow) = match->flow; - - ovs_refcount_init(&netdev_flow->ref_cnt); + struct dp_netdev_flow *flow; + struct netdev_flow_key mask; - ovsthread_stats_init(&netdev_flow->stats); + netdev_flow_mask_init(&mask, match); + /* Make sure wc does not have metadata. */ + ovs_assert(!(mask.mf.map & (MINIFLOW_MAP(metadata) | MINIFLOW_MAP(regs)))); - ovsrcu_set(&netdev_flow->actions, - dp_netdev_actions_create(actions, actions_len)); + /* Do not allocate extra space. */ + flow = xmalloc(sizeof *flow - sizeof flow->cr.flow + mask.len); + flow->dead = false; + *CONST_CAST(struct flow *, &flow->flow) = match->flow; + ovs_refcount_init(&flow->ref_cnt); + ovsthread_stats_init(&flow->stats); + ovsrcu_set(&flow->actions, dp_netdev_actions_create(actions, actions_len)); - cls_rule_init(CONST_CAST(struct cls_rule *, &netdev_flow->cr), - match, NETDEV_RULE_PRIORITY); cmap_insert(&dp->flow_table, - CONST_CAST(struct cmap_node *, &netdev_flow->node), - flow_hash(&match->flow, 0)); - classifier_insert(&dp->cls, - CONST_CAST(struct cls_rule *, &netdev_flow->cr)); + CONST_CAST(struct cmap_node *, &flow->node), + flow_hash(&flow->flow, 0)); + netdev_flow_key_init_masked(&flow->cr.flow, &match->flow, &mask); + dpcls_insert(&dp->cls, &flow->cr, &mask); if (OVS_UNLIKELY(VLOG_IS_DBG_ENABLED())) { + struct match match; struct ds ds = DS_EMPTY_INITIALIZER; + match.flow = flow->flow; + miniflow_expand(&flow->cr.mask->mf, &match.wc.masks); + ds_put_cstr(&ds, "flow_add: "); - match_format(match, &ds, OFP_DEFAULT_PRIORITY); + match_format(&match, &ds, OFP_DEFAULT_PRIORITY); ds_put_cstr(&ds, ", actions:"); format_odp_actions(&ds, actions, actions_len); @@ -1590,7 +1715,7 @@ dp_netdev_flow_add(struct dp_netdev *dp, struct match *match, ds_destroy(&ds); } - return 0; + return flow; } static void @@ -1614,7 +1739,7 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put) { struct dp_netdev *dp = get_dp_netdev(dpif); struct dp_netdev_flow *netdev_flow; - struct miniflow miniflow; + struct netdev_flow_key key; struct match match; int error; @@ -1628,18 +1753,22 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put) if (error) { return error; } - miniflow_init(&miniflow, &match.flow); + + /* Must produce a netdev_flow_key for lookup. + * This interface is no longer performance critical, since it is not used + * for upcall processing any more. */ + netdev_flow_key_from_flow(&key, &match.flow); ovs_mutex_lock(&dp->flow_mutex); - netdev_flow = dp_netdev_lookup_flow(dp, &miniflow); + netdev_flow = dp_netdev_lookup_flow(dp, &key); if (!netdev_flow) { if (put->flags & DPIF_FP_CREATE) { if (cmap_count(&dp->flow_table) < MAX_FLOWS) { if (put->stats) { memset(put->stats, 0, sizeof *put->stats); } - error = dp_netdev_flow_add(dp, &match, put->actions, - put->actions_len); + dp_netdev_flow_add(dp, &match, put->actions, put->actions_len); + error = 0; } else { error = EFBIG; } @@ -1674,7 +1803,6 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put) } } ovs_mutex_unlock(&dp->flow_mutex); - miniflow_destroy(&miniflow); return error; } @@ -1817,7 +1945,7 @@ dpif_netdev_flow_dump_next(struct dpif_flow_dump_thread *thread_, struct flow_wildcards wc; struct ofpbuf buf; - minimask_expand(&netdev_flow->cr.match.mask, &wc); + miniflow_expand(&netdev_flow->cr.mask->mf, &wc.masks); /* Key. */ ofpbuf_use_stack(&buf, keybuf, sizeof *keybuf); @@ -2592,23 +2720,21 @@ emc_processing(struct dp_netdev_pmd_thread *pmd, struct dpif_packet **packets, size_t notfound_cnt = 0; n_batches = 0; - miniflow_initialize(&key.flow, key.buf); + miniflow_initialize(&key.mf, key.buf); for (i = 0; i < cnt; i++) { struct dp_netdev_flow *flow; - uint32_t hash; if (OVS_UNLIKELY(ofpbuf_size(&packets[i]->ofpbuf) < ETH_HEADER_LEN)) { dpif_packet_delete(packets[i]); continue; } - miniflow_extract(&packets[i]->ofpbuf, &packets[i]->md, &key.flow); + miniflow_extract(&packets[i]->ofpbuf, &packets[i]->md, &key.mf); + key.len = 0; /* Not computed yet. */ + key.hash = dpif_netdev_packet_get_dp_hash(packets[i], &key.mf); - hash = dpif_netdev_packet_get_dp_hash(packets[i], &key.flow); - - flow = emc_lookup(flow_cache, &key.flow, hash); - if (OVS_UNLIKELY(!dp_netdev_queue_batches(packets[i], - flow, &key.flow, + flow = emc_lookup(flow_cache, &key); + if (OVS_UNLIKELY(!dp_netdev_queue_batches(packets[i], flow, &key.mf, batches, &n_batches, ARRAY_SIZE(batches)))) { if (i != notfound_cnt) { @@ -2638,51 +2764,51 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, enum { PKT_ARRAY_SIZE = NETDEV_MAX_RX_BATCH }; #endif struct packet_batch batches[PKT_ARRAY_SIZE]; - const struct miniflow *mfs[PKT_ARRAY_SIZE]; /* May NOT be NULL. */ - struct cls_rule *rules[PKT_ARRAY_SIZE]; + struct dpcls_rule *rules[PKT_ARRAY_SIZE]; struct dp_netdev *dp = pmd->dp; struct emc_cache *flow_cache = &pmd->flow_cache; size_t n_batches, i; bool any_miss; for (i = 0; i < cnt; i++) { - mfs[i] = &keys[i].flow; /* No bad packets! */ + /* Key length is needed in all the cases, hash computed on demand. */ + keys[i].len = netdev_flow_key_size(count_1bits(keys[i].mf.map)); } - any_miss = !classifier_lookup_miniflow_batch(&dp->cls, mfs, rules, cnt); + any_miss = !dpcls_lookup(&dp->cls, keys, rules, cnt); 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; - struct match match; ofpbuf_use_stub(&actions, actions_stub, sizeof actions_stub); ofpbuf_use_stub(&put_actions, slow_stub, sizeof slow_stub); for (i = 0; i < cnt; i++) { - const struct dp_netdev_flow *netdev_flow; + struct dp_netdev_flow *netdev_flow; struct ofpbuf *add_actions; + struct match match; int error; - if (OVS_LIKELY(rules[i] || !mfs[i])) { + if (OVS_LIKELY(rules[i])) { continue; } /* It's possible that an earlier slow path execution installed - * the rule this flow needs. In this case, it's a lot cheaper + * 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_lookup_flow(dp, mfs[i]); + netdev_flow = dp_netdev_lookup_flow(dp, &keys[i]); if (netdev_flow) { - rules[i] = CONST_CAST(struct cls_rule *, &netdev_flow->cr); + rules[i] = &netdev_flow->cr; continue; } - miniflow_expand(mfs[i], &match.flow); + miniflow_expand(&keys[i].mf, &match.flow); ofpbuf_clear(&actions); ofpbuf_clear(&put_actions); error = dp_netdev_upcall(dp, packets[i], &match.flow, &match.wc, - DPIF_UC_MISS, NULL, &actions, - &put_actions); + DPIF_UC_MISS, NULL, &actions, + &put_actions); if (OVS_UNLIKELY(error && error != ENOSPC)) { continue; } @@ -2698,19 +2824,26 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, ? &put_actions : &actions; - ovs_mutex_lock(&dp->flow_mutex); - /* XXX: There's a brief race where this flow could have already - * been installed since we last did the flow lookup. This could be - * solved by moving the mutex lock outside the loop, but that's an - * awful long time to be locking everyone out of making flow - * installs. If we move to a per-core classifier, it would be - * reasonable. */ - if (OVS_LIKELY(error != ENOSPC) - && !dp_netdev_lookup_flow(dp, mfs[i])) { - dp_netdev_flow_add(dp, &match, ofpbuf_data(add_actions), - ofpbuf_size(add_actions)); + if (OVS_LIKELY(error != ENOSPC)) { + /* XXX: There's a race window where a flow covering this packet + * could have already been installed since we last did the flow + * lookup before upcall. This could be solved by moving the + * mutex lock outside the loop, but that's an awful long time + * to be locking everyone out of making flow installs. If we + * move to a per-core classifier, it would be reasonable. */ + ovs_mutex_lock(&dp->flow_mutex); + netdev_flow = dp_netdev_lookup_flow(dp, &keys[i]); + if (OVS_LIKELY(!netdev_flow)) { + netdev_flow = dp_netdev_flow_add(dp, &match, + ofpbuf_data(add_actions), + ofpbuf_size(add_actions)); + } + ovs_mutex_unlock(&dp->flow_mutex); + + /* EMC uses different hash. */ + keys[i].hash = dpif_packet_get_dp_hash(packets[i]); + emc_insert(flow_cache, &keys[i], netdev_flow); } - ovs_mutex_unlock(&dp->flow_mutex); } ofpbuf_uninit(&actions); @@ -2720,7 +2853,7 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, int dropped_cnt = 0; for (i = 0; i < cnt; i++) { - if (OVS_UNLIKELY(!rules[i] && mfs[i])) { + if (OVS_UNLIKELY(!rules[i])) { dpif_packet_delete(packets[i]); dropped_cnt++; } @@ -2734,15 +2867,16 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, struct dpif_packet *packet = packets[i]; struct dp_netdev_flow *flow; - if (OVS_UNLIKELY(!rules[i] || !mfs[i])) { + if (OVS_UNLIKELY(!rules[i])) { continue; } flow = dp_netdev_flow_cast(rules[i]); - emc_insert(flow_cache, mfs[i], dpif_packet_get_dp_hash(packet), - flow); - dp_netdev_queue_batches(packet, flow, mfs[i], batches, &n_batches, - ARRAY_SIZE(batches)); + + keys[i].hash = dpif_packet_get_dp_hash(packet); + emc_insert(flow_cache, &keys[i], flow); + dp_netdev_queue_batches(packet, flow, &keys[i].mf, batches, + &n_batches, ARRAY_SIZE(batches)); } for (i = 0; i < n_batches; i++) { @@ -3087,3 +3221,209 @@ dpif_dummy_register(bool override) unixctl_command_register("dpif-dummy/delete-port", "dp port", 2, 2, dpif_dummy_delete_port, NULL); } + +/* Datapath Classifier. */ + +/* A set of rules that all have the same fields wildcarded. */ +struct dpcls_subtable { + /* The fields are only used by writers. */ + struct cmap_node cmap_node OVS_GUARDED; /* Within dpcls 'subtables_map'. */ + + /* These fields are accessed by readers. */ + struct cmap rules; /* Contains "struct dpcls_rule"s. */ + struct netdev_flow_key mask; /* Wildcards for fields (const). */ + /* 'mask' must be the last field, additional space is allocated here. */ +}; + +/* Initializes 'cls' as a classifier that initially contains no classification + * rules. */ +static void +dpcls_init(struct dpcls *cls) +{ + cmap_init(&cls->subtables_map); + pvector_init(&cls->subtables); +} + +static void +dpcls_destroy_subtable(struct dpcls *cls, struct dpcls_subtable *subtable) +{ + pvector_remove(&cls->subtables, subtable); + cmap_remove(&cls->subtables_map, &subtable->cmap_node, + subtable->mask.hash); + cmap_destroy(&subtable->rules); + ovsrcu_postpone(free, subtable); +} + +/* Destroys 'cls'. Rules within 'cls', if any, are not freed; this is the + * caller's responsibility. + * May only be called after all the readers have been terminated. */ +static void +dpcls_destroy(struct dpcls *cls) +{ + if (cls) { + struct dpcls_subtable *subtable; + + CMAP_FOR_EACH (subtable, cmap_node, &cls->subtables_map) { + dpcls_destroy_subtable(cls, subtable); + } + cmap_destroy(&cls->subtables_map); + pvector_destroy(&cls->subtables); + } +} + +static struct dpcls_subtable * +dpcls_create_subtable(struct dpcls *cls, const struct netdev_flow_key *mask) +{ + struct dpcls_subtable *subtable; + + /* Need to add one. */ + subtable = xmalloc(sizeof *subtable - sizeof subtable->mask + mask->len); + cmap_init(&subtable->rules); + netdev_flow_key_clone(&subtable->mask, mask); + cmap_insert(&cls->subtables_map, &subtable->cmap_node, mask->hash); + pvector_insert(&cls->subtables, subtable, 0); + + return subtable; +} + +static inline struct dpcls_subtable * +dpcls_find_subtable(struct dpcls *cls, const struct netdev_flow_key *mask) +{ + struct dpcls_subtable *subtable; + + CMAP_FOR_EACH_WITH_HASH (subtable, cmap_node, mask->hash, + &cls->subtables_map) { + if (netdev_flow_key_equal(&subtable->mask, mask)) { + return subtable; + } + } + return dpcls_create_subtable(cls, mask); +} + +/* Insert 'rule' into 'cls'. */ +static void +dpcls_insert(struct dpcls *cls, struct dpcls_rule *rule, + const struct netdev_flow_key *mask) +{ + struct dpcls_subtable *subtable = dpcls_find_subtable(cls, mask); + + rule->mask = &subtable->mask; + cmap_insert(&subtable->rules, &rule->cmap_node, rule->flow.hash); +} + +/* Removes 'rule' from 'cls', also destructing the 'rule'. */ +static void +dpcls_remove(struct dpcls *cls, struct dpcls_rule *rule) +{ + struct dpcls_subtable *subtable; + + ovs_assert(rule->mask); + + INIT_CONTAINER(subtable, rule->mask, mask); + + if (cmap_remove(&subtable->rules, &rule->cmap_node, rule->flow.hash) + == 0) { + dpcls_destroy_subtable(cls, subtable); + } +} + +/* Returns true if 'target' satisifies 'key' in 'mask', that is, if each 1-bit + * in 'mask' the values in 'key' and 'target' are the same. + * + * Note: 'key' and 'mask' have the same mask, and 'key' is already masked. */ +static inline bool +dpcls_rule_matches_key(const struct dpcls_rule *rule, + const struct netdev_flow_key *target) +{ + const uint32_t *keyp = rule->flow.mf.inline_values; + const uint32_t *maskp = rule->mask->mf.inline_values; + uint32_t target_u32; + + NETDEV_FLOW_KEY_FOR_EACH_IN_MAP(target_u32, target, rule->flow.mf.map) { + if (OVS_UNLIKELY((target_u32 & *maskp++) != *keyp++)) { + return false; + } + } + return true; +} + +/* For each miniflow in 'flows' performs a classifier lookup writing the result + * into the corresponding slot in 'rules'. If a particular entry in 'flows' is + * NULL it is skipped. + * + * This function is optimized for use in the userspace datapath and therefore + * does not implement a lot of features available in the standard + * classifier_lookup() function. Specifically, it does not implement + * priorities, instead returning any rule which matches the flow. + * + * Returns true if all flows found a corresponding rule. */ +static bool +dpcls_lookup(const struct dpcls *cls, const struct netdev_flow_key keys[], + struct dpcls_rule **rules, const size_t cnt) +{ + /* The batch size 16 was experimentally found faster than 8 or 32. */ + typedef uint16_t map_type; +#define MAP_BITS (sizeof(map_type) * CHAR_BIT) + +#if !defined(__CHECKER__) && !defined(_WIN32) + const int N_MAPS = DIV_ROUND_UP(cnt, MAP_BITS); +#else + enum { N_MAPS = DIV_ROUND_UP(NETDEV_MAX_RX_BATCH, MAP_BITS) }; +#endif + map_type maps[N_MAPS]; + struct dpcls_subtable *subtable; + + memset(maps, 0xff, sizeof maps); + if (cnt % MAP_BITS) { + maps[N_MAPS - 1] >>= MAP_BITS - cnt % MAP_BITS; /* Clear extra bits. */ + } + memset(rules, 0, cnt * sizeof *rules); + + PVECTOR_FOR_EACH (subtable, &cls->subtables) { + const struct netdev_flow_key *mkeys = keys; + struct dpcls_rule **mrules = rules; + map_type remains = 0; + int m; + + BUILD_ASSERT_DECL(sizeof remains == sizeof *maps); + + for (m = 0; m < N_MAPS; m++, mkeys += MAP_BITS, mrules += MAP_BITS) { + uint32_t hashes[MAP_BITS]; + const struct cmap_node *nodes[MAP_BITS]; + unsigned long map = maps[m]; + int i; + + if (!map) { + continue; /* Skip empty maps. */ + } + + /* Compute hashes for the remaining keys. */ + ULONG_FOR_EACH_1(i, map) { + hashes[i] = netdev_flow_key_hash_in_mask(&mkeys[i], + &subtable->mask); + } + /* Lookup. */ + map = cmap_find_batch(&subtable->rules, map, hashes, nodes); + /* Check results. */ + ULONG_FOR_EACH_1(i, map) { + struct dpcls_rule *rule; + + CMAP_NODE_FOR_EACH (rule, cmap_node, nodes[i]) { + if (OVS_LIKELY(dpcls_rule_matches_key(rule, &mkeys[i]))) { + mrules[i] = rule; + goto next; + } + } + ULONG_SET0(map, i); /* Did not match. */ + next: + ; /* Keep Sparse happy. */ + } + maps[m] &= ~map; /* Clear the found rules. */ + remains |= maps[m]; + } + if (!remains) { + return true; /* All found. */ + } + } + return false; /* Some misses. */ +} diff --git a/lib/flow.c b/lib/flow.c index 7fb53de0d..3935ea62b 100644 --- a/lib/flow.c +++ b/lib/flow.c @@ -814,6 +814,9 @@ void flow_wildcards_init_for_packet(struct flow_wildcards *wc, { memset(&wc->masks, 0x0, sizeof wc->masks); + /* Update this function whenever struct flow changes. */ + BUILD_ASSERT_DECL(FLOW_WC_SEQ == 27); + if (flow->tunnel.ip_dst) { if (flow->tunnel.flags & FLOW_TNL_F_KEY) { WC_MASK_FIELD(wc, tunnel.tun_id); @@ -898,11 +901,69 @@ void flow_wildcards_init_for_packet(struct flow_wildcards *wc, } } +/* Return a map of possible fields for a packet of the same type as 'flow'. + * Including extra bits in the returned mask is not wrong, it is just less + * optimal. + * + * This is a less precise version of flow_wildcards_init_for_packet() above. */ +uint64_t +flow_wc_map(const struct flow *flow) +{ + /* Update this function whenever struct flow changes. */ + BUILD_ASSERT_DECL(FLOW_WC_SEQ == 27); + + uint64_t map = (flow->tunnel.ip_dst) ? MINIFLOW_MAP(tunnel) : 0; + + /* Metadata fields that can appear on packet input. */ + map |= MINIFLOW_MAP(skb_priority) | MINIFLOW_MAP(pkt_mark) + | MINIFLOW_MAP(recirc_id) | MINIFLOW_MAP(dp_hash) + | MINIFLOW_MAP(in_port) + | MINIFLOW_MAP(dl_dst) | MINIFLOW_MAP(dl_src) + | MINIFLOW_MAP(dl_type) | MINIFLOW_MAP(vlan_tci); + + /* Ethertype-dependent fields. */ + if (OVS_LIKELY(flow->dl_type == htons(ETH_TYPE_IP))) { + map |= MINIFLOW_MAP(nw_src) | MINIFLOW_MAP(nw_dst) + | MINIFLOW_MAP(nw_proto) | MINIFLOW_MAP(nw_frag) + | MINIFLOW_MAP(nw_tos) | MINIFLOW_MAP(nw_ttl); + if (OVS_UNLIKELY(flow->nw_proto == IPPROTO_IGMP)) { + map |= MINIFLOW_MAP(igmp_group_ip4); + } else { + map |= MINIFLOW_MAP(tcp_flags) + | MINIFLOW_MAP(tp_src) | MINIFLOW_MAP(tp_dst); + } + } else if (flow->dl_type == htons(ETH_TYPE_IPV6)) { + map |= MINIFLOW_MAP(ipv6_src) | MINIFLOW_MAP(ipv6_dst) + | MINIFLOW_MAP(ipv6_label) + | MINIFLOW_MAP(nw_proto) | MINIFLOW_MAP(nw_frag) + | MINIFLOW_MAP(nw_tos) | MINIFLOW_MAP(nw_ttl); + if (OVS_UNLIKELY(flow->nw_proto == IPPROTO_ICMPV6)) { + map |= MINIFLOW_MAP(nd_target) + | MINIFLOW_MAP(arp_sha) | MINIFLOW_MAP(arp_tha); + } else { + map |= MINIFLOW_MAP(tcp_flags) + | MINIFLOW_MAP(tp_src) | MINIFLOW_MAP(tp_dst); + } + } else if (eth_type_mpls(flow->dl_type)) { + map |= MINIFLOW_MAP(mpls_lse); + } else if (flow->dl_type == htons(ETH_TYPE_ARP) || + flow->dl_type == htons(ETH_TYPE_RARP)) { + map |= MINIFLOW_MAP(nw_src) | MINIFLOW_MAP(nw_dst) + | MINIFLOW_MAP(nw_proto) + | MINIFLOW_MAP(arp_sha) | MINIFLOW_MAP(arp_tha); + } + + return map; +} + /* Clear the metadata and register wildcard masks. They are not packet * header fields. */ void flow_wildcards_clear_non_packet_fields(struct flow_wildcards *wc) { + /* Update this function whenever struct flow changes. */ + BUILD_ASSERT_DECL(FLOW_WC_SEQ == 27); + memset(&wc->masks.metadata, 0, sizeof wc->masks.metadata); memset(&wc->masks.regs, 0, sizeof wc->masks.regs); } diff --git a/lib/flow.h b/lib/flow.h index 2b053da47..09988f6cc 100644 --- a/lib/flow.h +++ b/lib/flow.h @@ -94,7 +94,7 @@ const char *flow_tun_flag_to_string(uint32_t flags); * reflected in miniflow_extract()! */ struct flow { - /* L1 */ + /* Metadata */ struct flow_tnl tunnel; /* Encapsulating tunnel parameters. */ ovs_be64 metadata; /* OpenFlow Metadata. */ uint32_t regs[FLOW_N_REGS]; /* Registers. */ @@ -327,6 +327,7 @@ void flow_wildcards_init_catchall(struct flow_wildcards *); void flow_wildcards_init_for_packet(struct flow_wildcards *, const struct flow *); +uint64_t flow_wc_map(const struct flow *); void flow_wildcards_clear_non_packet_fields(struct flow_wildcards *); -- 2.39.5