Hi Jan,
thanks for the patch!
I tried benchmarking with flow tables from utilities/ovs-pipegen.py and the
results look very good.
I had to manually edit the patch to apply: some whitespaces are trimmed,
some lines are truncated and the form feeds('^L') are two separate
characters. Would you mind using git send-email next time?
I have a few minor comments inline
Thanks,
Daniele
2016-08-10 7:16 GMT-07:00 Jan Scheurich <[email protected]>:
> This replaces PATCH v3 (http://patchwork.ozlabs.org/patch/657277).
> It should apply once Jarno's patch to revert the cpvector introduction
> (http://patchwork.ozlabs.org/patch/657508) has been merged.
>
>
I guess we shouldn't include this in the commit message
>
> The user-space datapath (dpif-netdev) consists of a first level "exact
> match
> cache" (EMC) matching on 5-tuples and the normal megaflow classifier. With
> many parallel packet flows (e.g. TCP connections) the EMC becomes
> inefficient
> and the OVS forwarding performance is determined by the megaflow
> classifier.
>
> The megaflow classifier (dpcls) consists of a variable number of hash
> tables
> (aka subtables), each containing megaflow entries with the same mask of
> packet header and metadata fields to match upon. A dpcls lookup matches a
> given packet against all subtables in sequence until it hits a match. As
> megaflow cache entries are by construction non-overlapping, the first match
> is the only match.
>
> Today the order of the subtables in the dpcls is essentially random so that
> on average a dpcls lookup has to visit N/2 subtables for a hit, when N is
> the
> total number of subtables. Even though every single hash-table lookup is
> fast, the performance of the current dpcls degrades when there are many
> subtables.
>
> How does the patch address this issue:
>
> In reality there is often a strong correlation between the ingress port
> and a
> small subset of subtables that have hits. The entire megaflow cache
> typically
> decomposes nicely into partitions that are hit only by packets entering
> from
> a range of similar ports (e.g. traffic from Phy -> VM vs. traffic from VM
> ->
> Phy).
>
> Therefore, maintaining a separate dpcls instance per ingress port with its
> subtable vector sorted by frequency of hits reduces the average number of
> subtables lookups in the dpcls to a minimum, even if the total number of
> subtables gets large. This is possible because megaflows always have an
> exact
> match on in_port, so every megaflow belongs to unique dpcls instance.
>
> For thread safety, the PMD thread needs to block out revalidators during
> the
> periodic optimization. We use ovs_mutex_trylock() to avoid blocking the
> PMD.
>
> To monitor the effectiveness of the patch we have enhanced the ovs-appctl
> dpif-netdev/pmd-stats-show command with an extra line "avg. subtable
> lookups
> per hit" to report the average number of subtable lookup needed for a
> megaflow match. Ideally, this should be close to 1 and almost all cases
> much
> smaller than N/2.
>
> The PMD tests have been adjusted to the additional line in pmd-stats-show.
>
> We have benchmarked a L3-VPN pipeline on top of a VXLAN overlay mesh.
> With pure L3 tenant traffic between VMs on different nodes the resulting
> netdev dpcls contains N=4 subtables. Each packet traversing the OVS
> datapath is subject to dpcls lookup twice due to the tunnel termination.
>
> Disabling the EMC, we have measured a baseline performance (in+out) of
> ~1.45
> Mpps (64 bytes, 10K L4 packet flows). The average number of subtable
> lookups
> per dpcls match is 2.5. With the patch the average number of subtable
> lookups
> per dpcls match is reduced to 1 and the forwarding performance grows by
> ~50%
> to 2.13 Mpps.
>
> Even with EMC enabled, the patch improves the performance by 9% (for 1000
> L4
> flows) and 34% (for 50K+ L4 flows).
>
> As the actual number of subtables will often be higher in reality, we can
> assume that this is at the lower end of the speed-up one can expect from
> this
> optimization. Just running a parallel ping between the VXLAN tunnel
> endpoints
> increases the number of subtables and hence the average number of subtable
> lookups from 2.5 to 3.5 on master with a corresponding decrease of
> throughput
> to 1.2 Mpps. With the patch the parallel ping has no impact on average
> number
> of subtable lookups and performance. The performance gain is then ~75%.
>
>
> Signed-off-by: Jan Scheurich <[email protected]>
>
> Changes in v4:
> - Renamed cpvector back to pvector after Jarno's revert patch
> http://patchwork.ozlabs.org/patch/657508
>
> Changes in v3:
> - Rebased to master (commit 6ef5fa92eb70)
> - Updated performance benchmark figures
> - Adapted to renamed cpvector API
> - Reverted dplcs to using cpvector due to threading issue during flow
> removal
> - Implemented v2 comments by Antonio Fischetti
>
> Changes in v2:
> - Rebased to master (commit 3041e1fc9638)
> - Take the pmd->flow_mutex during optimization to block out revalidators
> Use trylock in order to not block the PMD thread
> - Made in_port an explicit input parameter to fast_path_processing()
> - Fixed coding style issues
>
>
>
We usually include this after the '---' to avoid polluting the commit
message.
> ---
>
> lib/dpif-netdev.c | 202 ++++++++++++++++++++++++++++++
> +++++++++++++++++++++++--------
> tests/pmd.at | 6 +-
> 2 files changed, 181 insertions(+), 27 deletions(-)
>
>
> diff --git a/lib/dpif-netdev.c b/lib/dpif-netdev.c
> index fe19b38..a46223f 100644
> --- a/lib/dpif-netdev.c
> +++ b/lib/dpif-netdev.c
> @@ -161,7 +161,12 @@ struct emc_cache {
> ^L
> /* Simple non-wildcarding single-priority classifier. */
>
> +/* Time in ms between successive optimizations of the dpcls subtable
> vector */
> +#define DPCLS_OPTIMIZATION_INTERVAL 1000
> +
> struct dpcls {
>
I would add a comment here that links 'node' with
'dp_netdev_pmd_thread.classifier'
> + struct cmap_node node;
> + odp_port_t in_port;
> struct cmap subtables_map;
> struct pvector subtables;
> };
> @@ -176,12 +181,14 @@ struct dpcls_rule {
>
> static void dpcls_init(struct dpcls *);
> static void dpcls_destroy(struct dpcls *);
> +static void dpcls_sort_subtable_vector(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,
> +static bool dpcls_lookup(struct dpcls *cls,
> const struct netdev_flow_key keys[],
> - struct dpcls_rule **rules, size_t cnt);
> + struct dpcls_rule **rules, size_t cnt,
> + int *num_lookups_p);
> ^L
> /* Datapath based on the network device interface from netdev.h.
> *
> @@ -255,6 +262,8 @@ enum dp_stat_type {
> DP_STAT_MASKED_HIT, /* Packets that matched in the flow
> table. */
> DP_STAT_MISS, /* Packets that did not match. */
> DP_STAT_LOST, /* Packets not passed up to the client. */
> + DP_STAT_LOOKUP_HIT, /* Number of subtable lookups for flow
> table
> + hits */
> DP_N_STATS
> };
>
> @@ -448,15 +457,20 @@ struct dp_netdev_pmd_thread {
> * will only be accessed by its own pmd thread. */
> struct emc_cache flow_cache;
>
> - /* Classifier and Flow-Table.
> + /* Flow-Table and classifiers
> *
> * Writers of 'flow_table' must take the 'flow_mutex'. Corresponding
> - * changes to 'cls' must be made while still holding the 'flow_mutex'.
> + * changes to 'classifiers' must be made while still holding the
> + * 'flow_mutex'.
> */
> struct ovs_mutex flow_mutex;
> - struct dpcls cls;
> struct cmap flow_table OVS_GUARDED; /* Flow table. */
>
> + /* One classifier per in_port polled by the pmd */
> + struct cmap classifiers;
> + /* Periodically sort subtable vectors according to hit frequencies */
> + long long int next_optimization;
> +
> /* Statistics. */
> struct dp_netdev_pmd_stats stats;
>
> @@ -574,6 +588,8 @@ static void dp_netdev_pmd_unref(struct
> dp_netdev_pmd_thread *pmd);
> static void dp_netdev_pmd_flow_flush(struct dp_netdev_pmd_thread *pmd);
> static void pmd_load_cached_ports(struct dp_netdev_pmd_thread *pmd)
> OVS_REQUIRES(pmd->port_mutex);
> +static inline void
> +dp_netdev_pmd_try_optimize(struct dp_netdev_pmd_thread *pmd);
>
> static void
> dpif_netdev_xps_revalidate_pmd(const struct dp_netdev_pmd_thread *pmd,
> @@ -699,8 +715,12 @@ pmd_info_show_stats(struct ds *reply,
>
> ds_put_format(reply,
> "\temc hits:%llu\n\tmegaflow hits:%llu\n"
> + "\tavg. subtable lookups per hit:%.2f\n"
> "\tmiss:%llu\n\tlost:%llu\n",
> stats[DP_STAT_EXACT_HIT], stats[DP_STAT_MASKED_HIT],
> + stats[DP_STAT_MASKED_HIT] > 0
> + ? (1.0*stats[DP_STAT_LOOKUP_HIT]
> )/stats[DP_STAT_MASKED_HIT]
> + : 0,
> stats[DP_STAT_MISS], stats[DP_STAT_LOST]);
>
> if (total_cycles == 0) {
> @@ -1583,14 +1603,51 @@ dp_netdev_flow_hash(const ovs_u128 *ufid)
> return ufid->u32[0];
> }
>
> +static inline struct dpcls *
> +dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
> + odp_port_t in_port)
> +{
> + struct dpcls *cls;
> + uint32_t hash = hash_port_no(in_port);
> + CMAP_FOR_EACH_WITH_HASH (cls, node, hash, &pmd->classifiers) {
> + if (cls->in_port == in_port) {
> + /* Port classifier exists already */
> + return cls;
> + }
> + }
> + return NULL;
> +}
> +
> +static inline struct dpcls *
> +dp_netdev_pmd_find_dpcls(struct dp_netdev_pmd_thread *pmd,
> + odp_port_t in_port)
>
How about an OVS_REQUIRES(pmd->mutex) here?
> +{
> + struct dpcls *cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
> + uint32_t hash = hash_port_no(in_port);
> +
> + if (!cls) {
> + /* Create new classifier for in_port */
> + cls = xmalloc(sizeof(*cls));
> + dpcls_init(cls);
> + cls->in_port = in_port;
> + cmap_insert(&pmd->classifiers, &cls->node, hash);
> + VLOG_DBG("Creating dpcls %p for in_port %d", cls, in_port);
> + }
> + return cls;
> +}
> +
> static void
> dp_netdev_pmd_remove_flow(struct dp_netdev_pmd_thread *pmd,
> struct dp_netdev_flow *flow)
> OVS_REQUIRES(pmd->flow_mutex)
> {
> struct cmap_node *node = CONST_CAST(struct cmap_node *, &flow->node);
> + struct dpcls *cls;
> + odp_port_t in_port = flow->flow.in_port.odp_port;
>
> - dpcls_remove(&pmd->cls, &flow->cr);
> + cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
> + ovs_assert(cls != NULL);
> + dpcls_remove(cls, &flow->cr);
>
Do you think we should remove the dpcls if it becomes empty?
> cmap_remove(&pmd->flow_table, node, dp_netdev_flow_hash(&flow->
> ufid));
> flow->dead = true;
>
> @@ -1937,15 +1994,20 @@ emc_lookup(struct emc_cache *cache, const struct
> netdev_flow_ke
> }
>
> static struct dp_netdev_flow *
> -dp_netdev_pmd_lookup_flow(const struct dp_netdev_pmd_thread *pmd,
> - const struct netdev_flow_key *key)
> +dp_netdev_pmd_lookup_flow(struct dp_netdev_pmd_thread *pmd,
> + const struct netdev_flow_key *key,
> + int *lookup_num_p)
> {
> - struct dp_netdev_flow *netdev_flow;
> + struct dpcls *cls;
> struct dpcls_rule *rule;
> + odp_port_t in_port = MINIFLOW_GET_U32(&key->mf, in_port);
> + struct dp_netdev_flow *netdev_flow = NULL;
>
> - dpcls_lookup(&pmd->cls, key, &rule, 1);
> - netdev_flow = dp_netdev_flow_cast(rule);
> -
> + cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
> + if (OVS_LIKELY(cls)) {
> + dpcls_lookup(cls, key, &rule, 1, lookup_num_p);
> + netdev_flow = dp_netdev_flow_cast(rule);
> + }
> return netdev_flow;
> }
>
> @@ -2176,6 +2238,8 @@ dp_netdev_flow_add(struct dp_netdev_pmd_thread *pmd,
> {
> struct dp_netdev_flow *flow;
> struct netdev_flow_key mask;
> + struct dpcls *cls;
> + odp_port_t in_port = match->flow.in_port.odp_port;
>
> netdev_flow_mask_init(&mask, match);
> /* Make sure wc does not have metadata. */
> @@ -2194,7 +2258,11 @@ dp_netdev_flow_add(struct dp_netdev_pmd_thread *pmd,
> ovsrcu_set(&flow->actions, dp_netdev_actions_create(actions,
> actions_len));
>
> netdev_flow_key_init_masked(&flow->cr.flow, &match->flow, &mask);
> - dpcls_insert(&pmd->cls, &flow->cr, &mask);
> +
> + /* Select dpcls for in_port. Relies on in_port to be exact match */
> + ovs_assert(match->wc.masks.in_port.odp_port == UINT32_MAX);
> + cls = dp_netdev_pmd_find_dpcls(pmd, in_port);
> + dpcls_insert(cls, &flow->cr, &mask);
>
> cmap_insert(&pmd->flow_table, CONST_CAST(struct cmap_node *,
> &flow->node),
> dp_netdev_flow_hash(&flow->ufid));
> @@ -2275,7 +2343,7 @@ dpif_netdev_flow_put(struct dpif *dpif, const struct
> dpif_flow_pu
> }
>
> ovs_mutex_lock(&pmd->flow_mutex);
> - netdev_flow = dp_netdev_pmd_lookup_flow(pmd, &key);
> + netdev_flow = dp_netdev_pmd_lookup_flow(pmd, &key, NULL);
> if (!netdev_flow) {
> if (put->flags & DPIF_FP_CREATE) {
> if (cmap_count(&pmd->flow_table) < MAX_FLOWS) {
> @@ -3052,6 +3120,7 @@ reload:
> lc = 0;
>
> coverage_try_clear();
> + dp_netdev_pmd_try_optimize(pmd);
> if (!ovsrcu_try_quiesce()) {
> emc_cache_slow_sweep(&pmd->flow_cache);
> }
> @@ -3213,8 +3282,9 @@ dp_netdev_configure_pmd(struct dp_netdev_pmd_thread
> *pmd, struct
> ovs_mutex_init(&pmd->cond_mutex);
> ovs_mutex_init(&pmd->flow_mutex);
> ovs_mutex_init(&pmd->port_mutex);
> - dpcls_init(&pmd->cls);
> cmap_init(&pmd->flow_table);
> + cmap_init(&pmd->classifiers);
> + pmd->next_optimization = time_msec() + DPCLS_OPTIMIZATION_INTERVAL;
> ovs_list_init(&pmd->poll_list);
> hmap_init(&pmd->tx_ports);
> hmap_init(&pmd->port_cache);
> @@ -3230,10 +3300,16 @@ dp_netdev_configure_pmd(struct
> dp_netdev_pmd_thread *pmd, struc
> static void
> dp_netdev_destroy_pmd(struct dp_netdev_pmd_thread *pmd)
> {
> + struct dpcls *cls;
> +
> dp_netdev_pmd_flow_flush(pmd);
> - dpcls_destroy(&pmd->cls);
> hmap_destroy(&pmd->port_cache);
> hmap_destroy(&pmd->tx_ports);
> + /* All flows (including their dpcls_rules) have been deleted already
> */
> + CMAP_FOR_EACH(cls, node, &pmd->classifiers) {
>
space after CMAP_FOR_EACH
> + dpcls_destroy(cls);
> + }
> + cmap_destroy(&pmd->classifiers);
> cmap_destroy(&pmd->flow_table);
> ovs_mutex_destroy(&pmd->flow_mutex);
> latch_destroy(&pmd->exit_latch);
> @@ -4005,7 +4081,7 @@ handle_packet_upcall(struct dp_netdev_pmd_thread
> *pmd, struct dp_
> * to be locking everyone out of making flow installs. If we
> * move to a per-core classifier, it would be reasonable. */
> ovs_mutex_lock(&pmd->flow_mutex);
> - netdev_flow = dp_netdev_pmd_lookup_flow(pmd, key);
> + netdev_flow = dp_netdev_pmd_lookup_flow(pmd, key, NULL);
> if (OVS_LIKELY(!netdev_flow)) {
> netdev_flow = dp_netdev_flow_add(pmd, &match, &ufid,
> add_actions->data,
> @@ -4022,6 +4098,7 @@ fast_path_processing(struct dp_netdev_pmd_thread
> *pmd,
> struct dp_packet_batch *packets_,
> struct netdev_flow_key *keys,
> struct packet_batch_per_flow batches[], size_t
> *n_batches,
> + odp_port_t in_port,
> long long now)
> {
> int cnt = packets_->count;
> @@ -4032,10 +4109,12 @@ fast_path_processing(struct dp_netdev_pmd_thread
> *pmd,
> enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST };
> #endif
> struct dp_packet **packets = packets_->packets;
> + struct dpcls *cls;
> struct dpcls_rule *rules[PKT_ARRAY_SIZE];
> struct dp_netdev *dp = pmd->dp;
> struct emc_cache *flow_cache = &pmd->flow_cache;
> int miss_cnt = 0, lost_cnt = 0;
> + int lookup_cnt = 0, add_lookup_cnt;
> bool any_miss;
> size_t i;
>
> @@ -4043,7 +4122,14 @@ fast_path_processing(struct dp_netdev_pmd_thread
> *pmd,
> /* 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));
> }
> - any_miss = !dpcls_lookup(&pmd->cls, keys, rules, cnt);
> + /* 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);
> + } else {
> + any_miss = true;
> + memset(rules, 0, sizeof(rules));
> + }
> 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;
> @@ -4061,8 +4147,10 @@ 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;
> rules[i] = &netdev_flow->cr;
> continue;
> }
> @@ -4101,6 +4189,7 @@ fast_path_processing(struct dp_netdev_pmd_thread
> *pmd,
> }
>
> dp_netdev_count_packet(pmd, DP_STAT_MASKED_HIT, cnt - miss_cnt);
> + dp_netdev_count_packet(pmd, DP_STAT_LOOKUP_HIT, lookup_cnt);
> dp_netdev_count_packet(pmd, DP_STAT_MISS, miss_cnt);
> dp_netdev_count_packet(pmd, DP_STAT_LOST, lost_cnt);
> }
> @@ -4127,13 +4216,16 @@ dp_netdev_input__(struct dp_netdev_pmd_thread *pmd,
> struct packet_batch_per_flow batches[PKT_ARRAY_SIZE];
> long long now = time_msec();
> size_t newcnt, n_batches, i;
> + odp_port_t in_port;
>
> n_batches = 0;
> newcnt = emc_processing(pmd, packets, keys, batches, &n_batches,
> md_is_valid, port_no);
> if (OVS_UNLIKELY(newcnt)) {
> packets->count = newcnt;
> - fast_path_processing(pmd, packets, keys, batches, &n_batches,
> now);
> + /* Get ingress port from first packet's metadata. */
> + in_port = packets->packets[0]->md.in_port.odp_port;
> + fast_path_processing(pmd, packets, keys, batches, &n_batches,
> in_port, now);
> }
>
> for (i = 0; i < n_batches; i++) {
> @@ -4150,14 +4242,14 @@ dp_netdev_input(struct dp_netdev_pmd_thread *pmd,
> struct dp_packet_batch *packets,
> odp_port_t port_no)
> {
> - dp_netdev_input__(pmd, packets, false, port_no);
> + dp_netdev_input__(pmd, packets, false, port_no);
> }
>
> static void
> dp_netdev_recirculate(struct dp_netdev_pmd_thread *pmd,
> struct dp_packet_batch *packets)
> {
> - dp_netdev_input__(pmd, packets, true, 0);
> + dp_netdev_input__(pmd, packets, true, 0);
> }
>
> struct dp_netdev_execute_aux {
> @@ -4760,6 +4852,8 @@ struct dpcls_subtable {
>
> /* These fields are accessed by readers. */
> struct cmap rules; /* Contains "struct dpcls_rule"s. */
> + uint32_t hit_cnt; /* Number of match hits in subtable in
> current
> + optimization interval. */
> struct netdev_flow_key mask; /* Wildcards for fields (const). */
> /* 'mask' must be the last field, additional space is allocated here.
> */
> };
> @@ -4776,6 +4870,7 @@ dpcls_init(struct dpcls *cls)
> static void
> dpcls_destroy_subtable(struct dpcls *cls, struct dpcls_subtable *subtable)
> {
> + VLOG_DBG("Destroying subtable %p for in_port %d", subtable,
> cls->in_port);
> pvector_remove(&cls->subtables, subtable);
> cmap_remove(&cls->subtables_map, &subtable->cmap_node,
> subtable->mask.hash);
> @@ -4810,9 +4905,13 @@ dpcls_create_subtable(struct dpcls *cls, const
> struct netdev_flo
> subtable = xmalloc(sizeof *subtable
> - sizeof subtable->mask.mf + mask->len);
> cmap_init(&subtable->rules);
> + subtable->hit_cnt = 0;
> netdev_flow_key_clone(&subtable->mask, mask);
> cmap_insert(&cls->subtables_map, &subtable->cmap_node, mask->hash);
> + /* Add the new subtable at the end of the pvector (with no hits yet)
> */
> pvector_insert(&cls->subtables, subtable, 0);
> + VLOG_DBG("Creating %lu. subtable %p for in_port %d",
> + cmap_count(&cls->subtables_map), subtable, cls->in_port);
> pvector_publish(&cls->subtables);
>
> return subtable;
> @@ -4832,6 +4931,43 @@ dpcls_find_subtable(struct dpcls *cls, const struct
> netdev_flow_
> return dpcls_create_subtable(cls, mask);
> }
>
> +
> +/* Periodically sort the dpcls subtable vectors according to hit counts */
> +static inline void
> +dpcls_sort_subtable_vector(struct dpcls *cls)
> + OVS_REQUIRES(pmd->flow_mutex)
>
Clang complains here that 'pmd' is not defined. I think we can remove the
annotation.
> +{
> + struct dpcls_subtable *subtable;
> +
> + struct pvector *pvec = &cls->subtables;
> + PVECTOR_FOR_EACH (subtable, pvec) {
> + pvector_change_priority(pvec, subtable, subtable->hit_cnt);
> + subtable->hit_cnt = 0;
> + }
> + pvector_publish(pvec);
> +}
> +
> +static inline void
> +dp_netdev_pmd_try_optimize(struct dp_netdev_pmd_thread *pmd)
> +{
> + struct dpcls *cls;
> + long long int now = time_msec();
> +
> + if (now > pmd->next_optimization) {
> + /* Try to obtain the flow lock to block out revalidator threads.
> + * If not possible, just try next time. */
> + if (ovs_mutex_trylock(&pmd->flow_mutex)) {
>
ovs_mutex_trylock() returns non-zero if it fails, zero if it succeeds, we
need an extra !
This was reported by a clang warning, I think it's extremely useful to deal
with concurrency.
> + /* Optimize each classifier */
> + CMAP_FOR_EACH(cls, node, &pmd->classifiers) {
> + dpcls_sort_subtable_vector(cls);
> + }
> + ovs_mutex_unlock(&pmd->flow_mutex);
> + /* Start new measuring interval */
> + pmd->next_optimization = now + DPCLS_OPTIMIZATION_INTERVAL;
> + }
> + }
> +}
> +
> /* Insert 'rule' into 'cls'. */
> static void
> dpcls_insert(struct dpcls *cls, struct dpcls_rule *rule,
> @@ -4839,6 +4975,7 @@ dpcls_insert(struct dpcls *cls, struct dpcls_rule
> *rule,
> {
> struct dpcls_subtable *subtable = dpcls_find_subtable(cls, mask);
>
> + /* Refer to subtable's mask, also for later removal. */
> rule->mask = &subtable->mask;
> cmap_insert(&subtable->rules, &rule->cmap_node, rule->flow.hash);
> }
> @@ -4851,10 +4988,11 @@ dpcls_remove(struct dpcls *cls, struct dpcls_rule
> *rule)
>
> ovs_assert(rule->mask);
>
> + /* Get subtable from reference in rule->mask. */
> INIT_CONTAINER(subtable, rule->mask, mask);
> -
> if (cmap_remove(&subtable->rules, &rule->cmap_node, rule->flow.hash)
> == 0) {
> + /* Delete empty subtable. */
> dpcls_destroy_subtable(cls, subtable);
> pvector_publish(&cls->subtables);
> }
> @@ -4889,8 +5027,9 @@ dpcls_rule_matches_key(const struct dpcls_rule *rule,
> *
> * Returns true if all miniflows 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)
> +dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key keys[],
> + struct dpcls_rule **rules, const size_t cnt,
> + int *num_lookups_p)
> {
> /* The received 'cnt' miniflows are the search-keys that will be
> processed
> * in batches of 16 elements. N_MAPS will contain the number of these
> @@ -4913,6 +5052,8 @@ dpcls_lookup(const struct dpcls *cls, const struct
> netdev_flow_ke
> }
> memset(rules, 0, cnt * sizeof *rules);
>
> + int lookups_match = 0, subtable_pos = 1;
> +
> /* The Datapath classifier - aka dpcls - is composed of subtables.
> * Subtables are dynamically created as needed when new rules are
> inserted.
> * Each subtable collects rules with matches on a specific subset of
> packet
> @@ -4959,6 +5100,10 @@ dpcls_lookup(const struct dpcls *cls, const struct
> netdev_flow_k
> CMAP_NODE_FOR_EACH (rule, cmap_node, nodes[i]) {
> if (OVS_LIKELY(dpcls_rule_matches_key(rule,
> &mkeys[i]))) {
> mrules[i] = rule;
> + /* Even at 20 Mpps the 32-bit hit_cnt cannot wrap
> + * within one second optimization interval */
> + subtable->hit_cnt++;
> + lookups_match += subtable_pos;
> goto next;
> }
> }
> @@ -4972,8 +5117,15 @@ dpcls_lookup(const struct dpcls *cls, const struct
> netdev_flow_k
> remains |= maps[m];
> }
> if (!remains) {
> + if (num_lookups_p) {
> + *num_lookups_p = lookups_match;
> + }
> return true; /* All found. */
> }
> + subtable_pos++;
> + }
> + if (num_lookups_p) {
> + *num_lookups_p = lookups_match;
> }
> return false; /* Some misses. */
> }
> diff --git a/tests/pmd.at b/tests/pmd.at
> index 2188ea1..d1a2591 100644
> --- a/tests/pmd.at
> +++ b/tests/pmd.at
> @@ -168,10 +168,11 @@ dummy@ovs-dummy: hit:0 missed:0
> p0 7/1: (dummy-pmd: configured_rx_queues=4,
> configured_tx_queues=<clear
> ])
>
> -AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed
> SED_NUMA_CORE_PATTERN | sed '/cy
> +AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed
> SED_NUMA_CORE_PATTERN | sed '/cy
>
As I sais above, this line is truncated
> pmd thread numa_id <cleared> core_id <cleared>:
> emc hits:0
> megaflow hits:0
> + avg. subtable lookups per hit:0.00
> miss:0
> lost:0
> ])
> @@ -194,10 +195,11 @@ AT_CHECK([cat ovs-vswitchd.log | filter_flow_install
> | strip_xout
> recirc_id(0),in_port(1),eth(src=50:54:00:00:00:77,dst=50:
> 54:00:00:01:78),eth_type(0x08
> ])
>
> -AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed
> SED_NUMA_CORE_PATTERN | sed '/cy
> +AT_CHECK([ovs-appctl dpif-netdev/pmd-stats-show | sed
> SED_NUMA_CORE_PATTERN | sed '/cy
>
ditto
> pmd thread numa_id <cleared> core_id <cleared>:
> emc hits:19
> megaflow hits:0
> + avg. subtable lookups per hit:0.00
> miss:1
> lost:0
> ])
> _______________________________________________
> dev mailing list
> [email protected]
> http://openvswitch.org/mailman/listinfo/dev
>
_______________________________________________
dev mailing list
[email protected]
http://openvswitch.org/mailman/listinfo/dev