[ovs-dev] [PATCH v2] datapath: NUMA node flow stats.

Jarno Rajahalme jrajahalme at nicira.com
Thu Jan 23 19:06:37 UTC 2014


Keep kernel flow stats for each NUMA node rather than each (logical)
CPU.  This almost removes the kernel-side OVS locking overhead
otherwise on the top of perf report and allows OVS to scale better
with higher number of threads.

With 9 handlers and 4 revalidators netperf TCP_CRR test flow setup
rate doubles on a server with two hyper-threaded physical CPUs (16
logical cores each) compared to the current OVS master.  Tested with
non-trivial flow table with a TCP port match rule forcing all new
connections with unique port numbers to OVS userspace.  The IP
addresses are still wildcarded, so the kernel flows are not considered
as exact match 5-tuple flows.  This type of flows can be expected to
appear in large numbers as the result of more effective wildcarding
made possible by improvements in OVS userspace flow classifier.

Perf results for this test (master):

Events: 305K cycles
+   8.43%     ovs-vswitchd  [kernel.kallsyms]   [k] mutex_spin_on_owner
+   5.64%     ovs-vswitchd  [kernel.kallsyms]   [k] __ticket_spin_lock
+   4.75%     ovs-vswitchd  ovs-vswitchd        [.] find_match_wc
+   3.32%     ovs-vswitchd  libpthread-2.15.so  [.] pthread_mutex_lock
+   2.61%     ovs-vswitchd  [kernel.kallsyms]   [k] pcpu_alloc_area
+   2.19%     ovs-vswitchd  ovs-vswitchd        [.] flow_hash_in_minimask_range
+   2.03%          swapper  [kernel.kallsyms]   [k] intel_idle
+   1.84%     ovs-vswitchd  libpthread-2.15.so  [.] pthread_mutex_unlock
+   1.64%     ovs-vswitchd  ovs-vswitchd        [.] classifier_lookup
+   1.58%     ovs-vswitchd  libc-2.15.so        [.] 0x7f4e6
+   1.07%     ovs-vswitchd  [kernel.kallsyms]   [k] memset
+   1.03%          netperf  [kernel.kallsyms]   [k] __ticket_spin_lock
+   0.92%          swapper  [kernel.kallsyms]   [k] __ticket_spin_lock
...

And after this patch:

Events: 356K cycles
+   6.85%     ovs-vswitchd  ovs-vswitchd        [.] find_match_wc
+   4.63%     ovs-vswitchd  libpthread-2.15.so  [.] pthread_mutex_lock
+   3.06%     ovs-vswitchd  [kernel.kallsyms]   [k] __ticket_spin_lock
+   2.81%     ovs-vswitchd  ovs-vswitchd        [.] flow_hash_in_minimask_range
+   2.51%     ovs-vswitchd  libpthread-2.15.so  [.] pthread_mutex_unlock
+   2.27%     ovs-vswitchd  ovs-vswitchd        [.] classifier_lookup
+   1.84%     ovs-vswitchd  libc-2.15.so        [.] 0x15d30f
+   1.74%     ovs-vswitchd  [kernel.kallsyms]   [k] mutex_spin_on_owner
+   1.47%          swapper  [kernel.kallsyms]   [k] intel_idle
+   1.34%     ovs-vswitchd  ovs-vswitchd        [.] flow_hash_in_minimask
+   1.33%     ovs-vswitchd  ovs-vswitchd        [.] rule_actions_unref
+   1.16%     ovs-vswitchd  ovs-vswitchd        [.] hindex_node_with_hash
+   1.16%     ovs-vswitchd  ovs-vswitchd        [.] do_xlate_actions
+   1.09%     ovs-vswitchd  ovs-vswitchd        [.] ofproto_rule_ref
+   1.01%          netperf  [kernel.kallsyms]   [k] __ticket_spin_lock
...

There is a small increase in kernel spinlock overhead due to the same
spinlock being shared between multiple cores of the same physical CPU,
but that is barely visible in the netperf TCP_CRR test performance
(maybe ~1% performance drop, hard to tell exactly due to variance in the test
results), when testing for kernel module throughput (with no userspace
activity, max 8 kernel flows).

On flow setup, a single stats instance is allocated (for the NUMA node
0).  As CPUs from multiple NUMA nodes start updating stats, new
NUMA-node specific stats instances are allocated.  This allocation on
the packet processing code path is made to never sleep or look for
emergency memory pools, minimizing the allocation latency.  If the
allocation fails, the existing preallocated stats instance is used.
Also, if only CPUs from one NUMA-node are updating the preallocated
stats instance, no additional stats instances are allocated.  This
eliminates the need to pre-allocate stats instances that will not be
used, also relieving the stats reader from the burden of reading stats
that are never used.  Finally, this allocation strategy allows the
removal of the existing exact-5-tuple heuristics.

Signed-off-by: Jarno Rajahalme <jrajahalme at nicira.com>
---
 datapath/datapath.c     |   53 +++++++++++++--
 datapath/flow.c         |  171 ++++++++++++++++++++++++-----------------------
 datapath/flow.h         |   21 +++---
 datapath/flow_netlink.c |   53 +--------------
 datapath/flow_netlink.h |    1 -
 datapath/flow_table.c   |   68 ++++++++++++-------
 datapath/flow_table.h   |    5 +-
 7 files changed, 192 insertions(+), 180 deletions(-)

diff --git a/datapath/datapath.c b/datapath/datapath.c
index c756e2f..5f85d08 100644
--- a/datapath/datapath.c
+++ b/datapath/datapath.c
@@ -214,6 +214,34 @@ void ovs_dp_detach_port(struct vport *p)
 	ovs_vport_del(p);
 }
 
+static void ovs_flow_stats_prefetch(const struct sw_flow *flow)
+{
+	int node = numa_node_id();
+
+	if (flow->stats[node])
+		spin_lock_prefetch(&flow->stats[node]->lock);
+	else
+		spin_lock_prefetch(&flow->stats[0]->lock);
+}
+
+static void get_skb_stats(struct flow_stats *stat, const struct sw_flow *flow,
+			  const struct sk_buff *skb)
+{
+	ovs_flow_stats_prefetch(flow); /* Prepare for writing later. */
+
+	stat->used = jiffies;
+	stat->packet_count = 1;
+	stat->byte_count = skb->len;
+	stat->tcp_flags = 0;
+
+	if ((flow->key.eth.type == htons(ETH_P_IP) ||
+	     flow->key.eth.type == htons(ETH_P_IPV6)) &&
+	    flow->key.ip.proto == IPPROTO_TCP &&
+	    likely(skb->len >= skb_transport_offset(skb) + sizeof(struct tcphdr))) {
+		stat->tcp_flags = TCP_FLAGS_BE16(tcp_hdr(skb));
+	}
+}
+
 /* Must be called with rcu_read_lock. */
 void ovs_dp_process_received_packet(struct vport *p, struct sk_buff *skb)
 {
@@ -221,6 +249,7 @@ void ovs_dp_process_received_packet(struct vport *p, struct sk_buff *skb)
 	struct sw_flow *flow;
 	struct dp_stats_percpu *stats;
 	struct sw_flow_key key;
+	struct flow_stats stat;
 	u64 *stats_counter;
 	u32 n_mask_hit;
 	int error;
@@ -252,8 +281,9 @@ void ovs_dp_process_received_packet(struct vport *p, struct sk_buff *skb)
 	OVS_CB(skb)->flow = flow;
 	OVS_CB(skb)->pkt_key = &key;
 
-	ovs_flow_stats_update(OVS_CB(skb)->flow, skb);
+	get_skb_stats(&stat, flow, skb);
 	ovs_execute_actions(dp, skb);
+	ovs_flow_stats_update(flow, &stat);
 	stats_counter = &stats->n_hit;
 
 out:
@@ -522,7 +552,7 @@ static int ovs_packet_cmd_execute(struct sk_buff *skb, struct genl_info *info)
 		packet->protocol = htons(ETH_P_802_2);
 
 	/* Build an sw_flow for sending this packet. */
-	flow = ovs_flow_alloc(false);
+	flow = ovs_flow_alloc();
 	err = PTR_ERR(flow);
 	if (IS_ERR(flow))
 		goto err_kfree_skb;
@@ -650,6 +680,14 @@ static size_t ovs_flow_cmd_msg_size(const struct sw_flow_actions *acts)
 		+ nla_total_size(acts->actions_len); /* OVS_FLOW_ATTR_ACTIONS */
 }
 
+static void ovs_flow_stats_prefetch_all(struct sw_flow *flow)
+{
+	int node;
+	for (node = 0; node < ovs_numa_nodes; node++)
+		if (flow->stats[node])
+			spin_lock_prefetch(&flow->stats[node]->lock);
+}
+
 /* Called with ovs_mutex. */
 static int ovs_flow_cmd_fill_info(struct sw_flow *flow, struct datapath *dp,
 				  struct sk_buff *skb, u32 portid,
@@ -664,6 +702,8 @@ static int ovs_flow_cmd_fill_info(struct sw_flow *flow, struct datapath *dp,
 	struct nlattr *nla;
 	int err;
 
+	ovs_flow_stats_prefetch_all(flow);
+
 	ovs_header = genlmsg_put(skb, portid, seq, &dp_flow_genl_family, flags, cmd);
 	if (!ovs_header)
 		return -EMSGSIZE;
@@ -780,7 +820,6 @@ static int ovs_flow_cmd_new_or_set(struct sk_buff *skb, struct genl_info *info)
 	struct datapath *dp;
 	struct sw_flow_actions *acts = NULL;
 	struct sw_flow_match match;
-	bool exact_5tuple;
 	int error;
 
 	/* Extract key. */
@@ -789,7 +828,7 @@ static int ovs_flow_cmd_new_or_set(struct sk_buff *skb, struct genl_info *info)
 		goto error;
 
 	ovs_match_init(&match, &key, &mask);
-	error = ovs_nla_get_match(&match, &exact_5tuple,
+	error = ovs_nla_get_match(&match,
 				  a[OVS_FLOW_ATTR_KEY], a[OVS_FLOW_ATTR_MASK]);
 	if (error)
 		goto error;
@@ -828,7 +867,7 @@ static int ovs_flow_cmd_new_or_set(struct sk_buff *skb, struct genl_info *info)
 			goto err_unlock_ovs;
 
 		/* Allocate flow. */
-		flow = ovs_flow_alloc(!exact_5tuple);
+		flow = ovs_flow_alloc();
 		if (IS_ERR(flow)) {
 			error = PTR_ERR(flow);
 			goto err_unlock_ovs;
@@ -915,7 +954,7 @@ static int ovs_flow_cmd_get(struct sk_buff *skb, struct genl_info *info)
 	}
 
 	ovs_match_init(&match, &key, NULL);
-	err = ovs_nla_get_match(&match, NULL, a[OVS_FLOW_ATTR_KEY], NULL);
+	err = ovs_nla_get_match(&match, a[OVS_FLOW_ATTR_KEY], NULL);
 	if (err)
 		return err;
 
@@ -969,7 +1008,7 @@ static int ovs_flow_cmd_del(struct sk_buff *skb, struct genl_info *info)
 	}
 
 	ovs_match_init(&match, &key, NULL);
-	err = ovs_nla_get_match(&match, NULL, a[OVS_FLOW_ATTR_KEY], NULL);
+	err = ovs_nla_get_match(&match, a[OVS_FLOW_ATTR_KEY], NULL);
 	if (err)
 		goto unlock;
 
diff --git a/datapath/flow.c b/datapath/flow.c
index 8be3801..9a66eb8 100644
--- a/datapath/flow.c
+++ b/datapath/flow.c
@@ -47,6 +47,26 @@
 
 #include "vlan.h"
 
+static inline bool alloc_stats(struct sw_flow *flow,
+			       const struct flow_stats *stat, int node)
+{
+	struct flow_stats *stats;
+	/* Alloc stats for the 'node', but with minimal latency. */
+	stats = kmem_cache_alloc_node(flow_stats_cache, GFP_THISNODE |
+				      __GFP_NOTRACK | __GFP_NOMEMALLOC, node);
+	if (likely(stats)) {
+		stats->used = stat->used;
+		stats->packet_count = stat->packet_count;
+		stats->byte_count = stat->byte_count;
+		stats->tcp_flags = stat->tcp_flags;
+		spin_lock_init(&stats->lock);
+		stats->first_writer = node;
+
+		flow->stats[node] = stats;
+	}
+	return stats != NULL;
+}
+
 u64 ovs_flow_used_time(unsigned long flow_jiffies)
 {
 	struct timespec cur_ts;
@@ -60,115 +80,96 @@ u64 ovs_flow_used_time(unsigned long flow_jiffies)
 	return cur_ms - idle_ms;
 }
 
-#define TCP_FLAGS_BE16(tp) (*(__be16 *)&tcp_flag_word(tp) & htons(0x0FFF))
-
-void ovs_flow_stats_update(struct sw_flow *flow, struct sk_buff *skb)
+void ovs_flow_stats_update(struct sw_flow *flow, const struct flow_stats *stat)
 {
-	struct flow_stats *stats;
-	__be16 tcp_flags = 0;
-
-	if (!flow->stats.is_percpu)
-		stats = flow->stats.stat;
-	else
-		stats = this_cpu_ptr(flow->stats.cpu_stats);
-
-	if ((flow->key.eth.type == htons(ETH_P_IP) ||
-	     flow->key.eth.type == htons(ETH_P_IPV6)) &&
-	    flow->key.ip.proto == IPPROTO_TCP &&
-	    likely(skb->len >= skb_transport_offset(skb) + sizeof(struct tcphdr))) {
-		tcp_flags = TCP_FLAGS_BE16(tcp_hdr(skb));
-	}
+	int node = numa_node_id();
+	struct flow_stats *stats = flow->stats[node];
 
-	spin_lock(&stats->lock);
-	stats->used = jiffies;
-	stats->packet_count++;
-	stats->byte_count += skb->len;
-	stats->tcp_flags |= tcp_flags;
-	spin_unlock(&stats->lock);
-}
+	if (unlikely(!stats)) {
+		bool done = false;
+		stats = flow->stats[0]; /* Pre-allocated. */
 
-static void stats_read(struct flow_stats *stats, bool lock_bh,
-		       struct ovs_flow_stats *ovs_stats,
-		       unsigned long *used, __be16 *tcp_flags)
-{
-	if (lock_bh)
-		spin_lock_bh(&stats->lock);
-	else
 		spin_lock(&stats->lock);
+		/* If the current NUMA-node is the only writer on the 'stats'
+		 * keep using it.
+		 * A previous locker may have already allocated the stats. */
+		if (stats->first_writer != node && stats->first_writer >= 0
+		    && likely(!flow->stats[node]))
+			done = alloc_stats(flow, stat, node);
+		if (!done)
+			goto update;
+		goto unlock;
+	}
 
-	if (time_after(stats->used, *used))
-		*used = stats->used;
-	*tcp_flags |= stats->tcp_flags;
-	ovs_stats->n_packets += stats->packet_count;
-	ovs_stats->n_bytes += stats->byte_count;
-
-	if (lock_bh)
-		spin_unlock_bh(&stats->lock);
-	else
-		spin_unlock(&stats->lock);
+	spin_lock(&stats->lock);
+update:
+	stats->used = stat->used;
+	stats->packet_count += stat->packet_count;
+	stats->byte_count += stat->byte_count;
+	stats->tcp_flags |= stat->tcp_flags;
+	if (unlikely(stats->first_writer < 0))
+		stats->first_writer = node;
+unlock:
+	spin_unlock(&stats->lock);
 }
 
 void ovs_flow_stats_get(struct sw_flow *flow, struct ovs_flow_stats *ovs_stats,
 			unsigned long *used, __be16 *tcp_flags)
 {
-	int cpu, cur_cpu;
-
-	*used = 0;
-	*tcp_flags = 0;
-	memset(ovs_stats, 0, sizeof(*ovs_stats));
+	int node, current_node = numa_node_id();
+	struct flow_stats *stats;
 
-	if (!flow->stats.is_percpu) {
-		stats_read(flow->stats.stat, true, ovs_stats, used, tcp_flags);
+	/* Start from the current node, if any, as it is more likely to be
+	 * in the cache by now.  Avoid locking when there are no stats. */
+	stats = flow->stats[current_node];
+	if (stats && likely(stats->packet_count)) {
+		spin_lock_bh(&stats->lock);
+		*used = stats->used;
+		*tcp_flags = stats->tcp_flags;
+		ovs_stats->n_packets = stats->packet_count;
+		ovs_stats->n_bytes = stats->byte_count;
+		spin_unlock_bh(&stats->lock);
 	} else {
-		cur_cpu = get_cpu();
-
-		for_each_possible_cpu(cpu) {
-			struct flow_stats *stats;
-			bool lock_bh;
+		*used = 0;
+		*tcp_flags = 0;
+		ovs_stats->n_packets = 0;
+		ovs_stats->n_bytes = 0;
+	}
 
-			stats = per_cpu_ptr(flow->stats.cpu_stats, cpu);
-			lock_bh = (cpu == cur_cpu);
-			stats_read(stats, lock_bh, ovs_stats, used, tcp_flags);
+	/* Collect per NUMA-node stats from other nodes. */
+	for (node = 0; node < ovs_numa_nodes; node++) {
+		if (node == current_node)
+			continue; /* Done already. */
+		stats = flow->stats[node];
+		if (stats && likely(stats->packet_count)) {
+			spin_lock_bh(&stats->lock);
+			if (time_after(stats->used, *used))
+				*used = stats->used;
+			*tcp_flags |= stats->tcp_flags;
+			ovs_stats->n_packets += stats->packet_count;
+			ovs_stats->n_bytes += stats->byte_count;
+			spin_unlock_bh(&stats->lock);
 		}
-		put_cpu();
 	}
 }
 
-static void stats_reset(struct flow_stats *stats, bool lock_bh)
+static void stats_reset(struct flow_stats *stats)
 {
-	if (lock_bh)
+	if (stats) {
 		spin_lock_bh(&stats->lock);
-	else
-		spin_lock(&stats->lock);
-
-	stats->used = 0;
-	stats->packet_count = 0;
-	stats->byte_count = 0;
-	stats->tcp_flags = 0;
-
-	if (lock_bh)
+		stats->used = 0;
+		stats->packet_count = 0;
+		stats->byte_count = 0;
+		stats->tcp_flags = 0;
 		spin_unlock_bh(&stats->lock);
-	else
-		spin_unlock(&stats->lock);
+	}
 }
 
 void ovs_flow_stats_clear(struct sw_flow *flow)
 {
-	int cpu, cur_cpu;
-
-	if (!flow->stats.is_percpu) {
-		stats_reset(flow->stats.stat, true);
-	} else {
-		cur_cpu = get_cpu();
-
-		for_each_possible_cpu(cpu) {
-			bool lock_bh;
-
-			lock_bh = (cpu == cur_cpu);
-			stats_reset(per_cpu_ptr(flow->stats.cpu_stats, cpu), lock_bh);
-		}
-		put_cpu();
-	}
+	int node;
+	for (node = 0; node < ovs_numa_nodes; node++)
+		stats_reset(flow->stats[node]);
 }
 
 static int check_header(struct sk_buff *skb, int len)
diff --git a/datapath/flow.h b/datapath/flow.h
index eafcfd8..a12dc0d 100644
--- a/datapath/flow.h
+++ b/datapath/flow.h
@@ -67,6 +67,8 @@ static inline void ovs_flow_tun_key_init(struct ovs_key_ipv4_tunnel *tun_key,
 	       sizeof(*tun_key) - OVS_TUNNEL_KEY_SIZE);
 }
 
+#define TCP_FLAGS_BE16(tp) (*(__be16 *)&tcp_flag_word(tp) & htons(0x0FFF))
+
 struct sw_flow_key {
 	struct ovs_key_ipv4_tunnel tun_key;  /* Encapsulating tunnel key. */
 	struct {
@@ -155,15 +157,9 @@ struct flow_stats {
 	unsigned long used;		/* Last used time (in jiffies). */
 	spinlock_t lock;		/* Lock for atomic stats update. */
 	__be16 tcp_flags;		/* Union of seen TCP flags. */
-};
-
-struct sw_flow_stats {
-	bool is_percpu;
-	union {
-		struct flow_stats *stat;
-		struct flow_stats __percpu *cpu_stats;
-	};
-};
+	int first_writer;		/* NUMA-node id of the first writer.
+					 * Meaningful for 'stats[0]' only. */
+} ____cacheline_aligned_in_smp;
 
 struct sw_flow {
 	struct rcu_head rcu;
@@ -174,7 +170,10 @@ struct sw_flow {
 	struct sw_flow_key unmasked_key;
 	struct sw_flow_mask *mask;
 	struct sw_flow_actions __rcu *sf_acts;
-	struct sw_flow_stats stats;
+	struct flow_stats *stats[];	/* One for each NUMA node.  First one
+					 * is allocated at flow creation time,
+					 * the rest are allocated on demand
+					 * while holding the 'stats[0].lock'. */
 };
 
 struct arp_eth_header {
@@ -191,7 +190,7 @@ struct arp_eth_header {
 	unsigned char       ar_tip[4];		/* target IP address        */
 } __packed;
 
-void ovs_flow_stats_update(struct sw_flow *flow, struct sk_buff *skb);
+void ovs_flow_stats_update(struct sw_flow *flow, const struct flow_stats *stat);
 void ovs_flow_stats_get(struct sw_flow *flow, struct ovs_flow_stats *stats,
 			unsigned long *used, __be16 *tcp_flags);
 void ovs_flow_stats_clear(struct sw_flow *flow);
diff --git a/datapath/flow_netlink.c b/datapath/flow_netlink.c
index 9b26528..8b793f7 100644
--- a/datapath/flow_netlink.c
+++ b/datapath/flow_netlink.c
@@ -266,20 +266,6 @@ static bool is_all_zero(const u8 *fp, size_t size)
 	return true;
 }
 
-static bool is_all_set(const u8 *fp, size_t size)
-{
-	int i;
-
-	if (!fp)
-		return false;
-
-	for (i = 0; i < size; i++)
-		if (fp[i] != 0xff)
-			return false;
-
-	return true;
-}
-
 static int __parse_flow_nlattrs(const struct nlattr *attr,
 				const struct nlattr *a[],
 				u64 *attrsp, bool nz)
@@ -501,7 +487,7 @@ static int metadata_from_nlattrs(struct sw_flow_match *match,  u64 *attrs,
 	return 0;
 }
 
-static int ovs_key_from_nlattrs(struct sw_flow_match *match,  bool *exact_5tuple,
+static int ovs_key_from_nlattrs(struct sw_flow_match *match,
 				u64 attrs, const struct nlattr **a,
 				bool is_mask)
 {
@@ -560,11 +546,6 @@ static int ovs_key_from_nlattrs(struct sw_flow_match *match,  bool *exact_5tuple
 		SW_FLOW_KEY_PUT(match, eth.type, htons(ETH_P_802_2), is_mask);
 	}
 
-	if (is_mask && exact_5tuple) {
-		if (match->mask->key.eth.type != htons(0xffff))
-			*exact_5tuple = false;
-	}
-
 	if (attrs & (1ULL << OVS_KEY_ATTR_IPV4)) {
 		const struct ovs_key_ipv4 *ipv4_key;
 
@@ -587,13 +568,6 @@ static int ovs_key_from_nlattrs(struct sw_flow_match *match,  bool *exact_5tuple
 		SW_FLOW_KEY_PUT(match, ipv4.addr.dst,
 				ipv4_key->ipv4_dst, is_mask);
 		attrs &= ~(1ULL << OVS_KEY_ATTR_IPV4);
-
-		if (is_mask && exact_5tuple && *exact_5tuple) {
-			if (ipv4_key->ipv4_proto != 0xff ||
-			    ipv4_key->ipv4_src != htonl(0xffffffff) ||
-			    ipv4_key->ipv4_dst != htonl(0xffffffff))
-				*exact_5tuple = false;
-		}
 	}
 
 	if (attrs & (1ULL << OVS_KEY_ATTR_IPV6)) {
@@ -625,13 +599,6 @@ static int ovs_key_from_nlattrs(struct sw_flow_match *match,  bool *exact_5tuple
 				is_mask);
 
 		attrs &= ~(1ULL << OVS_KEY_ATTR_IPV6);
-
-		if (is_mask && exact_5tuple && *exact_5tuple) {
-			if (ipv6_key->ipv6_proto != 0xff ||
-			    !is_all_set((u8 *)ipv6_key->ipv6_src, sizeof(match->key->ipv6.addr.src)) ||
-			    !is_all_set((u8 *)ipv6_key->ipv6_dst, sizeof(match->key->ipv6.addr.dst)))
-				*exact_5tuple = false;
-		}
 	}
 
 	if (attrs & (1ULL << OVS_KEY_ATTR_ARP)) {
@@ -674,11 +641,6 @@ static int ovs_key_from_nlattrs(struct sw_flow_match *match,  bool *exact_5tuple
 					tcp_key->tcp_dst, is_mask);
 		}
 		attrs &= ~(1ULL << OVS_KEY_ATTR_TCP);
-
-		if (is_mask && exact_5tuple && *exact_5tuple &&
-		    (tcp_key->tcp_src != htons(0xffff) ||
-		     tcp_key->tcp_dst != htons(0xffff)))
-			*exact_5tuple = false;
 	}
 
 	if (attrs & (1ULL << OVS_KEY_ATTR_TCP_FLAGS)) {
@@ -710,11 +672,6 @@ static int ovs_key_from_nlattrs(struct sw_flow_match *match,  bool *exact_5tuple
 					udp_key->udp_dst, is_mask);
 		}
 		attrs &= ~(1ULL << OVS_KEY_ATTR_UDP);
-
-		if (is_mask && exact_5tuple && *exact_5tuple &&
-		    (udp_key->udp_src != htons(0xffff) ||
-		     udp_key->udp_dst != htons(0xffff)))
-			*exact_5tuple = false;
 	}
 
 	if (attrs & (1ULL << OVS_KEY_ATTR_SCTP)) {
@@ -800,7 +757,6 @@ static void sw_flow_mask_set(struct sw_flow_mask *mask,
  * attribute specifies the mask field of the wildcarded flow.
  */
 int ovs_nla_get_match(struct sw_flow_match *match,
-		      bool *exact_5tuple,
 		      const struct nlattr *key,
 		      const struct nlattr *mask)
 {
@@ -848,13 +804,10 @@ int ovs_nla_get_match(struct sw_flow_match *match,
 		}
 	}
 
-	err = ovs_key_from_nlattrs(match, NULL, key_attrs, a, false);
+	err = ovs_key_from_nlattrs(match, key_attrs, a, false);
 	if (err)
 		return err;
 
-	if (exact_5tuple)
-		*exact_5tuple = true;
-
 	if (mask) {
 		err = parse_flow_mask_nlattrs(mask, a, &mask_attrs);
 		if (err)
@@ -892,7 +845,7 @@ int ovs_nla_get_match(struct sw_flow_match *match,
 			}
 		}
 
-		err = ovs_key_from_nlattrs(match, exact_5tuple, mask_attrs, a, true);
+		err = ovs_key_from_nlattrs(match, mask_attrs, a, true);
 		if (err)
 			return err;
 	} else {
diff --git a/datapath/flow_netlink.h b/datapath/flow_netlink.h
index b31fbe2..4401510 100644
--- a/datapath/flow_netlink.h
+++ b/datapath/flow_netlink.h
@@ -45,7 +45,6 @@ int ovs_nla_put_flow(const struct sw_flow_key *,
 int ovs_nla_get_flow_metadata(struct sw_flow *flow,
 			      const struct nlattr *attr);
 int ovs_nla_get_match(struct sw_flow_match *match,
-		      bool *exact_5tuple,
 		      const struct nlattr *,
 		      const struct nlattr *);
 
diff --git a/datapath/flow_table.c b/datapath/flow_table.c
index 2e8557c..0b53455 100644
--- a/datapath/flow_table.c
+++ b/datapath/flow_table.c
@@ -50,6 +50,9 @@
 #define REHASH_INTERVAL		(10 * 60 * HZ)
 
 static struct kmem_cache *flow_cache;
+struct kmem_cache *flow_stats_cache;
+
+int ovs_numa_nodes = 0;
 
 static u16 range_n_bytes(const struct sw_flow_key_range *range)
 {
@@ -72,10 +75,10 @@ void ovs_flow_mask_key(struct sw_flow_key *dst, const struct sw_flow_key *src,
 		*d++ = *s++ & *m++;
 }
 
-struct sw_flow *ovs_flow_alloc(bool percpu_stats)
+struct sw_flow *ovs_flow_alloc(void)
 {
 	struct sw_flow *flow;
-	int cpu;
+	int node;
 
 	flow = kmem_cache_alloc(flow_cache, GFP_KERNEL);
 	if (!flow)
@@ -84,26 +87,18 @@ struct sw_flow *ovs_flow_alloc(bool percpu_stats)
 	flow->sf_acts = NULL;
 	flow->mask = NULL;
 
-	flow->stats.is_percpu = percpu_stats;
-
-	if (!percpu_stats) {
-		flow->stats.stat = kzalloc(sizeof(*flow->stats.stat), GFP_KERNEL);
-		if (!flow->stats.stat)
-			goto err;
+	/* Initialize the default stat node. */
+	flow->stats[0] = kmem_cache_alloc_node(flow_stats_cache,
+					       GFP_KERNEL | __GFP_ZERO, 0);
+	if (!flow->stats[0])
+		goto err;
 
-		spin_lock_init(&flow->stats.stat->lock);
-	} else {
-		flow->stats.cpu_stats = alloc_percpu(struct flow_stats);
-		if (!flow->stats.cpu_stats)
-			goto err;
+	spin_lock_init(&flow->stats[0]->lock);
+	flow->stats[0]->first_writer = -1;
 
-		for_each_possible_cpu(cpu) {
-			struct flow_stats *cpu_stats;
+	for (node = 1; node < ovs_numa_nodes; node++)
+		flow->stats[node] = NULL;
 
-			cpu_stats = per_cpu_ptr(flow->stats.cpu_stats, cpu);
-			spin_lock_init(&cpu_stats->lock);
-		}
-	}
 	return flow;
 err:
 	kmem_cache_free(flow_cache, flow);
@@ -140,11 +135,12 @@ static struct flex_array *alloc_buckets(unsigned int n_buckets)
 
 static void flow_free(struct sw_flow *flow)
 {
+	int node;
+
 	kfree((struct sf_flow_acts __force *)flow->sf_acts);
-	if (flow->stats.is_percpu)
-		free_percpu(flow->stats.cpu_stats);
-	else
-		kfree(flow->stats.stat);
+	for (node = 0; node < ovs_numa_nodes; node++)
+		if (flow->stats[node])
+			kmem_cache_free(flow_stats_cache, flow->stats[node]);
 	kmem_cache_free(flow_cache, flow);
 }
 
@@ -601,19 +597,41 @@ int ovs_flow_tbl_insert(struct flow_table *table, struct sw_flow *flow,
  * Returns zero if successful or a negative error code. */
 int ovs_flow_init(void)
 {
+	int node;
+
 	BUILD_BUG_ON(__alignof__(struct sw_flow_key) % __alignof__(long));
 	BUILD_BUG_ON(sizeof(struct sw_flow_key) % sizeof(long));
 
-	flow_cache = kmem_cache_create("sw_flow", sizeof(struct sw_flow), 0,
-					0, NULL);
+	/* Count the NUMA nodes. */
+	for_each_node_with_cpus(node) {
+		if (node >= ovs_numa_nodes)
+			ovs_numa_nodes = node + 1;
+	}
+	pr_info("ovs_flow_init: ovs_numa_nodes: %d.\n", ovs_numa_nodes);
+
+	flow_cache = kmem_cache_create("sw_flow", sizeof(struct sw_flow)
+				       + (ovs_numa_nodes
+					  * sizeof(struct flow_stats *)),
+				       __alignof__(struct sw_flow),
+				       SLAB_HWCACHE_ALIGN, NULL);
 	if (flow_cache == NULL)
 		return -ENOMEM;
 
+	flow_stats_cache
+		= kmem_cache_create("sw_flow_stats", sizeof(struct flow_stats),
+				    L1_CACHE_BYTES, SLAB_HWCACHE_ALIGN, NULL);
+	if (flow_stats_cache == NULL) {
+		kmem_cache_destroy(flow_cache);
+		flow_cache = NULL;
+		return -ENOMEM;
+	}
+
 	return 0;
 }
 
 /* Uninitializes the flow module. */
 void ovs_flow_exit(void)
 {
+	kmem_cache_destroy(flow_stats_cache);
 	kmem_cache_destroy(flow_cache);
 }
diff --git a/datapath/flow_table.h b/datapath/flow_table.h
index baaeb10..f6a2a45 100644
--- a/datapath/flow_table.h
+++ b/datapath/flow_table.h
@@ -52,10 +52,13 @@ struct flow_table {
 	unsigned int count;
 };
 
+extern int ovs_numa_nodes;
+extern struct kmem_cache *flow_stats_cache;
+
 int ovs_flow_init(void);
 void ovs_flow_exit(void);
 
-struct sw_flow *ovs_flow_alloc(bool percpu_stats);
+struct sw_flow *ovs_flow_alloc(void);
 void ovs_flow_free(struct sw_flow *, bool deferred);
 
 int ovs_flow_tbl_init(struct flow_table *);
-- 
1.7.10.4




More information about the dev mailing list