X-Git-Url: http://git.cascardo.eti.br/?a=blobdiff_plain;f=lib%2Fdpif-netdev.c;h=3dc0a6273cda45b9eec510973aff1e6c60a00496;hb=07659514c3c1e8998a4935a998b627d716c559f9;hp=892a024ef3619a60dd9b8696770199a074ddd014;hpb=11bfdaddf2035efe054b6a098dc3b8aeeb65eeb6;p=cascardo%2Fovs.git diff --git a/lib/dpif-netdev.c b/lib/dpif-netdev.c index 892a024ef..3dc0a6273 100644 --- a/lib/dpif-netdev.c +++ b/lib/dpif-netdev.c @@ -31,6 +31,7 @@ #include #include +#include "bitmap.h" #include "cmap.h" #include "csum.h" #include "dp-packet.h" @@ -41,10 +42,10 @@ #include "fat-rwlock.h" #include "flow.h" #include "cmap.h" +#include "coverage.h" #include "latch.h" #include "list.h" #include "match.h" -#include "meta-flow.h" #include "netdev.h" #include "netdev-dpdk.h" #include "netdev-vport.h" @@ -64,6 +65,7 @@ #include "sset.h" #include "timeval.h" #include "tnl-arp-cache.h" +#include "tnl-ports.h" #include "unixctl.h" #include "util.h" #include "openvswitch/vlog.h" @@ -87,13 +89,18 @@ static struct shash dp_netdevs OVS_GUARDED_BY(dp_netdev_mutex) static struct vlog_rate_limit upcall_rl = VLOG_RATE_LIMIT_INIT(600, 600); +static struct odp_support dp_netdev_support = { + .max_mpls_depth = SIZE_MAX, + .recirc = true, +}; + /* Stores a miniflow with inline values */ struct netdev_flow_key { uint32_t hash; /* Hash function differs for different users. */ uint32_t len; /* Length of the following miniflow (incl. map). */ struct miniflow mf; - uint64_t buf[FLOW_MAX_PACKET_U64S - MINI_N_INLINE]; + uint64_t buf[FLOW_MAX_PACKET_U64S]; }; /* Exact match cache for frequently used flows @@ -117,7 +124,7 @@ struct netdev_flow_key { * If dp_netdev_input is not called from a pmd thread, a mutex is used. */ -#define EM_FLOW_HASH_SHIFT 10 +#define EM_FLOW_HASH_SHIFT 13 #define EM_FLOW_HASH_ENTRIES (1u << EM_FLOW_HASH_SHIFT) #define EM_FLOW_HASH_MASK (EM_FLOW_HASH_ENTRIES - 1) #define EM_FLOW_HASH_SEGS 2 @@ -198,6 +205,11 @@ struct dp_netdev { upcall_callback *upcall_cb; /* Callback function for executing upcalls. */ void *upcall_aux; + /* Callback function for notifying the purging of dp flows (during + * reseting pmd deletion). */ + dp_purge_callback *dp_purge_cb; + void *dp_purge_aux; + /* Stores all 'struct dp_netdev_pmd_thread's. */ struct cmap poll_threads; @@ -235,9 +247,9 @@ enum pmd_cycles_counter_type { /* A port in a netdev-based datapath. */ struct dp_netdev_port { - struct cmap_node node; /* Node in dp_netdev's 'ports'. */ odp_port_t port_no; struct netdev *netdev; + struct cmap_node node; /* Node in dp_netdev's 'ports'. */ struct netdev_saved_flags *sf; struct netdev_rxq **rxq; struct ovs_refcount ref_cnt; @@ -290,14 +302,12 @@ struct dp_netdev_flow_stats { * requires synchronization, as noted in more detail below. */ struct dp_netdev_flow { - bool dead; - + const struct flow flow; /* Unmasked flow that created this entry. */ /* Hash table index by unmasked flow. */ const struct cmap_node node; /* In owning dp_netdev_pmd_thread's */ /* 'flow_table'. */ const ovs_u128 ufid; /* Unique flow identifier. */ - const struct flow flow; /* Unmasked flow that created this entry. */ - const int pmd_id; /* The 'core_id' of pmd thread owning this */ + const unsigned pmd_id; /* The 'core_id' of pmd thread owning this */ /* flow. */ /* Number of references. @@ -306,12 +316,20 @@ struct dp_netdev_flow { * reference. */ struct ovs_refcount ref_cnt; + bool dead; + /* Statistics. */ struct dp_netdev_flow_stats stats; /* Actions. */ OVSRCU_TYPE(struct dp_netdev_actions *) actions; + /* While processing a group of input packets, the datapath uses the next + * member to store a pointer to the output batch for the flow. It is + * reset after the batch has been sent out (See dp_netdev_queue_batches(), + * packet_batch_init() and packet_batch_execute()). */ + struct packet_batch *batch; + /* Packet classification. */ struct dpcls_rule cr; /* In owning dp_netdev's 'cls'. */ /* 'cr' must be the last member. */ @@ -407,8 +425,10 @@ struct dp_netdev_pmd_thread { pthread_t thread; int index; /* Idx of this pmd thread among pmd*/ /* threads on same numa node. */ - int core_id; /* CPU core id of this pmd thread. */ + unsigned core_id; /* CPU core id of this pmd thread. */ int numa_id; /* numa node id of this pmd thread. */ + int tx_qid; /* Queue id used by this pmd thread to + * send packets on all netdevs */ /* Only a pmd thread can write on its own 'cycles' and 'stats'. * The main thread keeps 'stats_zero' and 'cycles_zero' as base @@ -452,11 +472,11 @@ static void dp_netdev_disable_upcall(struct dp_netdev *); void dp_netdev_pmd_reload_done(struct dp_netdev_pmd_thread *pmd); static void dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd, struct dp_netdev *dp, int index, - int core_id, int numa_id); + unsigned core_id, int numa_id); static void dp_netdev_destroy_pmd(struct dp_netdev_pmd_thread *pmd); static void dp_netdev_set_nonpmd(struct dp_netdev *dp); static struct dp_netdev_pmd_thread *dp_netdev_get_pmd(struct dp_netdev *dp, - int core_id); + unsigned core_id); static struct dp_netdev_pmd_thread * dp_netdev_pmd_get_next(struct dp_netdev *dp, struct cmap_position *pos); static void dp_netdev_destroy_all_pmds(struct dp_netdev *dp); @@ -475,16 +495,12 @@ emc_cache_init(struct emc_cache *flow_cache) { int i; - BUILD_ASSERT(offsetof(struct miniflow, inline_values) == sizeof(uint64_t)); - flow_cache->sweep_idx = 0; for (i = 0; i < ARRAY_SIZE(flow_cache->entries); i++) { flow_cache->entries[i].flow = NULL; flow_cache->entries[i].key.hash = 0; - flow_cache->entries[i].key.len - = offsetof(struct miniflow, inline_values); - miniflow_initialize(&flow_cache->entries[i].key.mf, - flow_cache->entries[i].key.buf); + flow_cache->entries[i].key.len = sizeof(struct miniflow); + flowmap_init(&flow_cache->entries[i].key.mf.map); } } @@ -511,10 +527,17 @@ emc_cache_slow_sweep(struct emc_cache *flow_cache) flow_cache->sweep_idx = (flow_cache->sweep_idx + 1) & EM_FLOW_HASH_MASK; } +/* Returns true if 'dpif' is a netdev or dummy dpif, false otherwise. */ +bool +dpif_is_netdev(const struct dpif *dpif) +{ + return dpif->dpif_class->open == dpif_netdev_open; +} + static struct dpif_netdev * dpif_netdev_cast(const struct dpif *dpif) { - ovs_assert(dpif->dpif_class->open == dpif_netdev_open); + ovs_assert(dpif_is_netdev(dpif)); return CONTAINER_OF(dpif, struct dpif_netdev, dpif); } @@ -574,8 +597,8 @@ pmd_info_show_stats(struct ds *reply, if (pmd->numa_id != OVS_NUMA_UNSPEC) { ds_put_format(reply, " numa_id %d", pmd->numa_id); } - if (pmd->core_id != OVS_CORE_UNSPEC) { - ds_put_format(reply, " core_id %d", pmd->core_id); + if (pmd->core_id != OVS_CORE_UNSPEC && pmd->core_id != NON_PMD_CORE_ID) { + ds_put_format(reply, " core_id %u", pmd->core_id); } ds_put_cstr(reply, ":\n"); @@ -823,8 +846,6 @@ create_dp_netdev(const char *name, const struct dpif_class *class, ovs_mutex_init_recursive(&dp->non_pmd_mutex); ovsthread_key_create(&dp->per_pmd_key, NULL); - /* Reserves the core NON_PMD_CORE_ID for all non-pmd threads. */ - ovs_numa_try_pin_core_specific(NON_PMD_CORE_ID); dp_netdev_set_nonpmd(dp); dp->n_dpdk_rxqs = NR_QUEUE; @@ -1063,8 +1084,9 @@ do_add_port(struct dp_netdev *dp, const char *devname, const char *type, return ENOENT; } /* There can only be ovs_numa_get_n_cores() pmd threads, - * so creates a txq for each. */ - error = netdev_set_multiq(netdev, n_cores, dp->n_dpdk_rxqs); + * so creates a txq for each, and one extra for the non + * pmd threads. */ + error = netdev_set_multiq(netdev, n_cores + 1, dp->n_dpdk_rxqs); if (error && (error != EOPNOTSUPP)) { VLOG_ERR("%s, cannot set multiq", devname); return errno; @@ -1373,6 +1395,8 @@ dp_netdev_pmd_remove_flow(struct dp_netdev_pmd_thread *pmd, struct cmap_node *node = CONST_CAST(struct cmap_node *, &flow->node); dpcls_remove(&pmd->cls, &flow->cr); + flow->cr.mask = NULL; /* Accessing rule's mask after this is not safe. */ + cmap_remove(&pmd->flow_table, node, dp_netdev_flow_hash(&flow->ufid)); flow->dead = true; @@ -1501,22 +1525,14 @@ static bool dp_netdev_flow_ref(struct dp_netdev_flow *flow) * miniflow_extract(), if the map is different the miniflow is different. * Therefore we can be faster by comparing the map and the miniflow in a * single memcmp(). - * _ netdev_flow_key's miniflow has always inline values. - * - These functions can be inlined by the compiler. - * - * The following assertions make sure that what we're doing with miniflow is - * safe - */ -BUILD_ASSERT_DECL(offsetof(struct miniflow, inline_values) - == sizeof(uint64_t)); + * - These functions can be inlined by the compiler. */ -/* Given the number of bits set in the miniflow map, returns the size of the +/* Given the number of bits set in miniflow's maps, returns the size of the * 'netdev_flow_key.mf' */ -static inline uint32_t -netdev_flow_key_size(uint32_t flow_u32s) +static inline size_t +netdev_flow_key_size(size_t flow_u64s) { - return offsetof(struct miniflow, inline_values) + - MINIFLOW_VALUES_SIZE(flow_u32s); + return sizeof(struct miniflow) + MINIFLOW_VALUES_SIZE(flow_u64s); } static inline bool @@ -1553,15 +1569,13 @@ netdev_flow_key_from_flow(struct netdev_flow_key *dst, struct dp_packet packet; uint64_t buf_stub[512 / 8]; - miniflow_initialize(&dst->mf, dst->buf); - dp_packet_use_stub(&packet, buf_stub, sizeof buf_stub); pkt_metadata_from_flow(&packet.md, src); flow_compose(&packet, src); miniflow_extract(&packet, &dst->mf); dp_packet_uninit(&packet); - dst->len = netdev_flow_key_size(count_1bits(dst->mf.map)); + dst->len = netdev_flow_key_size(miniflow_n_values(&dst->mf)); dst->hash = 0; /* Not computed yet. */ } @@ -1570,66 +1584,62 @@ static inline void netdev_flow_mask_init(struct netdev_flow_key *mask, const struct match *match) { - const uint64_t *mask_u64 = (const uint64_t *) &match->wc.masks; - uint64_t *dst = mask->mf.inline_values; - uint64_t map, mask_map = 0; + uint64_t *dst = miniflow_values(&mask->mf); + struct flowmap fmap; uint32_t hash = 0; - int n; + size_t idx; /* Only check masks that make sense for the flow. */ - map = flow_wc_map(&match->flow); + flow_wc_map(&match->flow, &fmap); + flowmap_init(&mask->mf.map); - while (map) { - uint64_t rm1bit = rightmost_1bit(map); - int i = raw_ctz(map); + FLOWMAP_FOR_EACH_INDEX(idx, fmap) { + uint64_t mask_u64 = flow_u64_value(&match->wc.masks, idx); - if (mask_u64[i]) { - mask_map |= rm1bit; - *dst++ = mask_u64[i]; - hash = hash_add64(hash, mask_u64[i]); + if (mask_u64) { + flowmap_set(&mask->mf.map, idx, 1); + *dst++ = mask_u64; + hash = hash_add64(hash, mask_u64); } - map -= rm1bit; } - mask->mf.values_inline = true; - mask->mf.map = mask_map; + map_t map; - hash = hash_add64(hash, mask_map); + FLOWMAP_FOR_EACH_MAP (map, mask->mf.map) { + hash = hash_add64(hash, map); + } - n = dst - mask->mf.inline_values; + size_t n = dst - miniflow_get_values(&mask->mf); mask->hash = hash_finish(hash, n * 8); mask->len = netdev_flow_key_size(n); } -/* Initializes 'dst' as a copy of 'src' masked with 'mask'. */ +/* Initializes 'dst' as a copy of 'flow' 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) { - uint64_t *dst_u64 = dst->mf.inline_values; - const uint64_t *mask_u64 = mask->mf.inline_values; + uint64_t *dst_u64 = miniflow_values(&dst->mf); + const uint64_t *mask_u64 = miniflow_get_values(&mask->mf); uint32_t hash = 0; uint64_t value; dst->len = mask->len; - dst->mf.values_inline = true; - dst->mf.map = mask->mf.map; + dst->mf = mask->mf; /* Copy maps. */ - FLOW_FOR_EACH_IN_MAP(value, flow, mask->mf.map) { + FLOW_FOR_EACH_IN_MAPS(value, flow, mask->mf.map) { *dst_u64 = value & *mask_u64++; hash = hash_add64(hash, *dst_u64++); } - dst->hash = hash_finish(hash, (dst_u64 - dst->mf.inline_values) * 8); + dst->hash = hash_finish(hash, + (dst_u64 - miniflow_get_values(&dst->mf)) * 8); } -/* Iterate through all netdev_flow_key u64 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)); \ - ) +/* Iterate through netdev_flow_key TNL u64 values specified by 'FLOWMAP'. */ +#define NETDEV_FLOW_KEY_FOR_EACH_IN_FLOWMAP(VALUE, KEY, FLOWMAP) \ + MINIFLOW_FOR_EACH_IN_FLOWMAP(VALUE, &(KEY)->mf, FLOWMAP) /* Returns a hash value for the bits of 'key' where there are 1-bits in * 'mask'. */ @@ -1637,15 +1647,15 @@ static inline uint32_t netdev_flow_key_hash_in_mask(const struct netdev_flow_key *key, const struct netdev_flow_key *mask) { - const uint64_t *p = mask->mf.inline_values; + const uint64_t *p = miniflow_get_values(&mask->mf); uint32_t hash = 0; - uint64_t key_u64; + uint64_t value; - NETDEV_FLOW_KEY_FOR_EACH_IN_MAP(key_u64, key, mask->mf.map) { - hash = hash_add64(hash, key_u64 & *p++); + NETDEV_FLOW_KEY_FOR_EACH_IN_FLOWMAP(value, key, mask->mf.map) { + hash = hash_add64(hash, value & *p++); } - return hash_finish(hash, (p - mask->mf.inline_values) * 8); + return hash_finish(hash, (p - miniflow_get_values(&mask->mf)) * 8); } static inline bool @@ -1762,7 +1772,7 @@ dp_netdev_pmd_find_flow(const struct dp_netdev_pmd_thread *pmd, if (ufidp) { CMAP_FOR_EACH_WITH_HASH (netdev_flow, node, dp_netdev_flow_hash(ufidp), &pmd->flow_table) { - if (ovs_u128_equal(&netdev_flow->ufid, ufidp)) { + if (ovs_u128_equals(&netdev_flow->ufid, ufidp)) { return netdev_flow; } } @@ -1807,22 +1817,27 @@ dp_netdev_flow_to_dpif_flow(const struct dp_netdev_flow *netdev_flow, struct flow_wildcards wc; struct dp_netdev_actions *actions; size_t offset; + struct odp_flow_key_parms odp_parms = { + .flow = &netdev_flow->flow, + .mask = &wc.masks, + .support = dp_netdev_support, + }; miniflow_expand(&netdev_flow->cr.mask->mf, &wc.masks); /* Key */ offset = key_buf->size; flow->key = ofpbuf_tail(key_buf); - odp_flow_key_from_flow(key_buf, &netdev_flow->flow, &wc.masks, - netdev_flow->flow.in_port.odp_port, true); + odp_parms.odp_in_port = netdev_flow->flow.in_port.odp_port; + odp_flow_key_from_flow(&odp_parms, key_buf); flow->key_len = key_buf->size - offset; /* Mask */ offset = mask_buf->size; flow->mask = ofpbuf_tail(mask_buf); - odp_flow_key_from_mask(mask_buf, &wc.masks, &netdev_flow->flow, - odp_to_u32(wc.masks.in_port.odp_port), - SIZE_MAX, true); + odp_parms.odp_in_port = wc.masks.in_port.odp_port; + odp_parms.key_buf = key_buf; + odp_flow_key_from_mask(&odp_parms, mask_buf); flow->mask_len = mask_buf->size - offset; /* Actions */ @@ -1841,12 +1856,13 @@ static int dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len, const struct nlattr *mask_key, uint32_t mask_key_len, const struct flow *flow, - struct flow *mask) + struct flow_wildcards *wc) { if (mask_key_len) { enum odp_key_fitness fitness; - fitness = odp_flow_key_to_mask(mask_key, mask_key_len, mask, flow); + fitness = odp_flow_key_to_mask_udpif(mask_key, mask_key_len, key, + key_len, &wc->masks, flow); if (fitness) { /* This should not happen: it indicates that * odp_flow_key_from_mask() and odp_flow_key_to_mask() @@ -1868,31 +1884,9 @@ dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len, return EINVAL; } } else { - enum mf_field_id id; - /* No mask key, unwildcard everything except fields whose - * prerequisities are not met. */ - memset(mask, 0x0, sizeof *mask); - - for (id = 0; id < MFF_N_IDS; ++id) { - /* Skip registers and metadata. */ - if (!(id >= MFF_REG0 && id < MFF_REG0 + FLOW_N_REGS) - && id != MFF_METADATA) { - const struct mf_field *mf = mf_from_id(id); - if (mf_are_prereqs_ok(mf, flow)) { - mf_mask_field(mf, mask); - } - } - } + flow_wildcards_init_for_packet(wc, flow); } - /* Force unwildcard the in_port. - * - * We need to do this even in the case where we unwildcard "everything" - * above because "everything" only includes the 16-bit OpenFlow port number - * mask->in_port.ofp_port, which only covers half of the 32-bit datapath - * port number mask->in_port.odp_port. */ - mask->in_port.odp_port = u32_to_odp(UINT32_MAX); - return 0; } @@ -1902,7 +1896,7 @@ dpif_netdev_flow_from_nlattrs(const struct nlattr *key, uint32_t key_len, { odp_port_t in_port; - if (odp_flow_key_to_flow(key, key_len, flow)) { + if (odp_flow_key_to_flow_udpif(key, key_len, flow)) { /* This should not happen: it indicates that odp_flow_key_from_flow() * and odp_flow_key_to_flow() disagree on the acceptable form of a * flow. Log the problem as an error, with enough details to enable @@ -1926,6 +1920,11 @@ dpif_netdev_flow_from_nlattrs(const struct nlattr *key, uint32_t key_len, return EINVAL; } + /* Userspace datapath doesn't support conntrack. */ + if (flow->ct_state || flow->ct_zone) { + return EINVAL; + } + return 0; } @@ -1935,7 +1934,8 @@ dpif_netdev_flow_get(const struct dpif *dpif, const struct dpif_flow_get *get) struct dp_netdev *dp = get_dp_netdev(dpif); struct dp_netdev_flow *netdev_flow; struct dp_netdev_pmd_thread *pmd; - int pmd_id = get->pmd_id == PMD_ID_NULL ? NON_PMD_CORE_ID : get->pmd_id; + unsigned pmd_id = get->pmd_id == PMD_ID_NULL + ? NON_PMD_CORE_ID : get->pmd_id; int error = 0; pmd = dp_netdev_get_pmd(dp, pmd_id); @@ -1968,13 +1968,15 @@ dp_netdev_flow_add(struct dp_netdev_pmd_thread *pmd, netdev_flow_mask_init(&mask, match); /* Make sure wc does not have metadata. */ - ovs_assert(!(mask.mf.map & (MINIFLOW_MAP(metadata) | MINIFLOW_MAP(regs)))); + ovs_assert(!FLOWMAP_HAS_FIELD(&mask.mf.map, metadata) + && !FLOWMAP_HAS_FIELD(&mask.mf.map, regs)); /* Do not allocate extra space. */ flow = xmalloc(sizeof *flow - sizeof flow->cr.flow.mf + mask.len); memset(&flow->stats, 0, sizeof flow->stats); flow->dead = false; - *CONST_CAST(int *, &flow->pmd_id) = pmd->core_id; + flow->batch = NULL; + *CONST_CAST(unsigned *, &flow->pmd_id) = pmd->core_id; *CONST_CAST(struct flow *, &flow->flow) = match->flow; *CONST_CAST(ovs_u128 *, &flow->ufid) = *ufid; ovs_refcount_init(&flow->ref_cnt); @@ -2017,7 +2019,8 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put) struct dp_netdev_pmd_thread *pmd; struct match match; ovs_u128 ufid; - int pmd_id = put->pmd_id == PMD_ID_NULL ? NON_PMD_CORE_ID : put->pmd_id; + unsigned pmd_id = put->pmd_id == PMD_ID_NULL + ? NON_PMD_CORE_ID : put->pmd_id; int error; error = dpif_netdev_flow_from_nlattrs(put->key, put->key_len, &match.flow); @@ -2026,7 +2029,7 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put) } error = dpif_netdev_mask_from_nlattrs(put->key, put->key_len, put->mask, put->mask_len, - &match.flow, &match.wc.masks); + &match.flow, &match.wc); if (error) { return error; } @@ -2112,7 +2115,8 @@ dpif_netdev_flow_del(struct dpif *dpif, const struct dpif_flow_del *del) struct dp_netdev *dp = get_dp_netdev(dpif); struct dp_netdev_flow *netdev_flow; struct dp_netdev_pmd_thread *pmd; - int pmd_id = del->pmd_id == PMD_ID_NULL ? NON_PMD_CORE_ID : del->pmd_id; + unsigned pmd_id = del->pmd_id == PMD_ID_NULL + ? NON_PMD_CORE_ID : del->pmd_id; int error = 0; pmd = dp_netdev_get_pmd(dp, pmd_id); @@ -2393,7 +2397,8 @@ dpif_netdev_pmd_set(struct dpif *dpif, unsigned int n_rxqs, const char *cmask) } /* Sets the new rx queue config. */ - err = netdev_set_multiq(port->netdev, ovs_numa_get_n_cores(), + err = netdev_set_multiq(port->netdev, + ovs_numa_get_n_cores() + 1, n_rxqs); if (err && (err != EOPNOTSUPP)) { VLOG_ERR("Failed to set dpdk interface %s rx_queue to:" @@ -2500,7 +2505,7 @@ dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd, struct dp_netdev_port *port, struct netdev_rxq *rxq) { - struct dp_packet *packets[NETDEV_MAX_RX_BATCH]; + struct dp_packet *packets[NETDEV_MAX_BURST]; int error, cnt; cycles_count_start(pmd); @@ -2513,7 +2518,7 @@ dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd, /* XXX: initialize md in netdev implementation. */ for (i = 0; i < cnt; i++) { - packets[i]->md = PKT_METADATA_INITIALIZER(port->port_no); + pkt_metadata_init(&packets[i]->md, port->port_no); } cycles_count_start(pmd); dp_netdev_input(pmd, packets, cnt); @@ -2550,6 +2555,7 @@ dpif_netdev_run(struct dpif *dpif) dp_netdev_pmd_unref(non_pmd); tnl_arp_cache_run(); + tnl_port_map_run(); new_tnl_seq = seq_read(tnl_conf_seq); if (dp->last_tnl_conf_seq != new_tnl_seq) { @@ -2651,6 +2657,11 @@ reload: emc_cache_init(&pmd->flow_cache); poll_cnt = pmd_load_queues(pmd, &poll_list, poll_cnt); + /* List port/core affinity */ + for (i = 0; i < poll_cnt; i++) { + VLOG_INFO("Core %d processing port \'%s\'\n", pmd->core_id, netdev_get_name(poll_list[i].port->netdev)); + } + /* Signal here to make sure the pmd finishes * reloading the updated configuration. */ dp_netdev_pmd_reload_done(pmd); @@ -2668,6 +2679,7 @@ reload: lc = 0; emc_cache_slow_sweep(&pmd->flow_cache); + coverage_try_clear(); ovsrcu_quiesce(); atomic_read_relaxed(&pmd->change_seq, &seq); @@ -2737,7 +2749,7 @@ dp_netdev_pmd_reload_done(struct dp_netdev_pmd_thread *pmd) * * Caller must unrefs the returned reference. */ static struct dp_netdev_pmd_thread * -dp_netdev_get_pmd(struct dp_netdev *dp, int core_id) +dp_netdev_get_pmd(struct dp_netdev *dp, unsigned core_id) { struct dp_netdev_pmd_thread *pmd; const struct cmap_node *pnode; @@ -2797,14 +2809,25 @@ dp_netdev_pmd_get_next(struct dp_netdev *dp, struct cmap_position *pos) return next; } +static int +core_id_to_qid(unsigned core_id) +{ + if (core_id != NON_PMD_CORE_ID) { + return core_id; + } else { + return ovs_numa_get_n_cores(); + } +} + /* Configures the 'pmd' based on the input argument. */ static void dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd, struct dp_netdev *dp, - int index, int core_id, int numa_id) + int index, unsigned core_id, int numa_id) { pmd->dp = dp; pmd->index = index; pmd->core_id = core_id; + pmd->tx_qid = core_id_to_qid(core_id); pmd->numa_id = numa_id; ovs_refcount_init(&pmd->ref_cnt); @@ -2840,7 +2863,7 @@ dp_netdev_destroy_pmd(struct dp_netdev_pmd_thread *pmd) /* Stops the pmd thread, removes it from the 'dp->poll_threads', * and unrefs the struct. */ static void -dp_netdev_del_pmd(struct dp_netdev_pmd_thread *pmd) +dp_netdev_del_pmd(struct dp_netdev *dp, struct dp_netdev_pmd_thread *pmd) { /* Uninit the 'flow_cache' since there is * no actual thread uninit it for NON_PMD_CORE_ID. */ @@ -2852,6 +2875,11 @@ dp_netdev_del_pmd(struct dp_netdev_pmd_thread *pmd) ovs_numa_unpin_core(pmd->core_id); xpthread_join(pmd->thread, NULL); } + /* Purges the 'pmd''s flows after stopping the thread, but before + * destroying the flows, so that the flow stats can be collected. */ + if (dp->dp_purge_cb) { + dp->dp_purge_cb(dp->dp_purge_aux, pmd->core_id); + } cmap_remove(&pmd->dp->poll_threads, &pmd->node, hash_int(pmd->core_id, 0)); dp_netdev_pmd_unref(pmd); } @@ -2863,7 +2891,7 @@ dp_netdev_destroy_all_pmds(struct dp_netdev *dp) struct dp_netdev_pmd_thread *pmd; CMAP_FOR_EACH (pmd, node, &dp->poll_threads) { - dp_netdev_del_pmd(pmd); + dp_netdev_del_pmd(dp, pmd); } } @@ -2875,7 +2903,7 @@ dp_netdev_del_pmds_on_numa(struct dp_netdev *dp, int numa_id) CMAP_FOR_EACH (pmd, node, &dp->poll_threads) { if (pmd->numa_id == numa_id) { - dp_netdev_del_pmd(pmd); + dp_netdev_del_pmd(dp, pmd); } } } @@ -2900,6 +2928,7 @@ dp_netdev_set_pmds_on_numa(struct dp_netdev *dp, int numa_id) * pmd threads for the numa node. */ if (!n_pmds) { int can_have, n_unpinned, i; + struct dp_netdev_pmd_thread **pmds; n_unpinned = ovs_numa_get_n_unpinned_cores_on_numa(numa_id); if (!n_unpinned) { @@ -2911,15 +2940,22 @@ dp_netdev_set_pmds_on_numa(struct dp_netdev *dp, int numa_id) /* If cpu mask is specified, uses all unpinned cores, otherwise * tries creating NR_PMD_THREADS pmd threads. */ can_have = dp->pmd_cmask ? n_unpinned : MIN(n_unpinned, NR_PMD_THREADS); + pmds = xzalloc(can_have * sizeof *pmds); + for (i = 0; i < can_have; i++) { + unsigned core_id = ovs_numa_get_unpinned_core_on_numa(numa_id); + pmds[i] = xzalloc(sizeof **pmds); + dp_netdev_configure_pmd(pmds[i], dp, i, core_id, numa_id); + } + /* The pmd thread code needs to see all the others configured pmd + * threads on the same numa node. That's why we call + * 'dp_netdev_configure_pmd()' on all the threads and then we actually + * start them. */ for (i = 0; i < can_have; i++) { - struct dp_netdev_pmd_thread *pmd = xzalloc(sizeof *pmd); - int core_id = ovs_numa_get_unpinned_core_on_numa(numa_id); - - dp_netdev_configure_pmd(pmd, dp, i, core_id, numa_id); /* Each thread will distribute all devices rx-queues among * themselves. */ - pmd->thread = ovs_thread_create("pmd", pmd_thread_main, pmd); + pmds[i]->thread = ovs_thread_create("pmd", pmd_thread_main, pmds[i]); } + free(pmds); VLOG_INFO("Created %d pmd threads on numa node %d", can_have, numa_id); } } @@ -2975,19 +3011,40 @@ dp_netdev_upcall(struct dp_netdev_pmd_thread *pmd, struct dp_packet *packet_, struct ofpbuf *actions, struct ofpbuf *put_actions) { struct dp_netdev *dp = pmd->dp; + struct flow_tnl orig_tunnel; + int err; if (OVS_UNLIKELY(!dp->upcall_cb)) { return ENODEV; } + /* Upcall processing expects the Geneve options to be in the translated + * format but we need to retain the raw format for datapath use. */ + orig_tunnel.flags = flow->tunnel.flags; + if (flow->tunnel.flags & FLOW_TNL_F_UDPIF) { + orig_tunnel.metadata.present.len = flow->tunnel.metadata.present.len; + memcpy(orig_tunnel.metadata.opts.gnv, flow->tunnel.metadata.opts.gnv, + flow->tunnel.metadata.present.len); + err = tun_metadata_from_geneve_udpif(&orig_tunnel, &orig_tunnel, + &flow->tunnel); + if (err) { + return err; + } + } + if (OVS_UNLIKELY(!VLOG_DROP_DBG(&upcall_rl))) { struct ds ds = DS_EMPTY_INITIALIZER; char *packet_str; struct ofpbuf key; + struct odp_flow_key_parms odp_parms = { + .flow = flow, + .mask = &wc->masks, + .odp_in_port = flow->in_port.odp_port, + .support = dp_netdev_support, + }; ofpbuf_init(&key, 0); - odp_flow_key_from_flow(&key, flow, &wc->masks, flow->in_port.odp_port, - true); + odp_flow_key_from_flow(&odp_parms, &key); packet_str = ofp_packet_to_string(dp_packet_data(packet_), dp_packet_size(packet_)); @@ -3002,20 +3059,65 @@ dp_netdev_upcall(struct dp_netdev_pmd_thread *pmd, struct dp_packet *packet_, ds_destroy(&ds); } - return dp->upcall_cb(packet_, flow, ufid, pmd->core_id, type, userdata, - actions, wc, put_actions, dp->upcall_aux); + err = dp->upcall_cb(packet_, flow, ufid, pmd->core_id, type, userdata, + actions, wc, put_actions, dp->upcall_aux); + if (err && err != ENOSPC) { + return err; + } + + /* Translate tunnel metadata masks to datapath format. */ + if (wc) { + if (wc->masks.tunnel.metadata.present.map) { + struct geneve_opt opts[GENEVE_TOT_OPT_SIZE / + sizeof(struct geneve_opt)]; + + tun_metadata_to_geneve_udpif_mask(&flow->tunnel, + &wc->masks.tunnel, + orig_tunnel.metadata.opts.gnv, + orig_tunnel.metadata.present.len, + opts); + + memset(&wc->masks.tunnel.metadata, 0, + sizeof wc->masks.tunnel.metadata); + memcpy(&wc->masks.tunnel.metadata.opts.gnv, opts, + orig_tunnel.metadata.present.len); + } + wc->masks.tunnel.metadata.present.len = 0xff; + } + + /* Restore tunnel metadata. We need to use the saved options to ensure + * that any unknown options are not lost. The generated mask will have + * the same structure, matching on types and lengths but wildcarding + * option data we don't care about. */ + if (orig_tunnel.flags & FLOW_TNL_F_UDPIF) { + memcpy(&flow->tunnel.metadata.opts.gnv, orig_tunnel.metadata.opts.gnv, + orig_tunnel.metadata.present.len); + flow->tunnel.metadata.present.len = orig_tunnel.metadata.present.len; + flow->tunnel.flags |= FLOW_TNL_F_UDPIF; + } + + return err; } static inline uint32_t -dpif_netdev_packet_get_dp_hash(struct dp_packet *packet, - const struct miniflow *mf) +dpif_netdev_packet_get_rss_hash(struct dp_packet *packet, + const struct miniflow *mf) { - uint32_t hash; + uint32_t hash, recirc_depth; - hash = dp_packet_get_dp_hash(packet); - if (OVS_UNLIKELY(!hash)) { + if (OVS_LIKELY(dp_packet_rss_valid(packet))) { + hash = dp_packet_get_rss_hash(packet); + } else { hash = miniflow_hash_5tuple(mf, 0); - dp_packet_set_dp_hash(packet, hash); + dp_packet_set_rss_hash(packet, hash); + } + + /* The RSS hash must account for the recirculation depth to avoid + * collisions in the exact match cache */ + recirc_depth = *recirc_depth_get_unsafe(); + if (OVS_UNLIKELY(recirc_depth)) { + hash = hash_finish(hash, recirc_depth); + dp_packet_set_rss_hash(packet, hash); } return hash; } @@ -3027,7 +3129,7 @@ struct packet_batch { struct dp_netdev_flow *flow; - struct dp_packet *packets[NETDEV_MAX_RX_BATCH]; + struct dp_packet *packets[NETDEV_MAX_BURST]; }; static inline void @@ -3042,8 +3144,9 @@ packet_batch_update(struct packet_batch *batch, struct dp_packet *packet, static inline void packet_batch_init(struct packet_batch *batch, struct dp_netdev_flow *flow) { - batch->flow = flow; + flow->batch = batch; + batch->flow = flow; batch->packet_count = 0; batch->byte_count = 0; batch->tcp_flags = 0; @@ -3052,56 +3155,35 @@ packet_batch_init(struct packet_batch *batch, struct dp_netdev_flow *flow) static inline void packet_batch_execute(struct packet_batch *batch, struct dp_netdev_pmd_thread *pmd, - enum dp_stat_type hit_type, long long now) { struct dp_netdev_actions *actions; struct dp_netdev_flow *flow = batch->flow; - dp_netdev_flow_used(batch->flow, batch->packet_count, batch->byte_count, + dp_netdev_flow_used(flow, batch->packet_count, batch->byte_count, batch->tcp_flags, now); actions = dp_netdev_flow_get_actions(flow); dp_netdev_execute_actions(pmd, batch->packets, batch->packet_count, true, actions->actions, actions->size); - - dp_netdev_count_packet(pmd, hit_type, batch->packet_count); } -static inline bool +static inline void dp_netdev_queue_batches(struct dp_packet *pkt, struct dp_netdev_flow *flow, const struct miniflow *mf, - struct packet_batch *batches, size_t *n_batches, - size_t max_batches) -{ - struct packet_batch *batch = NULL; - int j; - - if (OVS_UNLIKELY(!flow)) { - return false; - } - /* XXX: This O(n^2) algortihm makes sense if we're operating under the - * assumption that the number of distinct flows (and therefore the - * number of distinct batches) is quite small. If this turns out not - * to be the case, it may make sense to pre sort based on the - * netdev_flow pointer. That done we can get the appropriate batching - * in O(n * log(n)) instead. */ - for (j = *n_batches - 1; j >= 0; j--) { - if (batches[j].flow == flow) { - batch = &batches[j]; - packet_batch_update(batch, pkt, mf); - return true; - } - } - if (OVS_UNLIKELY(*n_batches >= max_batches)) { - return false; + struct packet_batch *batches, size_t *n_batches) +{ + struct packet_batch *batch = flow->batch; + + if (OVS_LIKELY(batch)) { + packet_batch_update(batch, pkt, mf); + return; } batch = &batches[(*n_batches)++]; packet_batch_init(batch, flow); packet_batch_update(batch, pkt, mf); - return true; } static inline void @@ -3113,25 +3195,22 @@ dp_packet_swap(struct dp_packet **a, struct dp_packet **b) } /* Try to process all ('cnt') the 'packets' using only the exact match cache - * 'flow_cache'. If a flow is not found for a packet 'packets[i]', or if there - * is no matching batch for a packet's flow, the miniflow is copied into 'keys' - * and the packet pointer is moved at the beginning of the 'packets' array. + * '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. * * 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). */ static inline size_t emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets, - size_t cnt, struct netdev_flow_key *keys, long long now) + size_t cnt, struct netdev_flow_key *keys, + struct packet_batch batches[], size_t *n_batches) { - struct netdev_flow_key key; - struct packet_batch batches[4]; struct emc_cache *flow_cache = &pmd->flow_cache; - size_t n_batches, i; - size_t notfound_cnt = 0; + struct netdev_flow_key key; + size_t i, notfound_cnt = 0; - n_batches = 0; - miniflow_initialize(&key.mf, key.buf); for (i = 0; i < cnt; i++) { struct dp_netdev_flow *flow; @@ -3140,14 +3219,20 @@ emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets, continue; } + if (i != cnt - 1) { + /* Prefetch next packet data */ + OVS_PREFETCH(dp_packet_data(packets[i+1])); + } + miniflow_extract(packets[i], &key.mf); key.len = 0; /* Not computed yet. */ - key.hash = dpif_netdev_packet_get_dp_hash(packets[i], &key.mf); + key.hash = dpif_netdev_packet_get_rss_hash(packets[i], &key.mf); 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 (OVS_LIKELY(flow)) { + dp_netdev_queue_batches(packets[i], flow, &key.mf, batches, + n_batches); + } else { if (i != notfound_cnt) { dp_packet_swap(&packets[i], &packets[notfound_cnt]); } @@ -3156,9 +3241,7 @@ emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets, } } - for (i = 0; i < n_batches; i++) { - packet_batch_execute(&batches[i], pmd, DP_STAT_EXACT_HIT, now); - } + dp_netdev_count_packet(pmd, DP_STAT_EXACT_HIT, cnt - notfound_cnt); return notfound_cnt; } @@ -3166,30 +3249,30 @@ emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets, static inline void fast_path_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets, size_t cnt, - struct netdev_flow_key *keys, long long now) + struct netdev_flow_key *keys, + struct packet_batch batches[], size_t *n_batches) { #if !defined(__CHECKER__) && !defined(_WIN32) const size_t PKT_ARRAY_SIZE = cnt; #else /* Sparse or MSVC doesn't like variable length array. */ - enum { PKT_ARRAY_SIZE = NETDEV_MAX_RX_BATCH }; + enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST }; #endif - struct packet_batch batches[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; + int miss_cnt = 0, lost_cnt = 0; bool any_miss; + size_t i; for (i = 0; i < cnt; i++) { /* 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)); + keys[i].len = netdev_flow_key_size(miniflow_n_values(&keys[i].mf)); } any_miss = !dpcls_lookup(&pmd->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; - int miss_cnt = 0, lost_cnt = 0; ovs_u128 ufid; ofpbuf_use_stub(&actions, actions_stub, sizeof actions_stub); @@ -3231,6 +3314,16 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, continue; } + /* The Netlink encoding of datapath flow keys cannot express + * wildcarding the presence of a VLAN tag. Instead, a missing VLAN + * tag is interpreted as exact match on the fact that there is no + * VLAN. Unless we refactor a lot of code that translates between + * Netlink and struct flow representations, we have to do the same + * here. */ + if (!match.wc.masks.vlan_tci) { + match.wc.masks.vlan_tci = htons(0xffff); + } + /* We can't allow the packet batching in the next loop to execute * the actions. Otherwise, if there are any slow path actions, * we'll send the packet up twice. */ @@ -3261,23 +3354,17 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, ofpbuf_uninit(&actions); ofpbuf_uninit(&put_actions); fat_rwlock_unlock(&dp->upcall_rwlock); - dp_netdev_count_packet(pmd, DP_STAT_MISS, miss_cnt); dp_netdev_count_packet(pmd, DP_STAT_LOST, lost_cnt); } else if (OVS_UNLIKELY(any_miss)) { - int dropped_cnt = 0; - for (i = 0; i < cnt; i++) { if (OVS_UNLIKELY(!rules[i])) { dp_packet_delete(packets[i]); - dropped_cnt++; + lost_cnt++; + miss_cnt++; } } - - dp_netdev_count_packet(pmd, DP_STAT_MISS, dropped_cnt); - dp_netdev_count_packet(pmd, DP_STAT_LOST, dropped_cnt); } - n_batches = 0; for (i = 0; i < cnt; i++) { struct dp_packet *packet = packets[i]; struct dp_netdev_flow *flow; @@ -3289,13 +3376,12 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd, flow = dp_netdev_flow_cast(rules[i]); emc_insert(flow_cache, &keys[i], flow); - dp_netdev_queue_batches(packet, flow, &keys[i].mf, batches, - &n_batches, ARRAY_SIZE(batches)); + dp_netdev_queue_batches(packet, flow, &keys[i].mf, batches, n_batches); } - for (i = 0; i < n_batches; i++) { - packet_batch_execute(&batches[i], pmd, DP_STAT_MASKED_HIT, now); - } + dp_netdev_count_packet(pmd, DP_STAT_MASKED_HIT, cnt - miss_cnt); + dp_netdev_count_packet(pmd, DP_STAT_MISS, miss_cnt); + dp_netdev_count_packet(pmd, DP_STAT_LOST, lost_cnt); } static void @@ -3306,15 +3392,25 @@ dp_netdev_input(struct dp_netdev_pmd_thread *pmd, const size_t PKT_ARRAY_SIZE = cnt; #else /* Sparse or MSVC doesn't like variable length array. */ - enum { PKT_ARRAY_SIZE = NETDEV_MAX_RX_BATCH }; + enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST }; #endif struct netdev_flow_key keys[PKT_ARRAY_SIZE]; + struct packet_batch batches[PKT_ARRAY_SIZE]; long long now = time_msec(); - size_t newcnt; + size_t newcnt, n_batches, i; - newcnt = emc_processing(pmd, packets, cnt, keys, now); + n_batches = 0; + newcnt = emc_processing(pmd, packets, cnt, keys, batches, &n_batches); if (OVS_UNLIKELY(newcnt)) { - fast_path_processing(pmd, packets, newcnt, keys, now); + fast_path_processing(pmd, packets, newcnt, keys, batches, &n_batches); + } + + for (i = 0; i < n_batches; i++) { + batches[i].flow->batch = NULL; + } + + for (i = 0; i < n_batches; i++) { + packet_batch_execute(&batches[i], pmd, now); } } @@ -3322,6 +3418,15 @@ struct dp_netdev_execute_aux { struct dp_netdev_pmd_thread *pmd; }; +static void +dpif_netdev_register_dp_purge_cb(struct dpif *dpif, dp_purge_callback *cb, + void *aux) +{ + struct dp_netdev *dp = get_dp_netdev(dpif); + dp->dp_purge_aux = aux; + dp->dp_purge_cb = cb; +} + static void dpif_netdev_register_upcall_cb(struct dpif *dpif, upcall_callback *cb, void *aux) @@ -3332,7 +3437,7 @@ dpif_netdev_register_upcall_cb(struct dpif *dpif, upcall_callback *cb, } static void -dp_netdev_drop_packets(struct dp_packet ** packets, int cnt, bool may_steal) +dp_netdev_drop_packets(struct dp_packet **packets, int cnt, bool may_steal) { if (may_steal) { int i; @@ -3363,13 +3468,13 @@ push_tnl_action(const struct dp_netdev *dp, } static void -dp_netdev_clone_pkt_batch(struct dp_packet **tnl_pkt, - struct dp_packet **packets, int cnt) +dp_netdev_clone_pkt_batch(struct dp_packet **dst_pkts, + struct dp_packet **src_pkts, int cnt) { int i; for (i = 0; i < cnt; i++) { - tnl_pkt[i] = dp_packet_clone(packets[i]); + dst_pkts[i] = dp_packet_clone(src_pkts[i]); } } @@ -3380,8 +3485,8 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt, { struct dp_netdev_execute_aux *aux = aux_; uint32_t *depth = recirc_depth_get(); - struct dp_netdev_pmd_thread *pmd= aux->pmd; - struct dp_netdev *dp= pmd->dp; + struct dp_netdev_pmd_thread *pmd = aux->pmd; + struct dp_netdev *dp = pmd->dp; int type = nl_attr_type(a); struct dp_netdev_port *p; int i; @@ -3390,14 +3495,14 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt, case OVS_ACTION_ATTR_OUTPUT: p = dp_netdev_lookup_port(dp, u32_to_odp(nl_attr_get_u32(a))); if (OVS_LIKELY(p)) { - netdev_send(p->netdev, pmd->core_id, packets, cnt, may_steal); + netdev_send(p->netdev, pmd->tx_qid, packets, cnt, may_steal); return; } break; case OVS_ACTION_ATTR_TUNNEL_PUSH: if (*depth < MAX_RECIRC_DEPTH) { - struct dp_packet *tnl_pkt[NETDEV_MAX_RX_BATCH]; + struct dp_packet *tnl_pkt[NETDEV_MAX_BURST]; int err; if (!may_steal) { @@ -3423,7 +3528,7 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt, p = dp_netdev_lookup_port(dp, portno); if (p) { - struct dp_packet *tnl_pkt[NETDEV_MAX_RX_BATCH]; + struct dp_packet *tnl_pkt[NETDEV_MAX_BURST]; int err; if (!may_steal) { @@ -3485,21 +3590,19 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt, case OVS_ACTION_ATTR_RECIRC: if (*depth < MAX_RECIRC_DEPTH) { + struct dp_packet *recirc_pkts[NETDEV_MAX_BURST]; - (*depth)++; - for (i = 0; i < cnt; i++) { - struct dp_packet *recirc_pkt; - - recirc_pkt = (may_steal) ? packets[i] - : dp_packet_clone(packets[i]); - - recirc_pkt->md.recirc_id = nl_attr_get_u32(a); - - /* Hash is private to each packet */ - recirc_pkt->md.dp_hash = dp_packet_get_dp_hash(packets[i]); + if (!may_steal) { + dp_netdev_clone_pkt_batch(recirc_pkts, packets, cnt); + packets = recirc_pkts; + } - dp_netdev_input(pmd, &recirc_pkt, 1); + for (i = 0; i < cnt; i++) { + packets[i]->md.recirc_id = nl_attr_get_u32(a); } + + (*depth)++; + dp_netdev_input(pmd, packets, cnt); (*depth)--; return; @@ -3508,6 +3611,13 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt, VLOG_WARN("Packet dropped. Max recirculation depth exceeded."); break; + case OVS_ACTION_ATTR_CT: + /* If a flow with this action is slow-pathed, datapath assistance is + * required to implement it. However, we don't support this action + * in the userspace datapath. */ + VLOG_WARN("Cannot execute conntrack action in userspace."); + break; + case OVS_ACTION_ATTR_PUSH_VLAN: case OVS_ACTION_ATTR_POP_VLAN: case OVS_ACTION_ATTR_PUSH_MPLS: @@ -3571,6 +3681,7 @@ const struct dpif_class dpif_netdev_class = { NULL, /* recv */ NULL, /* recv_wait */ NULL, /* recv_purge */ + dpif_netdev_register_dp_purge_cb, dpif_netdev_register_upcall_cb, dpif_netdev_enable_upcall, dpif_netdev_disable_upcall, @@ -3671,21 +3782,29 @@ dpif_dummy_register__(const char *type) dp_register_provider(class); } +static void +dpif_dummy_override(const char *type) +{ + if (!dp_unregister_provider(type)) { + dpif_dummy_register__(type); + } +} + void -dpif_dummy_register(bool override) +dpif_dummy_register(enum dummy_level level) { - if (override) { + if (level == DUMMY_OVERRIDE_ALL) { struct sset types; const char *type; sset_init(&types); dp_enumerate_types(&types); SSET_FOR_EACH (type, &types) { - if (!dp_unregister_provider(type)) { - dpif_dummy_register__(type); - } + dpif_dummy_override(type); } sset_destroy(&types); + } else if (level == DUMMY_OVERRIDE_SYSTEM) { + dpif_dummy_override("system"); } dpif_dummy_register__("dummy"); @@ -3739,6 +3858,7 @@ dpcls_destroy(struct dpcls *cls) struct dpcls_subtable *subtable; CMAP_FOR_EACH (subtable, cmap_node, &cls->subtables_map) { + ovs_assert(cmap_count(&subtable->rules) == 0); dpcls_destroy_subtable(cls, subtable); } cmap_destroy(&cls->subtables_map); @@ -3805,20 +3925,18 @@ dpcls_remove(struct dpcls *cls, struct dpcls_rule *rule) } } -/* 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. */ +/* 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 dpcls_rule_matches_key(const struct dpcls_rule *rule, const struct netdev_flow_key *target) { - const uint64_t *keyp = rule->flow.mf.inline_values; - const uint64_t *maskp = rule->mask->mf.inline_values; - uint64_t target_u64; + const uint64_t *keyp = miniflow_get_values(&rule->flow.mf); + const uint64_t *maskp = miniflow_get_values(&rule->mask->mf); + uint64_t value; - NETDEV_FLOW_KEY_FOR_EACH_IN_MAP(target_u64, target, rule->flow.mf.map) { - if (OVS_UNLIKELY((target_u64 & *maskp++) != *keyp++)) { + NETDEV_FLOW_KEY_FOR_EACH_IN_FLOWMAP(value, target, rule->flow.mf.map) { + if (OVS_UNLIKELY((value & *maskp++) != *keyp++)) { return false; } } @@ -3846,7 +3964,7 @@ dpcls_lookup(const struct dpcls *cls, const struct netdev_flow_key keys[], #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) }; + enum { N_MAPS = DIV_ROUND_UP(NETDEV_MAX_BURST, MAP_BITS) }; #endif map_type maps[N_MAPS]; struct dpcls_subtable *subtable; @@ -3876,14 +3994,14 @@ dpcls_lookup(const struct dpcls *cls, const struct netdev_flow_key keys[], } /* Compute hashes for the remaining keys. */ - ULONG_FOR_EACH_1(i, map) { + ULLONG_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) { + ULLONG_FOR_EACH_1(i, map) { struct dpcls_rule *rule; CMAP_NODE_FOR_EACH (rule, cmap_node, nodes[i]) { @@ -3892,7 +4010,7 @@ dpcls_lookup(const struct dpcls *cls, const struct netdev_flow_key keys[], goto next; } } - ULONG_SET0(map, i); /* Did not match. */ + ULLONG_SET0(map, i); /* Did not match. */ next: ; /* Keep Sparse happy. */ }