All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH v5] rfs: Receive Flow Steering
@ 2010-04-16  5:47 Tom Herbert
  2010-04-16  6:33 ` David Miller
  2010-04-16 11:57 ` Andi Kleen
  0 siblings, 2 replies; 60+ messages in thread
From: Tom Herbert @ 2010-04-16  5:47 UTC (permalink / raw)
  To: davem, netdev, eric.dumazet

Version 5 of RFS:
- Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
static function.
- Apply limits to rps_sock_flow_entires systcl and rps_flow_count
sysfs variable.
---
This patch implements receive flow steering (RFS).  RFS steers
received packets for layer 3 and 4 processing to the CPU where
the application for the corresponding flow is running.  RFS is an
extension of Receive Packet Steering (RPS).

The basic idea of RFS is that when an application calls recvmsg
(or sendmsg) the application's running CPU is stored in a hash
table that is indexed by the connection's rxhash which is stored in
the socket structure.  The rxhash is passed in skb's received on
the connection from netif_receive_skb.  For each received packet,
the associated rxhash is used to look up the CPU in the hash table,
if a valid CPU is set then the packet is steered to that CPU using
the RPS mechanisms.

The convolution of the simple approach is that it would potentially
allow OOO packets.  If threads are thrashing around CPUs or multiple
threads are trying to read from the same sockets, a quickly changing
CPU value in the hash table could cause rampant OOO packets--
we consider this a non-starter.

To avoid OOO packets, this solution implements two types of hash
tables: rps_sock_flow_table and rps_dev_flow_table.

rps_sock_table is a global hash table.  Each entry is just a CPU
number and it is populated in recvmsg and sendmsg as described above.
This table contains the "desired" CPUs for flows.

rps_dev_flow_table is specific to each device queue.  Each entry
contains a CPU and a tail queue counter.  The CPU is the "current"
CPU for a matching flow.  The tail queue counter holds the value
of a tail queue counter for the associated CPU's backlog queue at
the time of last enqueue for a flow matching the entry.

Each backlog queue has a queue head counter which is incremented
on dequeue, and so a queue tail counter is computed as queue head
count + queue length.  When a packet is enqueued on a backlog queue,
the current value of the queue tail counter is saved in the hash
entry of the rps_dev_flow_table.

And now the trick: when selecting the CPU for RPS (get_rps_cpu)
the rps_sock_flow table and the rps_dev_flow table for the RX queue
are consulted.  When the desired CPU for the flow (found in the
rps_sock_flow table) does not match the current CPU (found in the
rps_dev_flow table), the current CPU is changed to the desired CPU
if one of the following is true:

- The current CPU is unset (equal to RPS_NO_CPU)
- Current CPU is offline
- The current CPU's queue head counter >= queue tail counter in the
rps_dev_flow table.  This checks if the queue tail has advanced
beyond the last packet that was enqueued using this table entry.
This guarantees that all packets queued using this entry have been
dequeued, thus preserving in order delivery.

Making each queue have its own rps_dev_flow table has two advantages:
1) the tail queue counters will be written on each receive, so
keeping the table local to interrupting CPU s good for locality.  2)
this allows lockless access to the table-- the CPU number and queue
tail counter need to be accessed together under mutual exclusion
from netif_receive_skb, we assume that this is only called from
device napi_poll which is non-reentrant.

This patch implements RFS for TCP and connected UDP sockets.
It should be usable for other flow oriented protocols.

There are two configuration parameters for RFS.  The
"rps_flow_entries" kernel init parameter sets the number of
entries in the rps_sock_flow_table, the per rxqueue sysfs entry
"rps_flow_cnt" contains the number of entries in the rps_dev_flow
table for the rxqueue.  Both are rounded to power of two.

The obvious benefit of RFS (over just RPS) is that it achieves
CPU locality between the receive processing for a flow and the
applications processing; this can result in increased performance
(higher pps, lower latency).

The benefits of RFS are dependent on cache hierarchy, application
load, and other factors.  On simple benchmarks, we don't necessarily
see improvement and sometimes see degradation.  However, for more
complex benchmarks and for applications where cache pressure is
much higher this technique seems to perform very well.

Below are some benchmark results which show the potential benfit of
this patch.  The netperf test has 500 instances of netperf TCP_RR
test with 1 byte req. and resp.  The RPC test is an request/response
test similar in structure to netperf RR test ith 100 threads on
each host, but does more work in userspace that netperf.

e1000e on 8 core Intel
   No RFS or RPS		104K tps at 30% CPU
   No RFS (best RPS config):    290K tps at 63% CPU
   RFS				303K tps at 61% CPU

RPC test	tps	CPU%	50/90/99% usec latency	Latency StdDev
  No RFS/RPS	103K	48%	757/900/3185		4472.35
  RPS only:	174K	73%	415/993/2468		491.66
  RFS		223K	73%	379/651/1382		315.61

Signed-off-by: Tom Herbert <therbert@google.com> ---
diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index 55c2086..649a025 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -530,14 +530,73 @@ struct rps_map {
 };
 #define RPS_MAP_SIZE(_num) (sizeof(struct rps_map) + (_num * sizeof(u16)))
 
+/*
+ * The rps_dev_flow structure contains the mapping of a flow to a CPU and the
+ * tail pointer for that CPU's input queue at the time of last enqueue.
+ */
+struct rps_dev_flow {
+	u16 cpu;
+	u16 fill;
+	unsigned int last_qtail;
+};
+
+/*
+ * The rps_dev_flow_table structure contains a table of flow mappings.
+ */
+struct rps_dev_flow_table {
+	unsigned int mask;
+	struct rcu_head rcu;
+	struct work_struct free_work;
+	struct rps_dev_flow flows[0];
+};
+#define RPS_DEV_FLOW_TABLE_SIZE(_num) (sizeof(struct rps_dev_flow_table) + \
+    (_num * sizeof(struct rps_dev_flow)))
+
+/*
+ * The rps_sock_flow_table contains mappings of flows to the last CPU
+ * on which they were processed by the application (set in recvmsg).
+ */
+struct rps_sock_flow_table {
+	unsigned int mask;
+	u16 ents[0];
+};
+#define	RPS_SOCK_FLOW_TABLE_SIZE(_num) (sizeof(struct rps_sock_flow_table) + \
+    (_num * sizeof(u16)))
+
+#define RPS_NO_CPU 0xffff
+
+static inline void rps_record_sock_flow(struct rps_sock_flow_table *table,
+					u32 hash)
+{
+	if (table && hash) {
+		unsigned int cpu, index = hash & table->mask;
+
+		/* We only give a hint, preemption can change cpu under us */
+		cpu = raw_smp_processor_id();
+
+		if (table->ents[index] != cpu)
+			table->ents[index] = cpu;
+	}
+}
+
+static inline void rps_reset_sock_flow(struct rps_sock_flow_table *table,
+				       u32 hash)
+{
+	if (table && hash)
+		table->ents[hash & table->mask] = RPS_NO_CPU;
+}
+
+extern struct rps_sock_flow_table *rps_sock_flow_table;
+
 /* This structure contains an instance of an RX queue. */
 struct netdev_rx_queue {
 	struct rps_map *rps_map;
+	struct rps_dev_flow_table *rps_flow_table;
 	struct kobject kobj;
 	struct netdev_rx_queue *first;
 	atomic_t count;
 } ____cacheline_aligned_in_smp;
-#endif
+#endif /* CONFIG_RPS */
 
 /*
  * This structure defines the management hooks for network devices.
@@ -1333,11 +1392,19 @@ struct softnet_data {
 	/* Elements below can be accessed between CPUs for RPS */
 #ifdef CONFIG_RPS
 	struct call_single_data	csd ____cacheline_aligned_in_smp;
+	unsigned int		input_queue_head;
 #endif
 	struct sk_buff_head	input_pkt_queue;
 	struct napi_struct	backlog;
 };
 
+static inline void incr_input_queue_head(struct softnet_data *queue)
+{
+#ifdef CONFIG_RPS
+	queue->input_queue_head++;
+#endif
+}
+
 DECLARE_PER_CPU_ALIGNED(struct softnet_data, softnet_data);
 
 #define HAVE_NETIF_QUEUE
diff --git a/include/net/inet_sock.h b/include/net/inet_sock.h
index 83fd344..b487bc1 100644
--- a/include/net/inet_sock.h
+++ b/include/net/inet_sock.h
@@ -21,6 +21,7 @@
 #include <linux/string.h>
 #include <linux/types.h>
 #include <linux/jhash.h>
+#include <linux/netdevice.h>
 
 #include <net/flow.h>
 #include <net/sock.h>
@@ -101,6 +102,7 @@ struct rtable;
  * @uc_ttl - Unicast TTL
  * @inet_sport - Source port
  * @inet_id - ID counter for DF pkts
+ * @rxhash - flow hash received from netif layer
  * @tos - TOS
  * @mc_ttl - Multicasting TTL
  * @is_icsk - is this an inet_connection_sock?
@@ -124,6 +126,9 @@ struct inet_sock {
 	__u16			cmsg_flags;
 	__be16			inet_sport;
 	__u16			inet_id;
+#ifdef CONFIG_RPS
+	__u32			rxhash;
+#endif
 
 	struct ip_options	*opt;
 	__u8			tos;
@@ -219,4 +224,37 @@ static inline __u8 inet_sk_flowi_flags(const struct sock *sk)
 	return inet_sk(sk)->transparent ? FLOWI_FLAG_ANYSRC : 0;
 }
 
+static inline void inet_rps_record_flow(const struct sock *sk)
+{
+#ifdef CONFIG_RPS
+	struct rps_sock_flow_table *sock_flow_table;
+
+	rcu_read_lock();
+	sock_flow_table = rcu_dereference(rps_sock_flow_table);
+	rps_record_sock_flow(sock_flow_table, inet_sk(sk)->rxhash);
+	rcu_read_unlock();
+#endif
+}
+
+static inline void inet_rps_reset_flow(const struct sock *sk)
+{
+#ifdef CONFIG_RPS
+	struct rps_sock_flow_table *sock_flow_table;
+
+	rcu_read_lock();
+	sock_flow_table = rcu_dereference(rps_sock_flow_table);
+	rps_reset_sock_flow(sock_flow_table, inet_sk(sk)->rxhash);
+	rcu_read_unlock();
+#endif
+}
+
+static inline void inet_rps_save_rxhash(const struct sock *sk, u32 rxhash)
+{
+#ifdef CONFIG_RPS
+	if (unlikely(inet_sk(sk)->rxhash != rxhash)) {
+		inet_rps_reset_flow(sk);
+		inet_sk(sk)->rxhash = rxhash;
+	}
+#endif
+}
 #endif	/* _INET_SOCK_H */
diff --git a/net/core/dev.c b/net/core/dev.c
index e8041eb..d7107ac 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -2203,19 +2203,28 @@ int weight_p __read_mostly = 64;            /* old backlog weight */
 DEFINE_PER_CPU(struct netif_rx_stats, netdev_rx_stat) = { 0, };
 
 #ifdef CONFIG_RPS
+
+/* One global table that all flow-based protocols share. */
+struct rps_sock_flow_table *rps_sock_flow_table;
+EXPORT_SYMBOL(rps_sock_flow_table);
+
 /*
  * get_rps_cpu is called from netif_receive_skb and returns the target
  * CPU from the RPS map of the receiving queue for a given skb.
  * rcu_read_lock must be held on entry.
  */
-static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb)
+static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb,
+		       struct rps_dev_flow **rflowp)
 {
 	struct ipv6hdr *ip6;
 	struct iphdr *ip;
 	struct netdev_rx_queue *rxqueue;
 	struct rps_map *map;
+	struct rps_dev_flow_table *flow_table;
+	struct rps_sock_flow_table *sock_flow_table;
 	int cpu = -1;
 	u8 ip_proto;
+	u16 tcpu;
 	u32 addr1, addr2, ports, ihl;
 
 	if (skb_rx_queue_recorded(skb)) {
@@ -2232,7 +2241,7 @@ static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb)
 	} else
 		rxqueue = dev->_rx;
 
-	if (!rxqueue->rps_map)
+	if (!rxqueue->rps_map && !rxqueue->rps_flow_table)
 		goto done;
 
 	if (skb->rxhash)
@@ -2284,9 +2293,48 @@ static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb)
 		skb->rxhash = 1;
 
 got_hash:
+	flow_table = rcu_dereference(rxqueue->rps_flow_table);
+	sock_flow_table = rcu_dereference(rps_sock_flow_table);
+	if (flow_table && sock_flow_table) {
+		u16 next_cpu;
+		struct rps_dev_flow *rflow;
+
+		rflow = &flow_table->flows[skb->rxhash & flow_table->mask];
+		tcpu = rflow->cpu;
+
+		next_cpu = sock_flow_table->ents[skb->rxhash &
+		    sock_flow_table->mask];
+
+		/*
+		 * If the desired CPU (where last recvmsg was done) is
+		 * different from current CPU (one in the rx-queue flow
+		 * table entry), switch if one of the following holds:
+		 *   - Current CPU is unset (equal to RPS_NO_CPU).
+		 *   - Current CPU is offline.
+		 *   - The current CPU's queue tail has advanced beyond the
+		 *     last packet that was enqueued using this table entry.
+		 *     This guarantees that all previous packets for the flow
+		 *     have been dequeued, thus preserving in order delivery.
+		 */
+		if (unlikely(tcpu != next_cpu) &&
+		    (tcpu == RPS_NO_CPU || !cpu_online(tcpu) ||
+		     ((int)(per_cpu(softnet_data, tcpu).input_queue_head -
+		      rflow->last_qtail)) >= 0)) {
+			tcpu = rflow->cpu = next_cpu;
+			if (tcpu != RPS_NO_CPU)
+				rflow->last_qtail = per_cpu(softnet_data,
+				    tcpu).input_queue_head;
+		}
+		if (tcpu != RPS_NO_CPU && cpu_online(tcpu)) {
+			*rflowp = rflow;
+			cpu = tcpu;
+			goto done;
+		}
+	}
+
 	map = rcu_dereference(rxqueue->rps_map);
 	if (map) {
-		u16 tcpu = map->cpus[((u64) skb->rxhash * map->len) >> 32];
+		tcpu = map->cpus[((u64) skb->rxhash * map->len) >> 32];
 
 		if (cpu_online(tcpu)) {
 			cpu = tcpu;
@@ -2320,13 +2368,14 @@ static void trigger_softirq(void *data)
 	__napi_schedule(&queue->backlog);
 	__get_cpu_var(netdev_rx_stat).received_rps++;
 }
-#endif /* CONFIG_SMP */
+#endif /* CONFIG_RPS */
 
 /*
  * enqueue_to_backlog is called to queue an skb to a per CPU backlog
  * queue (may be a remote CPU queue).
  */
-static int enqueue_to_backlog(struct sk_buff *skb, int cpu)
+static int enqueue_to_backlog(struct sk_buff *skb, int cpu,
+			      unsigned int *qtail)
 {
 	struct softnet_data *queue;
 	unsigned long flags;
@@ -2341,6 +2390,10 @@ static int enqueue_to_backlog(struct sk_buff *skb, int cpu)
 		if (queue->input_pkt_queue.qlen) {
 enqueue:
 			__skb_queue_tail(&queue->input_pkt_queue, skb);
+#ifdef CONFIG_RPS
+			*qtail = queue->input_queue_head +
+			    queue->input_pkt_queue.qlen;
+#endif
 			rps_unlock(queue);
 			local_irq_restore(flags);
 			return NET_RX_SUCCESS;
@@ -2355,11 +2408,10 @@ enqueue:
 
 				cpu_set(cpu, rcpus->mask[rcpus->select]);
 				__raise_softirq_irqoff(NET_RX_SOFTIRQ);
-			} else
-				__napi_schedule(&queue->backlog);
-#else
-			__napi_schedule(&queue->backlog);
+				goto enqueue;
+			}
 #endif
+			__napi_schedule(&queue->backlog);
 		}
 		goto enqueue;
 	}
@@ -2401,18 +2453,25 @@ int netif_rx(struct sk_buff *skb)
 
 #ifdef CONFIG_RPS
 	{
+		struct rps_dev_flow voidflow, *rflow = &voidflow;
 		int cpu;
 
 		rcu_read_lock();
-		cpu = get_rps_cpu(skb->dev, skb);
+
+		cpu = get_rps_cpu(skb->dev, skb, &rflow);
 		if (cpu < 0)
 			cpu = smp_processor_id();
-		ret = enqueue_to_backlog(skb, cpu);
+
+		ret = enqueue_to_backlog(skb, cpu, &rflow->last_qtail);
+
 		rcu_read_unlock();
 	}
 #else
-	ret = enqueue_to_backlog(skb, get_cpu());
-	put_cpu();
+	{
+		unsigned int qtail;
+		ret = enqueue_to_backlog(skb, get_cpu(), &qtail);
+		put_cpu();
+	}
 #endif
 	return ret;
 }
@@ -2830,14 +2889,22 @@ out:
 int netif_receive_skb(struct sk_buff *skb)
 {
 #ifdef CONFIG_RPS
-	int cpu;
+	struct rps_dev_flow voidflow, *rflow = &voidflow;
+	int cpu, ret;
+
+	rcu_read_lock();
 
-	cpu = get_rps_cpu(skb->dev, skb);
+	cpu = get_rps_cpu(skb->dev, skb, &rflow);
 
-	if (cpu < 0)
-		return __netif_receive_skb(skb);
-	else
-		return enqueue_to_backlog(skb, cpu);
+	if (cpu >= 0) {
+		ret = enqueue_to_backlog(skb, cpu, &rflow->last_qtail);
+		rcu_read_unlock();
+	} else {
+		rcu_read_unlock();
+		ret = __netif_receive_skb(skb);
+	}
+
+	return ret;
 #else
 	return __netif_receive_skb(skb);
 #endif
@@ -2856,6 +2923,7 @@ static void flush_backlog(void *arg)
 		if (skb->dev == dev) {
 			__skb_unlink(skb, &queue->input_pkt_queue);
 			kfree_skb(skb);
+			incr_input_queue_head(queue);
 		}
 	rps_unlock(queue);
 }
@@ -3179,6 +3247,7 @@ static int process_backlog(struct napi_struct *napi, int quota)
 			local_irq_enable();
 			break;
 		}
+		incr_input_queue_head(queue);
 		rps_unlock(queue);
 		local_irq_enable();
 
@@ -5542,8 +5611,10 @@ static int dev_cpu_callback(struct notifier_block *nfb,
 	local_irq_enable();
 
 	/* Process offline CPU's input_pkt_queue */
-	while ((skb = __skb_dequeue(&oldsd->input_pkt_queue)))
+	while ((skb = __skb_dequeue(&oldsd->input_pkt_queue))) {
 		netif_rx(skb);
+		incr_input_queue_head(oldsd);
+	}
 
 	return NOTIFY_OK;
 }
diff --git a/net/core/net-sysfs.c b/net/core/net-sysfs.c
index 96ed690..f0f1bb7 100644
--- a/net/core/net-sysfs.c
+++ b/net/core/net-sysfs.c
@@ -601,22 +601,109 @@ ssize_t store_rps_map(struct netdev_rx_queue *queue,
 	return len;
 }
 
+static ssize_t show_rps_dev_flow_table_cnt(struct netdev_rx_queue *queue,
+					   struct rx_queue_attribute *attr,
+					   char *buf)
+{
+	struct rps_dev_flow_table *flow_table;
+	unsigned int val = 0;
+
+	rcu_read_lock();
+	flow_table = rcu_dereference(queue->rps_flow_table);
+	if (flow_table)
+		val = flow_table->mask + 1;
+	rcu_read_unlock();
+
+	return sprintf(buf, "%u\n", val);
+}
+
+static void rps_dev_flow_table_release_work(struct work_struct *work)
+{
+	struct rps_dev_flow_table *table = container_of(work,
+	    struct rps_dev_flow_table, free_work);
+
+	vfree(table);
+}
+
+static void rps_dev_flow_table_release(struct rcu_head *rcu)
+{
+	struct rps_dev_flow_table *table = container_of(rcu,
+	    struct rps_dev_flow_table, rcu);
+
+	INIT_WORK(&table->free_work, rps_dev_flow_table_release_work);
+	schedule_work(&table->free_work);
+}
+
+ssize_t store_rps_dev_flow_table_cnt(struct netdev_rx_queue *queue,
+				     struct rx_queue_attribute *attr,
+				     const char *buf, size_t len)
+{
+	unsigned int count;
+	char *endp;
+	struct rps_dev_flow_table *table, *old_table;
+	static DEFINE_SPINLOCK(rps_dev_flow_lock);
+
+	if (!capable(CAP_NET_ADMIN))
+		return -EPERM;
+
+	count = simple_strtoul(buf, &endp, 0);
+	if (endp == buf)
+		return -EINVAL;
+
+	if (count) {
+		int i;
+
+		if (count > 1<<30) {
+			/* Enforce a limit to prevent overflow */
+			return -EINVAL;
+		}
+		count = roundup_pow_of_two(count);
+		table = vmalloc(RPS_DEV_FLOW_TABLE_SIZE(count));
+		if (!table)
+			return -ENOMEM;
+
+		table->mask = count - 1;
+		for (i = 0; i < count; i++)
+			table->flows[i].cpu = RPS_NO_CPU;
+	} else
+		table = NULL;
+
+	spin_lock(&rps_dev_flow_lock);
+	old_table = queue->rps_flow_table;
+	rcu_assign_pointer(queue->rps_flow_table, table);
+	spin_unlock(&rps_dev_flow_lock);
+
+	if (old_table)
+		call_rcu(&old_table->rcu, rps_dev_flow_table_release);
+
+	return len;
+}
+
 static struct rx_queue_attribute rps_cpus_attribute =
 	__ATTR(rps_cpus, S_IRUGO | S_IWUSR, show_rps_map, store_rps_map);
 
+
+static struct rx_queue_attribute rps_dev_flow_table_cnt_attribute =
+	__ATTR(rps_flow_cnt, S_IRUGO | S_IWUSR,
+	    show_rps_dev_flow_table_cnt, store_rps_dev_flow_table_cnt);
+
 static struct attribute *rx_queue_default_attrs[] = {
 	&rps_cpus_attribute.attr,
+	&rps_dev_flow_table_cnt_attribute.attr,
 	NULL
 };
 
 static void rx_queue_release(struct kobject *kobj)
 {
 	struct netdev_rx_queue *queue = to_rx_queue(kobj);
-	struct rps_map *map = queue->rps_map;
 	struct netdev_rx_queue *first = queue->first;
 
-	if (map)
-		call_rcu(&map->rcu, rps_map_release);
+	if (queue->rps_map)
+		call_rcu(&queue->rps_map->rcu, rps_map_release);
+
+	if (queue->rps_flow_table)
+		call_rcu(&queue->rps_flow_table->rcu,
+		    rps_dev_flow_table_release);
 
 	if (atomic_dec_and_test(&first->count))
 		kfree(first);
diff --git a/net/core/sysctl_net_core.c b/net/core/sysctl_net_core.c
index b7b6b82..e023c93 100644
--- a/net/core/sysctl_net_core.c
+++ b/net/core/sysctl_net_core.c
@@ -17,6 +17,65 @@
 #include <net/ip.h>
 #include <net/sock.h>
 
+#ifdef CONFIG_RPS
+static int rps_sock_flow_sysctl(ctl_table *table, int write,
+				void __user *buffer, size_t *lenp, loff_t *ppos)
+{
+	unsigned int orig_size, size;
+	int ret, i;
+	ctl_table tmp = {
+		.data = &size,
+		.maxlen = sizeof(size),
+		.mode = table->mode
+	};
+	struct rps_sock_flow_table *orig_sock_table, *sock_table;
+	static DEFINE_MUTEX(sock_flow_mutex);
+
+	mutex_lock(&sock_flow_mutex);
+
+	orig_sock_table = rps_sock_flow_table;
+	size = orig_size = orig_sock_table ? orig_sock_table->mask + 1 : 0;
+
+	ret = proc_dointvec(&tmp, write, buffer, lenp, ppos);
+
+	if (write) {
+		if (size) {
+			if (size > 1<<30) {
+				/* Enforce limit to prevent overflow */
+				mutex_unlock(&sock_flow_mutex);
+				return -EINVAL;
+			}
+			size = roundup_pow_of_two(size);
+			if (size != orig_size) {
+				sock_table =
+				    vmalloc(RPS_SOCK_FLOW_TABLE_SIZE(size));
+				if (!sock_table) {
+					mutex_unlock(&sock_flow_mutex);
+					return -ENOMEM;
+				}
+
+				sock_table->mask = size - 1;
+			} else
+				sock_table = orig_sock_table;
+
+			for (i = 0; i < size; i++)
+				sock_table->ents[i] = RPS_NO_CPU;
+		} else
+			sock_table = NULL;
+
+		if (sock_table != orig_sock_table) {
+			rcu_assign_pointer(rps_sock_flow_table, sock_table);
+			synchronize_rcu();
+			vfree(orig_sock_table);
+		}
+	}
+
+	mutex_unlock(&sock_flow_mutex);
+
+	return ret;
+}
+#endif /* CONFIG_RPS */
+
 static struct ctl_table net_core_table[] = {
 #ifdef CONFIG_NET
 	{
@@ -82,6 +141,14 @@ static struct ctl_table net_core_table[] = {
 		.mode		= 0644,
 		.proc_handler	= proc_dointvec
 	},
+#ifdef CONFIG_RPS
+	{
+		.procname	= "rps_sock_flow_entries",
+		.maxlen		= sizeof(int),
+		.mode		= 0644,
+		.proc_handler	= rps_sock_flow_sysctl
+	},
+#endif
 #endif /* CONFIG_NET */
 	{
 		.procname	= "netdev_budget",
diff --git a/net/ipv4/af_inet.c b/net/ipv4/af_inet.c
index 193dcd6..c5376c7 100644
--- a/net/ipv4/af_inet.c
+++ b/net/ipv4/af_inet.c
@@ -419,6 +419,8 @@ int inet_release(struct socket *sock)
 	if (sk) {
 		long timeout;
 
+		inet_rps_reset_flow(sk);
+
 		/* Applications forget to leave groups before exiting */
 		ip_mc_drop_socket(sk);
 
@@ -720,6 +722,8 @@ int inet_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
 {
 	struct sock *sk = sock->sk;
 
+	inet_rps_record_flow(sk);
+
 	/* We may need to bind the socket. */
 	if (!inet_sk(sk)->inet_num && inet_autobind(sk))
 		return -EAGAIN;
@@ -728,12 +732,13 @@ int inet_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
 }
 EXPORT_SYMBOL(inet_sendmsg);
 
-
 static ssize_t inet_sendpage(struct socket *sock, struct page *page, int offset,
 			     size_t size, int flags)
 {
 	struct sock *sk = sock->sk;
 
+	inet_rps_record_flow(sk);
+
 	/* We may need to bind the socket. */
 	if (!inet_sk(sk)->inet_num && inet_autobind(sk))
 		return -EAGAIN;
@@ -743,6 +748,22 @@ static ssize_t inet_sendpage(struct socket *sock, struct page *page, int offset,
 	return sock_no_sendpage(sock, page, offset, size, flags);
 }
 
+int inet_recvmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
+		 size_t size, int flags)
+{
+	struct sock *sk = sock->sk;
+	int addr_len = 0;
+	int err;
+
+	inet_rps_record_flow(sk);
+
+	err = sk->sk_prot->recvmsg(iocb, sk, msg, size, flags & MSG_DONTWAIT,
+				   flags & ~MSG_DONTWAIT, &addr_len);
+	if (err >= 0)
+		msg->msg_namelen = addr_len;
+	return err;
+}
+EXPORT_SYMBOL(inet_recvmsg);
 
 int inet_shutdown(struct socket *sock, int how)
 {
@@ -872,7 +893,7 @@ const struct proto_ops inet_stream_ops = {
 	.setsockopt	   = sock_common_setsockopt,
 	.getsockopt	   = sock_common_getsockopt,
 	.sendmsg	   = tcp_sendmsg,
-	.recvmsg	   = sock_common_recvmsg,
+	.recvmsg	   = inet_recvmsg,
 	.mmap		   = sock_no_mmap,
 	.sendpage	   = tcp_sendpage,
 	.splice_read	   = tcp_splice_read,
@@ -899,7 +920,7 @@ const struct proto_ops inet_dgram_ops = {
 	.setsockopt	   = sock_common_setsockopt,
 	.getsockopt	   = sock_common_getsockopt,
 	.sendmsg	   = inet_sendmsg,
-	.recvmsg	   = sock_common_recvmsg,
+	.recvmsg	   = inet_recvmsg,
 	.mmap		   = sock_no_mmap,
 	.sendpage	   = inet_sendpage,
 #ifdef CONFIG_COMPAT
@@ -929,7 +950,7 @@ static const struct proto_ops inet_sockraw_ops = {
 	.setsockopt	   = sock_common_setsockopt,
 	.getsockopt	   = sock_common_getsockopt,
 	.sendmsg	   = inet_sendmsg,
-	.recvmsg	   = sock_common_recvmsg,
+	.recvmsg	   = inet_recvmsg,
 	.mmap		   = sock_no_mmap,
 	.sendpage	   = inet_sendpage,
 #ifdef CONFIG_COMPAT
diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
index a24995c..ad08392 100644
--- a/net/ipv4/tcp_ipv4.c
+++ b/net/ipv4/tcp_ipv4.c
@@ -1672,6 +1672,8 @@ process:
 
 	skb->dev = NULL;
 
+	inet_rps_save_rxhash(sk, skb->rxhash);
+
 	bh_lock_sock_nested(sk);
 	ret = 0;
 	if (!sock_owned_by_user(sk)) {
diff --git a/net/ipv4/udp.c b/net/ipv4/udp.c
index 8fef859..666b963 100644
--- a/net/ipv4/udp.c
+++ b/net/ipv4/udp.c
@@ -1217,6 +1217,7 @@ int udp_disconnect(struct sock *sk, int flags)
 	sk->sk_state = TCP_CLOSE;
 	inet->inet_daddr = 0;
 	inet->inet_dport = 0;
+	inet_rps_save_rxhash(sk, 0);
 	sk->sk_bound_dev_if = 0;
 	if (!(sk->sk_userlocks & SOCK_BINDADDR_LOCK))
 		inet_reset_saddr(sk);
@@ -1258,8 +1259,12 @@ EXPORT_SYMBOL(udp_lib_unhash);
 
 static int __udp_queue_rcv_skb(struct sock *sk, struct sk_buff *skb)
 {
-	int rc = sock_queue_rcv_skb(sk, skb);
+	int rc;
+
+	if (inet_sk(sk)->inet_daddr)
+		inet_rps_save_rxhash(sk, skb->rxhash);
 
+	rc = sock_queue_rcv_skb(sk, skb);
 	if (rc < 0) {
 		int is_udplite = IS_UDPLITE(sk);
 

^ permalink raw reply related	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  5:47 [PATCH v5] rfs: Receive Flow Steering Tom Herbert
@ 2010-04-16  6:33 ` David Miller
  2010-04-16  6:56   ` Eric Dumazet
  2010-04-16 19:37   ` Eric Dumazet
  2010-04-16 11:57 ` Andi Kleen
  1 sibling, 2 replies; 60+ messages in thread
From: David Miller @ 2010-04-16  6:33 UTC (permalink / raw)
  To: therbert; +Cc: netdev, eric.dumazet

From: Tom Herbert <therbert@google.com>
Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)

> Version 5 of RFS:
> - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
> static function.
> - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
> sysfs variable.

I've read this over a few times and I think it's ready to go into
net-next-2.6, we can tweak things as-needed from here on out.

Eric, what do you think?

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  6:33 ` David Miller
@ 2010-04-16  6:56   ` Eric Dumazet
  2010-04-16  7:18     ` Eric Dumazet
                       ` (2 more replies)
  2010-04-16 19:37   ` Eric Dumazet
  1 sibling, 3 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16  6:56 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le jeudi 15 avril 2010 à 23:33 -0700, David Miller a écrit :
> From: Tom Herbert <therbert@google.com>
> Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)
> 
> > Version 5 of RFS:
> > - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
> > static function.
> > - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
> > sysfs variable.
> 
> I've read this over a few times and I think it's ready to go into
> net-next-2.6, we can tweak things as-needed from here on out.
> 
> Eric, what do you think?

I read the patch and found no error.

I booted a test machine and performed some tests

I am a bit worried of a tbench regression I am looking at right now.

if RFS disabled , tbench 16   ->  4408.63 MB/sec 


# grep . /sys/class/net/lo/queues/rx-0/*
/sys/class/net/lo/queues/rx-0/rps_cpus:00000000
/sys/class/net/lo/queues/rx-0/rps_flow_cnt:8192
# cat /proc/sys/net/core/rps_sock_flow_entries
8192


echo ffff >/sys/class/net/lo/queues/rx-0/rps_cpus

tbench 16 -> 2336.32 MB/sec


-----------------------------------------------------------------------------------------------------------------------------------------------------
   PerfTop:   14561 irqs/sec  kernel:86.3% [1000Hz cycles],  (all, 16 CPUs)
-----------------------------------------------------------------------------------------------------------------------------------------------------

             samples  pcnt function                       DSO
             _______ _____ ______________________________ __________________________________________________________

             2664.00  5.1% copy_user_generic_string       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
             2323.00  4.4% acpi_os_read_port              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
             1641.00  3.1% _raw_spin_lock_irqsave         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
             1260.00  2.4% schedule                       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
             1159.00  2.2% _raw_spin_lock                 /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
             1051.00  2.0% tcp_ack                        /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              991.00  1.9% tcp_sendmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              922.00  1.8% tcp_recvmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              821.00  1.6% child_run                      /usr/bin/tbench                                           
              766.00  1.5% all_string_sub                 /usr/bin/tbench                                           
              630.00  1.2% __switch_to                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              608.00  1.2% __GI_strchr                    /lib/tls/libc-2.3.4.so                                    
              606.00  1.2% ipt_do_table                   /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              600.00  1.1% __GI_strstr                    /lib/tls/libc-2.3.4.so                                    
              556.00  1.1% __netif_receive_skb            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              504.00  1.0% tcp_transmit_skb               /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              502.00  1.0% tick_nohz_stop_sched_tick      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              481.00  0.9% _raw_spin_unlock_irqrestore    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              473.00  0.9% next_token                     /usr/bin/tbench                                           
              449.00  0.9% ip_rcv                         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              423.00  0.8% call_function_single_interrupt /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              422.00  0.8% ia32_sysenter_target           /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              420.00  0.8% compat_sys_socketcall          /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              401.00  0.8% mod_timer                      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              400.00  0.8% process_backlog                /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              399.00  0.8% ip_queue_xmit                  /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              387.00  0.7% select_task_rq_fair            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              377.00  0.7% _raw_spin_lock_bh              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
              360.00  0.7% tcp_v4_rcv                     /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux

But if RFS is on, why activating rps_cpus change tbench ?




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  6:56   ` Eric Dumazet
@ 2010-04-16  7:18     ` Eric Dumazet
  2010-04-16  7:26       ` David Miller
  2010-04-16 15:35     ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
  2010-04-16 18:35     ` Tom Herbert
  2 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16  7:18 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le vendredi 16 avril 2010 à 08:56 +0200, Eric Dumazet a écrit :

> I read the patch and found no error.
> 
> I booted a test machine and performed some tests
> 
> I am a bit worried of a tbench regression I am looking at right now.
> 
> if RFS disabled , tbench 16   ->  4408.63 MB/sec 
> 
> 
> # grep . /sys/class/net/lo/queues/rx-0/*
> /sys/class/net/lo/queues/rx-0/rps_cpus:00000000
> /sys/class/net/lo/queues/rx-0/rps_flow_cnt:8192
> # cat /proc/sys/net/core/rps_sock_flow_entries
> 8192
> 
> 
> echo ffff >/sys/class/net/lo/queues/rx-0/rps_cpus
> 
> tbench 16 -> 2336.32 MB/sec
> 
> 
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>    PerfTop:   14561 irqs/sec  kernel:86.3% [1000Hz cycles],  (all, 16 CPUs)
> -----------------------------------------------------------------------------------------------------------------------------------------------------
> 
>              samples  pcnt function                       DSO
>              _______ _____ ______________________________ __________________________________________________________
> 
>              2664.00  5.1% copy_user_generic_string       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              2323.00  4.4% acpi_os_read_port              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              1641.00  3.1% _raw_spin_lock_irqsave         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              1260.00  2.4% schedule                       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              1159.00  2.2% _raw_spin_lock                 /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              1051.00  2.0% tcp_ack                        /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               991.00  1.9% tcp_sendmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               922.00  1.8% tcp_recvmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               821.00  1.6% child_run                      /usr/bin/tbench                                           
>               766.00  1.5% all_string_sub                 /usr/bin/tbench                                           
>               630.00  1.2% __switch_to                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               608.00  1.2% __GI_strchr                    /lib/tls/libc-2.3.4.so                                    
>               606.00  1.2% ipt_do_table                   /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               600.00  1.1% __GI_strstr                    /lib/tls/libc-2.3.4.so                                    
>               556.00  1.1% __netif_receive_skb            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               504.00  1.0% tcp_transmit_skb               /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               502.00  1.0% tick_nohz_stop_sched_tick      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               481.00  0.9% _raw_spin_unlock_irqrestore    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               473.00  0.9% next_token                     /usr/bin/tbench                                           
>               449.00  0.9% ip_rcv                         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               423.00  0.8% call_function_single_interrupt /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               422.00  0.8% ia32_sysenter_target           /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               420.00  0.8% compat_sys_socketcall          /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               401.00  0.8% mod_timer                      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               400.00  0.8% process_backlog                /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               399.00  0.8% ip_queue_xmit                  /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               387.00  0.7% select_task_rq_fair            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               377.00  0.7% _raw_spin_lock_bh              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>               360.00  0.7% tcp_v4_rcv                     /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> 
> But if RFS is on, why activating rps_cpus change tbench ?
> 

Hmm, I wonder if its not an artifact of net-next-2.6 being a bit old
(versus linux-2.6). I know scheduler guys did some tweaks.

Because apparently, some cpus are idle part of their time (30% ???)

Or a new bug on cpu accounting, reporting idle time while cpus are
busy....

# vmstat 1
procs -----------memory---------- ---swap-- -----io---- -system-- ----cpu----
 r  b   swpd   free   buff  cache   si   so    bi    bo   in   cs us sy id wa
16  0      0 5670264  13280  63392    0    0     2     1 1512  227 12 47 41  0
18  0      0 5669396  13280  63392    0    0     0     0 657952 1606102 14 58 28  0
17  0      0 5668776  13288  63392    0    0     0    12 656701 1606369 14 58 28  0
18  0      0 5669644  13288  63392    0    0     0     0 657636 1603960 15 57 28  0
17  0      0 5670900  13288  63392    0    0     0     0 666425 1584847 15 56 29  0
15  0      0 5669164  13288  63392    0    0     0     0 682578 1472616 14 56 30  0
16  0      0 5669412  13288  63392    0    0     0     0 695767 1506302 14 54 32  0
14  0      0 5668916  13296  63396    0    0     4   148 685286 1482897 14 56 30  0
17  0      0 5669784  13296  63396    0    0     0     0 683910 1477994 14 56 30  0
18  0      0 5670032  13296  63396    0    0     0     0 692023 1497195 14 55 31  0
16  0      0 5669040  13296  63396    0    0     0     0 677477 1468157 14 56 30  0
16  0      0 5668916  13312  63396    0    0     0    32 489358 1048553 14 57 30  0
18  0      0 5667924  13320  63396    0    0     0    12 424787 897145 15 55 29  0

RFS off :

# vmstat 1
procs -----------memory---------- ---swap-- -----io---- -system-- ----cpu----
 r  b   swpd   free   buff  cache   si   so    bi    bo   in   cs us sy id wa
24  0      0 5669624  13632  63476    0    0     2     1  261   82 12 48 40  0
26  0      0 5669492  13632  63476    0    0     0     0 4223 1740651 21 71  7  0
23  0      0 5669864  13640  63476    0    0     0    12 4205 1731882 21 71  8  0
23  0      0 5670484  13640  63476    0    0     0     0 4176 1733448 21 71  8  0
24  0      0 5670588  13640  63476    0    0     0     0 4176 1733845 21 72  7  0
21  0      0 5671084  13640  63476    0    0     0     0 4200 1734990 20 73  7  0
23  0      0 5671580  13640  63476    0    0     0     0 4168 1735100 21 71  8  0
23  0      0 5671704  13640  63480    0    0     4   132 4221 1733428 21 72  7  0
22  0      0 5671952  13640  63480    0    0     0     0 4190 1730370 21 72  8  0
20  0      0 5672292  13640  63480    0    0     0     0 4212 1732084 22 70  8  0




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  7:18     ` Eric Dumazet
@ 2010-04-16  7:26       ` David Miller
  2010-04-16  7:48         ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-16  7:26 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Fri, 16 Apr 2010 09:18:03 +0200

> Hmm, I wonder if its not an artifact of net-next-2.6 being a bit old
> (versus linux-2.6). I know scheduler guys did some tweaks.

I synced net-next-2.6 up with Linus's current tree just a day
or two ago when I pulled net-2.6 into net-next-2.6.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  7:26       ` David Miller
@ 2010-04-16  7:48         ` Eric Dumazet
  2010-04-17  7:52           ` [PATCH net-next-2.6] rps: rps_sock_flow_table is mostly read Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16  7:48 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le vendredi 16 avril 2010 à 00:26 -0700, David Miller a écrit :
> From: Eric Dumazet <eric.dumazet@gmail.com>
> Date: Fri, 16 Apr 2010 09:18:03 +0200
> 
> > Hmm, I wonder if its not an artifact of net-next-2.6 being a bit old
> > (versus linux-2.6). I know scheduler guys did some tweaks.
> 
> I synced net-next-2.6 up with Linus's current tree just a day
> or two ago when I pulled net-2.6 into net-next-2.6.

OK thanks :)

Tom, please add a read_mostly to rps_sock_flow_table

struct rps_sock_flow_table *rps_sock_flow_table __read_mostly;

I'll spend some hours today to track the problem.




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  5:47 [PATCH v5] rfs: Receive Flow Steering Tom Herbert
  2010-04-16  6:33 ` David Miller
@ 2010-04-16 11:57 ` Andi Kleen
  2010-04-16 13:32   ` jamal
  1 sibling, 1 reply; 60+ messages in thread
From: Andi Kleen @ 2010-04-16 11:57 UTC (permalink / raw)
  To: Tom Herbert; +Cc: davem, netdev, eric.dumazet

Tom Herbert <therbert@google.com> writes:
> +
> +		/*
> +		 * If the desired CPU (where last recvmsg was done) is
> +		 * different from current CPU (one in the rx-queue flow
> +		 * table entry), switch if one of the following holds:
> +		 *   - Current CPU is unset (equal to RPS_NO_CPU).
> +		 *   - Current CPU is offline.
> +		 *   - The current CPU's queue tail has advanced beyond the
> +		 *     last packet that was enqueued using this table entry.
> +		 *     This guarantees that all previous packets for the flow
> +		 *     have been dequeued, thus preserving in order delivery.
> +		 */
> +		if (unlikely(tcpu != next_cpu) &&
> +		    (tcpu == RPS_NO_CPU || !cpu_online(tcpu) ||
> +		     ((int)(per_cpu(softnet_data, tcpu).input_queue_head -

One thing I've been wondering while reading if this should be made
socket or SMT aware.

If you're on a hyperthreaded system and sending a IPI
to your core sibling, which has a completely shared cache hierarchy,
might not be the best use of cycles.

The same could potentially true for shared L2 or shared L3 cache
(e.g. only redirect flows between different sockets)

Have you ever considered that?

This is of course something that could be addressed post-merge, not
a blocker.

-Andi

-- 
ak@linux.intel.com -- Speaking for myself only.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 11:57 ` Andi Kleen
@ 2010-04-16 13:32   ` jamal
  2010-04-16 13:42     ` Andi Kleen
  0 siblings, 1 reply; 60+ messages in thread
From: jamal @ 2010-04-16 13:32 UTC (permalink / raw)
  To: Andi Kleen; +Cc: Tom Herbert, davem, netdev, eric.dumazet

On Fri, 2010-04-16 at 13:57 +0200, Andi Kleen wrote:

> One thing I've been wondering while reading if this should be made
> socket or SMT aware.
> 
> If you're on a hyperthreaded system and sending a IPI
> to your core sibling, which has a completely shared cache hierarchy,
> might not be the best use of cycles.
> 
> The same could potentially true for shared L2 or shared L3 cache
> (e.g. only redirect flows between different sockets)
> 
> Have you ever considered that?
> 

How are you going to schedule the net softirq on an empty queue if you
do this?
BTW, in my tests sending an IPI to an SMT sibling or to another core
didnt make any difference in terms of latency - still 5 microsecs.
I dont have dual Nehalem where we have to cross QPI - there i suspect
it will be longer than 5 microsecs.

cheers,
jamal


^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 13:32   ` jamal
@ 2010-04-16 13:42     ` Andi Kleen
  2010-04-16 14:05       ` jamal
  0 siblings, 1 reply; 60+ messages in thread
From: Andi Kleen @ 2010-04-16 13:42 UTC (permalink / raw)
  To: jamal; +Cc: Andi Kleen, Tom Herbert, davem, netdev, eric.dumazet

On Fri, Apr 16, 2010 at 09:32:06AM -0400, jamal wrote:
> How are you going to schedule the net softirq on an empty queue if you
> do this?

Sorry don't understand the question? 

You can always do the flow as if rps was not there.

> BTW, in my tests sending an IPI to an SMT sibling or to another core
> didnt make any difference in terms of latency - still 5 microsecs.
> I dont have dual Nehalem where we have to cross QPI - there i suspect
> it will be longer than 5 microsecs.

I meant an IPI to a sibling is not useful. You send it to the IPI
to get cache locality in the target, but if the target has the same
cache locality as you you can as well avoid the cost of the IPI
and process directly.

For thread sibling I'm pretty sure it's useless. Not full sure about
socket sibling. Maybe.

-Andi
-- 
ak@linux.intel.com -- Speaking for myself only.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 13:42     ` Andi Kleen
@ 2010-04-16 14:05       ` jamal
  2010-04-16 15:28         ` Andi Kleen
  0 siblings, 1 reply; 60+ messages in thread
From: jamal @ 2010-04-16 14:05 UTC (permalink / raw)
  To: Andi Kleen; +Cc: Andi Kleen, Tom Herbert, davem, netdev, eric.dumazet

On Fri, 2010-04-16 at 15:42 +0200, Andi Kleen wrote:
> On Fri, Apr 16, 2010 at 09:32:06AM -0400, jamal wrote:
> > How are you going to schedule the net softirq on an empty queue if you
> > do this?
> 
> Sorry don't understand the question? 
> 
> You can always do the flow as if rps was not there.

Meaning you schedule the other side netrx softirq if queue is empty?

> I meant an IPI to a sibling is not useful. You send it to the IPI
> to get cache locality in the target, but if the target has the same
> cache locality as you you can as well avoid the cost of the IPI
> and process directly.
> 

Isnt the purpose of the IPI to signal remote side that theres something
for it to do? Does it also sync the remote cache?

> For thread sibling I'm pretty sure it's useless. Not full sure about
> socket sibling. Maybe.
> 

Agreed, the SMT threads share L2. All the cores share L3. And it is
inclusive, so if it is missing it is in L1 of one thread it must be
present in L2 of shared cache as well as L3. Across the QPI i dont think
that is true.
But if you speacial case this - arent you being specific to Nehalem?

cheers,
jamal


^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 14:05       ` jamal
@ 2010-04-16 15:28         ` Andi Kleen
  0 siblings, 0 replies; 60+ messages in thread
From: Andi Kleen @ 2010-04-16 15:28 UTC (permalink / raw)
  To: jamal; +Cc: Andi Kleen, Andi Kleen, Tom Herbert, davem, netdev, eric.dumazet

On Fri, Apr 16, 2010 at 10:05:15AM -0400, jamal wrote:
> On Fri, 2010-04-16 at 15:42 +0200, Andi Kleen wrote:
> > On Fri, Apr 16, 2010 at 09:32:06AM -0400, jamal wrote:
> > > How are you going to schedule the net softirq on an empty queue if you
> > > do this?
> > 
> > Sorry don't understand the question? 
> > 
> > You can always do the flow as if rps was not there.
> 
> Meaning you schedule the other side netrx softirq if queue is empty?

You handle the packet like if rps wasn't enabled. softirq on current
CPU and it queues it on the socket.

> > I meant an IPI to a sibling is not useful. You send it to the IPI
> > to get cache locality in the target, but if the target has the same
> > cache locality as you you can as well avoid the cost of the IPI
> > and process directly.
> > 
> 
> Isnt the purpose of the IPI to signal remote side that theres something
> for it to do? 

The current CPU can queue on that socket as well.

The whole point of the IPI is to do it with cache locality.
But if cache locality is already there on the current CPU you don't
need the IPI.

> Does it also sync the remote cache?

No, the caches are always coherent.

> 
> > For thread sibling I'm pretty sure it's useless. Not full sure about
> > socket sibling. Maybe.
> > 
> 
> Agreed, the SMT threads share L2. All the cores share L3. And it is
> inclusive, so if it is missing it is in L1 of one thread it must be
> present in L2 of shared cache as well as L3. Across the QPI i dont think
> that is true.
> But if you speacial case this - arent you being specific to Nehalem?

Other CPUs have SMT too (Niagara, POWER 6/7, mips, ...). It should
be the same there.

Assuming L3 affinity helps it might need to be a CPU specific tunable
yes. The scheduler has some information about this.

-Andi
-- 
ak@linux.intel.com -- Speaking for myself only.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  6:56   ` Eric Dumazet
  2010-04-16  7:18     ` Eric Dumazet
@ 2010-04-16 15:35     ` Tom Herbert
  2010-04-16 18:15       ` Eric Dumazet
  2010-04-16 18:35     ` Tom Herbert
  2 siblings, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-16 15:35 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: David Miller, netdev

Eric, thanks for testing that.  Admittedly, we have looked at enabling
RFS/RPS over loopback.   I'll look at that today also.


On Thu, Apr 15, 2010 at 11:56 PM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> Le jeudi 15 avril 2010 à 23:33 -0700, David Miller a écrit :
>> From: Tom Herbert <therbert@google.com>
>> Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)
>>
>> > Version 5 of RFS:
>> > - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
>> > static function.
>> > - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
>> > sysfs variable.
>>
>> I've read this over a few times and I think it's ready to go into
>> net-next-2.6, we can tweak things as-needed from here on out.
>>
>> Eric, what do you think?
>
> I read the patch and found no error.
>
> I booted a test machine and performed some tests
>
> I am a bit worried of a tbench regression I am looking at right now.
>
> if RFS disabled , tbench 16   ->  4408.63 MB/sec
>
>
> # grep . /sys/class/net/lo/queues/rx-0/*
> /sys/class/net/lo/queues/rx-0/rps_cpus:00000000
> /sys/class/net/lo/queues/rx-0/rps_flow_cnt:8192
> # cat /proc/sys/net/core/rps_sock_flow_entries
> 8192
>
>
> echo ffff >/sys/class/net/lo/queues/rx-0/rps_cpus
>
> tbench 16 -> 2336.32 MB/sec
>
>
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>   PerfTop:   14561 irqs/sec  kernel:86.3% [1000Hz cycles],  (all, 16 CPUs)
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>
>             samples  pcnt function                       DSO
>             _______ _____ ______________________________ __________________________________________________________
>
>             2664.00  5.1% copy_user_generic_string       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             2323.00  4.4% acpi_os_read_port              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1641.00  3.1% _raw_spin_lock_irqsave         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1260.00  2.4% schedule                       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1159.00  2.2% _raw_spin_lock                 /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1051.00  2.0% tcp_ack                        /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              991.00  1.9% tcp_sendmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              922.00  1.8% tcp_recvmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              821.00  1.6% child_run                      /usr/bin/tbench
>              766.00  1.5% all_string_sub                 /usr/bin/tbench
>              630.00  1.2% __switch_to                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              608.00  1.2% __GI_strchr                    /lib/tls/libc-2.3.4.so
>              606.00  1.2% ipt_do_table                   /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              600.00  1.1% __GI_strstr                    /lib/tls/libc-2.3.4.so
>              556.00  1.1% __netif_receive_skb            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              504.00  1.0% tcp_transmit_skb               /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              502.00  1.0% tick_nohz_stop_sched_tick      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              481.00  0.9% _raw_spin_unlock_irqrestore    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              473.00  0.9% next_token                     /usr/bin/tbench
>              449.00  0.9% ip_rcv                         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              423.00  0.8% call_function_single_interrupt /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              422.00  0.8% ia32_sysenter_target           /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              420.00  0.8% compat_sys_socketcall          /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              401.00  0.8% mod_timer                      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              400.00  0.8% process_backlog                /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              399.00  0.8% ip_queue_xmit                  /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              387.00  0.7% select_task_rq_fair            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              377.00  0.7% _raw_spin_lock_bh              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              360.00  0.7% tcp_v4_rcv                     /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>
> But if RFS is on, why activating rps_cpus change tbench ?
>
>
>
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 15:35     ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
@ 2010-04-16 18:15       ` Eric Dumazet
  0 siblings, 0 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16 18:15 UTC (permalink / raw)
  To: Tom Herbert; +Cc: David Miller, netdev

Le vendredi 16 avril 2010 à 08:35 -0700, Tom Herbert a écrit :
> Eric, thanks for testing that.  Admittedly, we have looked at enabling
> RFS/RPS over loopback.   I'll look at that today also.
> 
> 

Hi Tom

I am sorry, but I could not work on this today. I hope I can find some
time a bit later.



> On Thu, Apr 15, 2010 at 11:56 PM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> > Le jeudi 15 avril 2010 à 23:33 -0700, David Miller a écrit :
> >> From: Tom Herbert <therbert@google.com>
> >> Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)
> >>
> >> > Version 5 of RFS:
> >> > - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
> >> > static function.
> >> > - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
> >> > sysfs variable.
> >>
> >> I've read this over a few times and I think it's ready to go into
> >> net-next-2.6, we can tweak things as-needed from here on out.
> >>
> >> Eric, what do you think?
> >
> > I read the patch and found no error.
> >
> > I booted a test machine and performed some tests
> >
> > I am a bit worried of a tbench regression I am looking at right now.
> >
> > if RFS disabled , tbench 16   ->  4408.63 MB/sec
> >
> >
> > # grep . /sys/class/net/lo/queues/rx-0/*
> > /sys/class/net/lo/queues/rx-0/rps_cpus:00000000
> > /sys/class/net/lo/queues/rx-0/rps_flow_cnt:8192
> > # cat /proc/sys/net/core/rps_sock_flow_entries
> > 8192
> >
> >
> > echo ffff >/sys/class/net/lo/queues/rx-0/rps_cpus
> >
> > tbench 16 -> 2336.32 MB/sec
> >
> >
> > -----------------------------------------------------------------------------------------------------------------------------------------------------
> >   PerfTop:   14561 irqs/sec  kernel:86.3% [1000Hz cycles],  (all, 16 CPUs)
> > -----------------------------------------------------------------------------------------------------------------------------------------------------
> >
> >             samples  pcnt function                       DSO
> >             _______ _____ ______________________________ __________________________________________________________
> >
> >             2664.00  5.1% copy_user_generic_string       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >             2323.00  4.4% acpi_os_read_port              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >             1641.00  3.1% _raw_spin_lock_irqsave         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >             1260.00  2.4% schedule                       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >             1159.00  2.2% _raw_spin_lock                 /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >             1051.00  2.0% tcp_ack                        /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              991.00  1.9% tcp_sendmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              922.00  1.8% tcp_recvmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              821.00  1.6% child_run                      /usr/bin/tbench
> >              766.00  1.5% all_string_sub                 /usr/bin/tbench
> >              630.00  1.2% __switch_to                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              608.00  1.2% __GI_strchr                    /lib/tls/libc-2.3.4.so
> >              606.00  1.2% ipt_do_table                   /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              600.00  1.1% __GI_strstr                    /lib/tls/libc-2.3.4.so
> >              556.00  1.1% __netif_receive_skb            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              504.00  1.0% tcp_transmit_skb               /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              502.00  1.0% tick_nohz_stop_sched_tick      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              481.00  0.9% _raw_spin_unlock_irqrestore    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              473.00  0.9% next_token                     /usr/bin/tbench
> >              449.00  0.9% ip_rcv                         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              423.00  0.8% call_function_single_interrupt /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              422.00  0.8% ia32_sysenter_target           /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              420.00  0.8% compat_sys_socketcall          /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              401.00  0.8% mod_timer                      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              400.00  0.8% process_backlog                /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              399.00  0.8% ip_queue_xmit                  /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              387.00  0.7% select_task_rq_fair            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              377.00  0.7% _raw_spin_lock_bh              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >              360.00  0.7% tcp_v4_rcv                     /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
> >
> > But if RFS is on, why activating rps_cpus change tbench ?
> >
> >
> >
> >
> --
> To unsubscribe from this list: send the line "unsubscribe netdev" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> 



^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  6:56   ` Eric Dumazet
  2010-04-16  7:18     ` Eric Dumazet
  2010-04-16 15:35     ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
@ 2010-04-16 18:35     ` Tom Herbert
  2010-04-16 18:53       ` Eric Dumazet
  2 siblings, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-16 18:35 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: David Miller, netdev

Results with "tbench 16" on an 8 core Intel machine.

No RPS/RFS:  2155 MB/sec
RPS (0ff mask): 1700 MB/sec
RFS: 1097

I am not particularly surprised by the results, using loopback
interface already provides good parallelism and RPS/RFS really would
only add overhead and more trips between CPUs (last part is why RPS <
RFS I suspect)-- I guess this is why we've never enabled RPS on
loopback :-)

Eric, do you have a particular concern that this could affect a real workload?

Tom


On Thu, Apr 15, 2010 at 11:56 PM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> Le jeudi 15 avril 2010 à 23:33 -0700, David Miller a écrit :
>> From: Tom Herbert <therbert@google.com>
>> Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)
>>
>> > Version 5 of RFS:
>> > - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
>> > static function.
>> > - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
>> > sysfs variable.
>>
>> I've read this over a few times and I think it's ready to go into
>> net-next-2.6, we can tweak things as-needed from here on out.
>>
>> Eric, what do you think?
>
> I read the patch and found no error.
>
> I booted a test machine and performed some tests
>
> I am a bit worried of a tbench regression I am looking at right now.
>
> if RFS disabled , tbench 16   ->  4408.63 MB/sec
>
>
> # grep . /sys/class/net/lo/queues/rx-0/*
> /sys/class/net/lo/queues/rx-0/rps_cpus:00000000
> /sys/class/net/lo/queues/rx-0/rps_flow_cnt:8192
> # cat /proc/sys/net/core/rps_sock_flow_entries
> 8192
>
>
> echo ffff >/sys/class/net/lo/queues/rx-0/rps_cpus
>
> tbench 16 -> 2336.32 MB/sec
>
>
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>   PerfTop:   14561 irqs/sec  kernel:86.3% [1000Hz cycles],  (all, 16 CPUs)
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>
>             samples  pcnt function                       DSO
>             _______ _____ ______________________________ __________________________________________________________
>
>             2664.00  5.1% copy_user_generic_string       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             2323.00  4.4% acpi_os_read_port              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1641.00  3.1% _raw_spin_lock_irqsave         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1260.00  2.4% schedule                       /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1159.00  2.2% _raw_spin_lock                 /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>             1051.00  2.0% tcp_ack                        /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              991.00  1.9% tcp_sendmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              922.00  1.8% tcp_recvmsg                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              821.00  1.6% child_run                      /usr/bin/tbench
>              766.00  1.5% all_string_sub                 /usr/bin/tbench
>              630.00  1.2% __switch_to                    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              608.00  1.2% __GI_strchr                    /lib/tls/libc-2.3.4.so
>              606.00  1.2% ipt_do_table                   /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              600.00  1.1% __GI_strstr                    /lib/tls/libc-2.3.4.so
>              556.00  1.1% __netif_receive_skb            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              504.00  1.0% tcp_transmit_skb               /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              502.00  1.0% tick_nohz_stop_sched_tick      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              481.00  0.9% _raw_spin_unlock_irqrestore    /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              473.00  0.9% next_token                     /usr/bin/tbench
>              449.00  0.9% ip_rcv                         /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              423.00  0.8% call_function_single_interrupt /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              422.00  0.8% ia32_sysenter_target           /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              420.00  0.8% compat_sys_socketcall          /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              401.00  0.8% mod_timer                      /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              400.00  0.8% process_backlog                /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              399.00  0.8% ip_queue_xmit                  /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              387.00  0.7% select_task_rq_fair            /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              377.00  0.7% _raw_spin_lock_bh              /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>              360.00  0.7% tcp_v4_rcv                     /lib/modules/2.6.34-rc3-03375-ga4fbf84-dirty/build/vmlinux
>
> But if RFS is on, why activating rps_cpus change tbench ?
>
>
>
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 18:35     ` Tom Herbert
@ 2010-04-16 18:53       ` Eric Dumazet
  2010-04-16 20:42         ` Tom Herbert
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16 18:53 UTC (permalink / raw)
  To: Tom Herbert; +Cc: David Miller, netdev

Le vendredi 16 avril 2010 à 11:35 -0700, Tom Herbert a écrit :
> Results with "tbench 16" on an 8 core Intel machine.
> 
> No RPS/RFS:  2155 MB/sec
> RPS (0ff mask): 1700 MB/sec
> RFS: 1097
> 
> I am not particularly surprised by the results, using loopback
> interface already provides good parallelism and RPS/RFS really would
> only add overhead and more trips between CPUs (last part is why RPS <
> RFS I suspect)-- I guess this is why we've never enabled RPS on
> loopback :-)
> 
> Eric, do you have a particular concern that this could affect a real workload?
> 

I was expecting RFS to be better than RPS at least, for this particular
workload (tcp over loopback)

With RPS, the hash function of (127.0.0.1, port1, 127.0.0.1, port2)
is different than (127.0.0.1, port2, 127.0.0.1, port1), so basically we
force the server to run on different processor than client

However, I was expecting that with RFS, client and server would run on
same cpu.

Maybe we could change (for a test) hash function to use  (sport ^ dport)
instead of (sport << 16) + dport 




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16  6:33 ` David Miller
  2010-04-16  6:56   ` Eric Dumazet
@ 2010-04-16 19:37   ` Eric Dumazet
  2010-04-16 22:49     ` David Miller
  1 sibling, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16 19:37 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le jeudi 15 avril 2010 à 23:33 -0700, David Miller a écrit :
> From: Tom Herbert <therbert@google.com>
> Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)
> 
> > Version 5 of RFS:
> > - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
> > static function.
> > - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
> > sysfs variable.
> 
> I've read this over a few times and I think it's ready to go into
> net-next-2.6, we can tweak things as-needed from here on out.
> 
> Eric, what do you think?

I think I can give my Sob, and we have time to fully test it and tweak
it if necessary.

Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>

Thanks Tom !



^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 18:53       ` Eric Dumazet
@ 2010-04-16 20:42         ` Tom Herbert
  2010-04-16 21:12           ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-16 20:42 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: David Miller, netdev

On Fri, Apr 16, 2010 at 11:53 AM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> Le vendredi 16 avril 2010 à 11:35 -0700, Tom Herbert a écrit :
>> Results with "tbench 16" on an 8 core Intel machine.
>>
>> No RPS/RFS:  2155 MB/sec
>> RPS (0ff mask): 1700 MB/sec
>> RFS: 1097
>>

Blah, I mistakingly reported that... should have been:

No RPS/RFS:  2155 MB/sec
RPS (0ff mask): 1097 MB/sec
RFS: 1700 MB/sec

Sorry about that!

>> I am not particularly surprised by the results, using loopback
>> interface already provides good parallelism and RPS/RFS really would
>> only add overhead and more trips between CPUs (last part is why RPS <
>> RFS I suspect)-- I guess this is why we've never enabled RPS on
>> loopback :-)
>>
>> Eric, do you have a particular concern that this could affect a real workload?
>>
>
> I was expecting RFS to be better than RPS at least, for this particular
> workload (tcp over loopback)
>
This was my expectation too, and what my "corrected" numbers show :-)
But, I take it this is different in your results?

Tom

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 20:42         ` Tom Herbert
@ 2010-04-16 21:12           ` Eric Dumazet
  2010-04-16 21:25             ` Eric Dumazet
  2010-04-17 16:10             ` Eric Dumazet
  0 siblings, 2 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16 21:12 UTC (permalink / raw)
  To: Tom Herbert; +Cc: David Miller, netdev

Le vendredi 16 avril 2010 à 13:42 -0700, Tom Herbert a écrit :
> On Fri, Apr 16, 2010 at 11:53 AM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> > Le vendredi 16 avril 2010 à 11:35 -0700, Tom Herbert a écrit :
> >> Results with "tbench 16" on an 8 core Intel machine.
> >>
> >> No RPS/RFS:  2155 MB/sec
> >> RPS (0ff mask): 1700 MB/sec
> >> RFS: 1097
> >>
> 
> Blah, I mistakingly reported that... should have been:
> 
> No RPS/RFS:  2155 MB/sec
> RPS (0ff mask): 1097 MB/sec
> RFS: 1700 MB/sec
> 
> Sorry about that!

> This was my expectation too, and what my "corrected" numbers show :-)
> But, I take it this is different in your results?


My results are on a "tbench 16" on an dual X5570  @ 2.93GHz.
(16 logical cpus)

No RPS , no RFS : 4448.14 MB/sec 
RPS : 2298.00 MB/sec (but lot of variation)
RFS : 2600 MB/sec

Maybe my RFS setup is bad ?
(8192 flows)



^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 21:12           ` Eric Dumazet
@ 2010-04-16 21:25             ` Eric Dumazet
  2010-04-17 16:10             ` Eric Dumazet
  1 sibling, 0 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-16 21:25 UTC (permalink / raw)
  To: Tom Herbert; +Cc: David Miller, netdev

Le vendredi 16 avril 2010 à 23:12 +0200, Eric Dumazet a écrit :
> Le vendredi 16 avril 2010 à 13:42 -0700, Tom Herbert a écrit :
> > On Fri, Apr 16, 2010 at 11:53 AM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> > > Le vendredi 16 avril 2010 à 11:35 -0700, Tom Herbert a écrit :
> > >> Results with "tbench 16" on an 8 core Intel machine.
> > >>
> > >> No RPS/RFS:  2155 MB/sec
> > >> RPS (0ff mask): 1700 MB/sec
> > >> RFS: 1097
> > >>
> > 
> > Blah, I mistakingly reported that... should have been:
> > 
> > No RPS/RFS:  2155 MB/sec
> > RPS (0ff mask): 1097 MB/sec
> > RFS: 1700 MB/sec
> > 
> > Sorry about that!
> 
> > This was my expectation too, and what my "corrected" numbers show :-)
> > But, I take it this is different in your results?
> 
> 
> My results are on a "tbench 16" on an dual X5570  @ 2.93GHz.
> (16 logical cpus)
> 
> No RPS , no RFS : 4448.14 MB/sec 
> RPS : 2298.00 MB/sec (but lot of variation)
> RFS : 2600 MB/sec
> 
> Maybe my RFS setup is bad ?
> (8192 flows)
> 

Very strange, a second tbench-16 RFS=y run gave me 2134.08 MB/sec 

A third run gave me 1813.21 MB/sec 
A fourth run gave me 2472.91 MB/sec 

Hmm...





^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 19:37   ` Eric Dumazet
@ 2010-04-16 22:49     ` David Miller
  2010-04-16 22:53       ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-16 22:49 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Fri, 16 Apr 2010 21:37:59 +0200

> Le jeudi 15 avril 2010 à 23:33 -0700, David Miller a écrit :
>> From: Tom Herbert <therbert@google.com>
>> Date: Thu, 15 Apr 2010 22:47:08 -0700 (PDT)
>> 
>> > Version 5 of RFS:
>> > - Moved rps_sock_flow_sysctl into net/core/sysctl_net_core.c as a
>> > static function.
>> > - Apply limits to rps_sock_flow_entires systcl and rps_flow_count
>> > sysfs variable.
>> 
>> I've read this over a few times and I think it's ready to go into
>> net-next-2.6, we can tweak things as-needed from here on out.
>> 
>> Eric, what do you think?
> 
> I think I can give my Sob, and we have time to fully test it and tweak
> it if necessary.
> 
> Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>

Great, I'll add this to net-next-2.6 right now.

Thanks!

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 22:49     ` David Miller
@ 2010-04-16 22:53       ` David Miller
  2010-04-16 22:57         ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-16 22:53 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: David Miller <davem@davemloft.net>
Date: Fri, 16 Apr 2010 15:49:32 -0700 (PDT)

> Great, I'll add this to net-next-2.6 right now.

I had to add an include of linux/vmalloc.h to net/core/sysctl_net_core.c
to fix the build while committing this.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 22:53       ` David Miller
@ 2010-04-16 22:57         ` David Miller
  2010-04-17  0:22           ` Tom Herbert
  0 siblings, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-16 22:57 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: David Miller <davem@davemloft.net>
Date: Fri, 16 Apr 2010 15:53:40 -0700 (PDT)

> From: David Miller <davem@davemloft.net>
> Date: Fri, 16 Apr 2010 15:49:32 -0700 (PDT)
> 
>> Great, I'll add this to net-next-2.6 right now.
> 
> I had to add an include of linux/vmalloc.h to net/core/sysctl_net_core.c
> to fix the build while committing this.

net/core/net-sysfs.c needed it too :-/

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 22:57         ` David Miller
@ 2010-04-17  0:22           ` Tom Herbert
  2010-04-17  0:58             ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-17  0:22 UTC (permalink / raw)
  To: David Miller; +Cc: eric.dumazet, netdev

Ugh, vmalloc.h must be sneaking in through some other header file for
me :-(  Sorry about that.  Do you need me to respin the patch?

Tom

On Fri, Apr 16, 2010 at 3:57 PM, David Miller <davem@davemloft.net> wrote:
> From: David Miller <davem@davemloft.net>
> Date: Fri, 16 Apr 2010 15:53:40 -0700 (PDT)
>
>> From: David Miller <davem@davemloft.net>
>> Date: Fri, 16 Apr 2010 15:49:32 -0700 (PDT)
>>
>>> Great, I'll add this to net-next-2.6 right now.
>>
>> I had to add an include of linux/vmalloc.h to net/core/sysctl_net_core.c
>> to fix the build while committing this.
>
> net/core/net-sysfs.c needed it too :-/
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-17  0:22           ` Tom Herbert
@ 2010-04-17  0:58             ` David Miller
  0 siblings, 0 replies; 60+ messages in thread
From: David Miller @ 2010-04-17  0:58 UTC (permalink / raw)
  To: therbert; +Cc: eric.dumazet, netdev

From: Tom Herbert <therbert@google.com>
Date: Fri, 16 Apr 2010 17:22:49 -0700

> Ugh, vmalloc.h must be sneaking in through some other header file for
> me :-(  Sorry about that.  Do you need me to respin the patch?

No, I took care of it and am about to push things out to net-next-2.6
on kernel.org

^ permalink raw reply	[flat|nested] 60+ messages in thread

* [PATCH net-next-2.6] rps: rps_sock_flow_table is mostly read
  2010-04-16  7:48         ` Eric Dumazet
@ 2010-04-17  7:52           ` Eric Dumazet
  2010-04-17  7:57             ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-17  7:52 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev


Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>
---
diff --git a/net/core/dev.c b/net/core/dev.c
index d7107ac..7abf959 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -2205,7 +2205,7 @@ DEFINE_PER_CPU(struct netif_rx_stats, netdev_rx_stat) = { 0, };
 #ifdef CONFIG_RPS
 
 /* One global table that all flow-based protocols share. */
-struct rps_sock_flow_table *rps_sock_flow_table;
+struct rps_sock_flow_table *rps_sock_flow_table __read_mostly;
 EXPORT_SYMBOL(rps_sock_flow_table);
 
 /*



^ permalink raw reply related	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: rps_sock_flow_table is mostly read
  2010-04-17  7:52           ` [PATCH net-next-2.6] rps: rps_sock_flow_table is mostly read Eric Dumazet
@ 2010-04-17  7:57             ` David Miller
  0 siblings, 0 replies; 60+ messages in thread
From: David Miller @ 2010-04-17  7:57 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Sat, 17 Apr 2010 09:52:13 +0200

> 
> Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>

Applied, thanks Eric.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-16 21:12           ` Eric Dumazet
  2010-04-16 21:25             ` Eric Dumazet
@ 2010-04-17 16:10             ` Eric Dumazet
  2010-04-17 17:38               ` Tom Herbert
  2010-04-19 20:09               ` David Miller
  1 sibling, 2 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-17 16:10 UTC (permalink / raw)
  To: Tom Herbert; +Cc: David Miller, netdev

Le vendredi 16 avril 2010 à 23:12 +0200, Eric Dumazet a écrit :
> Le vendredi 16 avril 2010 à 13:42 -0700, Tom Herbert a écrit :
> > On Fri, Apr 16, 2010 at 11:53 AM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> > > Le vendredi 16 avril 2010 à 11:35 -0700, Tom Herbert a écrit :
> > >> Results with "tbench 16" on an 8 core Intel machine.
> > >>
> > >> No RPS/RFS:  2155 MB/sec
> > >> RPS (0ff mask): 1700 MB/sec
> > >> RFS: 1097
> > >>
> > 
> > Blah, I mistakingly reported that... should have been:
> > 
> > No RPS/RFS:  2155 MB/sec
> > RPS (0ff mask): 1097 MB/sec
> > RFS: 1700 MB/sec
> > 
> > Sorry about that!
> 
> > This was my expectation too, and what my "corrected" numbers show :-)
> > But, I take it this is different in your results?
> 
> 
> My results are on a "tbench 16" on an dual X5570  @ 2.93GHz.
> (16 logical cpus)
> 
> No RPS , no RFS : 4448.14 MB/sec 
> RPS : 2298.00 MB/sec (but lot of variation)
> RFS : 2600 MB/sec
> 
> Maybe my RFS setup is bad ?
> (8192 flows)
> 

With attached patch, I reached 

Throughput 4465.13 MB/sec 16 procs

RFS better than no RPS/RFS :)

So, the old idea to make rxhash consistent (same value in both
directions) is a win for some workloads (Consider connection tracking /
firewalling) 

port1 = ...
port2 = ...
addr1 = ...
addr2 = ...
if (addr1 > addr2)
	exchange(addr1, addr2)
if (port1 > port2)
	exchange(port, port2)

hash = jhash(addr1, addr2, (port1<<16)+port2, ...)

diff --git a/net/core/dev.c b/net/core/dev.c
index 7abf959..6b757ff 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -2280,8 +2280,10 @@ static int get_rps_cpu(struct net_device *dev,
struct sk_buff *skb,
        case IPPROTO_AH:
        case IPPROTO_SCTP:
        case IPPROTO_UDPLITE:
-               if (pskb_may_pull(skb, (ihl * 4) + 4))
-                       ports = *((u32 *) (skb->data + (ihl * 4)));
+               if (pskb_may_pull(skb, (ihl * 4) + 4)) {
+                       u16 *_ports = (u16 *)(skb->data + (ihl * 4));
+                       ports = _ports[0] ^ _ports[1];
+               }
                break;
 
        default:



^ permalink raw reply related	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-17 16:10             ` Eric Dumazet
@ 2010-04-17 17:38               ` Tom Herbert
  2010-04-18  0:06                 ` Changli Gao
  2010-04-19 20:09               ` David Miller
  1 sibling, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-17 17:38 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: David Miller, netdev

>
> With attached patch, I reached
>
> Throughput 4465.13 MB/sec 16 procs
>
> RFS better than no RPS/RFS :)
>
> So, the old idea to make rxhash consistent (same value in both
> directions) is a win for some workloads (Consider connection tracking /
> firewalling)
>
> port1 = ...
> port2 = ...
> addr1 = ...
> addr2 = ...
> if (addr1 > addr2)
>        exchange(addr1, addr2)
> if (port1 > port2)
>        exchange(port, port2)
>
> hash = jhash(addr1, addr2, (port1<<16)+port2, ...)
>
> diff --git a/net/core/dev.c b/net/core/dev.c
> index 7abf959..6b757ff 100644
> --- a/net/core/dev.c
> +++ b/net/core/dev.c
> @@ -2280,8 +2280,10 @@ static int get_rps_cpu(struct net_device *dev,
> struct sk_buff *skb,
>        case IPPROTO_AH:
>        case IPPROTO_SCTP:
>        case IPPROTO_UDPLITE:
> -               if (pskb_may_pull(skb, (ihl * 4) + 4))
> -                       ports = *((u32 *) (skb->data + (ihl * 4)));
> +               if (pskb_may_pull(skb, (ihl * 4) + 4)) {
> +                       u16 *_ports = (u16 *)(skb->data + (ihl * 4));
> +                       ports = _ports[0] ^ _ports[1];
> +               }
>                break;
>
>        default:
>
That's cool!, but I still like the idea that this hash is treated as
an opaque value, getting the hash from the device to avoid the jhash
or cache misses on the packet can also be a win...  Maybe connection
tracking/firewall could use the skb->rxhash which provides the
consistency and also eliminates the need to do more jhashes.


>
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-17 17:38               ` Tom Herbert
@ 2010-04-18  0:06                 ` Changli Gao
  2010-04-18 11:06                   ` Franco Fichtner
  0 siblings, 1 reply; 60+ messages in thread
From: Changli Gao @ 2010-04-18  0:06 UTC (permalink / raw)
  To: Tom Herbert; +Cc: Eric Dumazet, David Miller, netdev

On Sun, Apr 18, 2010 at 1:38 AM, Tom Herbert <therbert@google.com> wrote:
> That's cool!, but I still like the idea that this hash is treated as
> an opaque value getting the hash from the device to avoid the jhash
> or cache misses on the packet can also be a win...  Maybe connection
> tracking/firewall could use the skb->rxhash which provides the
> consistency and also eliminates the need to do more jhashes.
>

consistent rxhash only adds the risk of the hash collision, and I
don't think it is a big problem. For connection tracking/firewall use,
I am afraid that we have to recompute this value after defrag.  So we
have to export the hash function we used in RPS.

As NIC's hash function can be changed dynamically, the rxhash isn't
consistent, so the rxhash can't be used by connection tracking, socket
lookup and others come later.

-- 
Regards,
Changli Gao(xiaosuo@gmail.com)

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-18  0:06                 ` Changli Gao
@ 2010-04-18 11:06                   ` Franco Fichtner
  0 siblings, 0 replies; 60+ messages in thread
From: Franco Fichtner @ 2010-04-18 11:06 UTC (permalink / raw)
  To: Changli Gao; +Cc: Tom Herbert, Eric Dumazet, David Miller, netdev

Changli Gao wrote:
> On Sun, Apr 18, 2010 at 1:38 AM, Tom Herbert <therbert@google.com> wrote:
>   
>> That's cool!, but I still like the idea that this hash is treated as
>> an opaque value getting the hash from the device to avoid the jhash
>> or cache misses on the packet can also be a win...  Maybe connection
>> tracking/firewall could use the skb->rxhash which provides the
>> consistency and also eliminates the need to do more jhashes.
>>
>>     
>
> consistent rxhash only adds the risk of the hash collision, and I
> don't think it is a big problem. For connection tracking/firewall use,
> I am afraid that we have to recompute this value after defrag.  So we
> have to export the hash function we used in RPS.
>
> As NIC's hash function can be changed dynamically, the rxhash isn't
> consistent, so the rxhash can't be used by connection tracking, socket
> lookup and others come later.
>
>   
I have to agree with Eric and Changli here.

It's especially true if you're passively tracking via one NIC, where all 
traffic is just forwarded.
In this scenario, you need to compute consistent hashes. rxhashes by NIC 
will be different for
"incoming" and "outgoing" traffic...

Where rxhash by NIC can be used (note: didn't say _useful_) are 
scenarios with different net
ports for incoming and outgoing traffic (in active but also passive 
traffic scenarios). Here,
rxhashes could be used on a per-port basis, but associating two 
seemingly separate rxhashes
with one another to match CPUs is a really annoying task. This would 
involve computing the
corresponding "txhash" and looking it up, which is what we'd be doing 
with the jhash anyway.

For proper flow tracking Eric's suggestion is the way to go. And if 
there are worries about
collisions, why not add IPPROTO_* to the mix.


Franco

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-17 16:10             ` Eric Dumazet
  2010-04-17 17:38               ` Tom Herbert
@ 2010-04-19 20:09               ` David Miller
  2010-04-19 20:23                 ` David Miller
  1 sibling, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-19 20:09 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Sat, 17 Apr 2010 18:10:33 +0200

> Le vendredi 16 avril 2010 à 23:12 +0200, Eric Dumazet a écrit :
>> My results are on a "tbench 16" on an dual X5570  @ 2.93GHz.
>> (16 logical cpus)
>> 
>> No RPS , no RFS : 4448.14 MB/sec 
>> RPS : 2298.00 MB/sec (but lot of variation)
>> RFS : 2600 MB/sec
>> 
>> Maybe my RFS setup is bad ?
>> (8192 flows)
>> 
> 
> With attached patch, I reached 
> 
> Throughput 4465.13 MB/sec 16 procs
> 
> RFS better than no RPS/RFS :)
> 
> So, the old idea to make rxhash consistent (same value in both
> directions) is a win for some workloads (Consider connection tracking /
> firewalling) 

Fun :-)  I toyed around with this on my 128 cpu machine (2 NUMA
nodes, 64 cpus each NUMA node).

Vanilla net-next-2.6, no configuration changes:

tbench 64: Throughput 1843.43 MB/sec 64 procs
tbench 128: Throughput 1889.67 MB/sec 128 procs

Vanilla net-next-2.6, rps_cpus="ffffffff,ffffffff,ffffffff,ffffffff"

tbench 64: Throughput 1455.89 MB/sec 64 procs
tbench 128: Throughput 2009.91 MB/sec 128 procs

net-next-2.6 + Eric's port hashing patch, rps_cpus="ffffffff,ffffffff,ffffffff,ffffffff"

tbench 64: Throughput 1593.13 MB/sec 64 procs
tbench 128: Throughput 2367.27 MB/sec 128 procs


^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-19 20:09               ` David Miller
@ 2010-04-19 20:23                 ` David Miller
  2010-04-19 20:32                   ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-19 20:23 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: David Miller <davem@davemloft.net>
Date: Mon, 19 Apr 2010 13:09:05 -0700 (PDT)

> net-next-2.6 + Eric's port hashing patch, rps_cpus="ffffffff,ffffffff,ffffffff,ffffffff"
> 
> tbench 64: Throughput 1593.13 MB/sec 64 procs
> tbench 128: Throughput 2367.27 MB/sec 128 procs

Eric, I think there is agreement that your patch is not a bad idea.

Your original posting had whitespace damange in the patch plus I want
to see a proper commit message and signoff, so could you please submit
this formally?

Thanks!

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-19 20:23                 ` David Miller
@ 2010-04-19 20:32                   ` Eric Dumazet
  2010-04-19 21:19                     ` David Miller
  2010-04-19 23:38                     ` Changli Gao
  0 siblings, 2 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-19 20:32 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le lundi 19 avril 2010 à 13:23 -0700, David Miller a écrit :
> From: David Miller <davem@davemloft.net>
> Date: Mon, 19 Apr 2010 13:09:05 -0700 (PDT)
> 
> > net-next-2.6 + Eric's port hashing patch, rps_cpus="ffffffff,ffffffff,ffffffff,ffffffff"
> > 
> > tbench 64: Throughput 1593.13 MB/sec 64 procs
> > tbench 128: Throughput 2367.27 MB/sec 128 procs
> 
> Eric, I think there is agreement that your patch is not a bad idea.
> 
> Your original posting had whitespace damange in the patch plus I want
> to see a proper commit message and signoff, so could you please submit
> this formally?

Hmm, this was not a formal patch, just an information.

Problem is if hardware provides rxhash, will it be "consistent" too ?




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-19 20:32                   ` Eric Dumazet
@ 2010-04-19 21:19                     ` David Miller
  2010-04-26  8:41                       ` Eric Dumazet
  2010-04-19 23:38                     ` Changli Gao
  1 sibling, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-19 21:19 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Mon, 19 Apr 2010 22:32:01 +0200

> Hmm, this was not a formal patch, just an information.
> 
> Problem is if hardware provides rxhash, will it be "consistent" too ?

Yes, it is an issue.  I am not aware of whether the Toeplitz
hash computed by cards is impervious to the order of the
input bits of not, probably it is.

I was thinking also about how we could compute rxhash in the
loopback driver :-)

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-19 20:32                   ` Eric Dumazet
  2010-04-19 21:19                     ` David Miller
@ 2010-04-19 23:38                     ` Changli Gao
  2010-04-20  5:59                       ` Eric Dumazet
  1 sibling, 1 reply; 60+ messages in thread
From: Changli Gao @ 2010-04-19 23:38 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: David Miller, therbert, netdev

On Tue, Apr 20, 2010 at 4:32 AM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
>
> Hmm, this was not a formal patch, just an information.
>
> Problem is if hardware provides rxhash, will it be "consistent" too ?
>
>

Does this problem has relationship with your patch? No. If the rxhash
isn't provided by hardware, we can get more throughput from you patch,
and on the other side, we don't lose anything but potential more hash
collision.

-- 
Regards,
Changli Gao(xiaosuo@gmail.com)

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-19 23:38                     ` Changli Gao
@ 2010-04-20  5:59                       ` Eric Dumazet
  2010-04-20  7:56                         ` [PATCH net-next-2.6] rps: consistent rxhash Eric Dumazet
  2010-04-20 15:04                         ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
  0 siblings, 2 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-20  5:59 UTC (permalink / raw)
  To: Changli Gao; +Cc: David Miller, therbert, netdev

Le mardi 20 avril 2010 à 07:38 +0800, Changli Gao a écrit :

> Does this problem has relationship with your patch? No. If the rxhash
> isn't provided by hardware, we can get more throughput from you patch,
> and on the other side, we don't lose anything but potential more hash
> collision.
> 

I am not sure what you call hash collision. There is no hash chain here.

This 32bit hash is a jhash one, and we only need 1 to 12 bits in it, I
am pretty sure its OK.




^ permalink raw reply	[flat|nested] 60+ messages in thread

* [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20  5:59                       ` Eric Dumazet
@ 2010-04-20  7:56                         ` Eric Dumazet
  2010-04-20  8:18                           ` David Miller
  2010-04-20 12:48                           ` Franco Fichtner
  2010-04-20 15:04                         ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
  1 sibling, 2 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-20  7:56 UTC (permalink / raw)
  To: Changli Gao, David Miller; +Cc: therbert, netdev

In case we compute a software skb->rxhash, we can generate a consistent
hash : Its value will be the same in both flow directions.

This helps some workloads, like conntracking, since the same state needs
to be accessed in both directions.

tbench + RFS + this patch gives better results than tbench with default
kernel configuration (no RPS, no RFS)

Also fixed some sparse warnings.

Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>
---
 net/core/dev.c |   25 ++++++++++++++++++-------
 1 files changed, 18 insertions(+), 7 deletions(-)

diff --git a/net/core/dev.c b/net/core/dev.c
index 05a2b29..cb150ec 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -1974,7 +1974,7 @@ u16 skb_tx_hash(const struct net_device *dev, const struct sk_buff *skb)
 	if (skb->sk && skb->sk->sk_hash)
 		hash = skb->sk->sk_hash;
 	else
-		hash = skb->protocol;
+		hash = (__force u16) skb->protocol;
 
 	hash = jhash_1word(hash, hashrnd);
 
@@ -2253,8 +2253,8 @@ static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb,
 
 		ip = (struct iphdr *) skb->data;
 		ip_proto = ip->protocol;
-		addr1 = ip->saddr;
-		addr2 = ip->daddr;
+		addr1 = (__force u32) ip->saddr;
+		addr2 = (__force u32) ip->daddr;
 		ihl = ip->ihl;
 		break;
 	case __constant_htons(ETH_P_IPV6):
@@ -2263,8 +2263,8 @@ static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb,
 
 		ip6 = (struct ipv6hdr *) skb->data;
 		ip_proto = ip6->nexthdr;
-		addr1 = ip6->saddr.s6_addr32[3];
-		addr2 = ip6->daddr.s6_addr32[3];
+		addr1 = (__force u32) ip6->saddr.s6_addr32[3];
+		addr2 = (__force u32) ip6->daddr.s6_addr32[3];
 		ihl = (40 >> 2);
 		break;
 	default:
@@ -2279,14 +2279,25 @@ static int get_rps_cpu(struct net_device *dev, struct sk_buff *skb,
 	case IPPROTO_AH:
 	case IPPROTO_SCTP:
 	case IPPROTO_UDPLITE:
-		if (pskb_may_pull(skb, (ihl * 4) + 4))
-			ports = *((u32 *) (skb->data + (ihl * 4)));
+		if (pskb_may_pull(skb, (ihl * 4) + 4)) {
+			__be16 *hports = (__be16 *) (skb->data + (ihl * 4));
+			u32 sport, dport;
+
+			sport = (__force u16) hports[0];
+			dport = (__force u16) hports[1];
+			if (dport < sport)
+				swap(sport, dport);
+			ports = (sport << 16) + dport;
+		}
 		break;
 
 	default:
 		break;
 	}
 
+	/* get a consistent hash (same value on both flow directions) */
+	if (addr2 < addr1)
+		swap(addr1, addr2);
 	skb->rxhash = jhash_3words(addr1, addr2, ports, hashrnd);
 	if (!skb->rxhash)
 		skb->rxhash = 1;



^ permalink raw reply related	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20  7:56                         ` [PATCH net-next-2.6] rps: consistent rxhash Eric Dumazet
@ 2010-04-20  8:18                           ` David Miller
  2010-04-20 12:48                           ` Franco Fichtner
  1 sibling, 0 replies; 60+ messages in thread
From: David Miller @ 2010-04-20  8:18 UTC (permalink / raw)
  To: eric.dumazet; +Cc: xiaosuo, therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Tue, 20 Apr 2010 09:56:38 +0200

> In case we compute a software skb->rxhash, we can generate a consistent
> hash : Its value will be the same in both flow directions.
> 
> This helps some workloads, like conntracking, since the same state needs
> to be accessed in both directions.
> 
> tbench + RFS + this patch gives better results than tbench with default
> kernel configuration (no RPS, no RFS)
> 
> Also fixed some sparse warnings.
> 
> Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>

Applied.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20  7:56                         ` [PATCH net-next-2.6] rps: consistent rxhash Eric Dumazet
  2010-04-20  8:18                           ` David Miller
@ 2010-04-20 12:48                           ` Franco Fichtner
  2010-04-20 13:16                             ` Eric Dumazet
  2010-04-20 15:09                             ` Tom Herbert
  1 sibling, 2 replies; 60+ messages in thread
From: Franco Fichtner @ 2010-04-20 12:48 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: Changli Gao, David Miller, therbert, netdev

Eric Dumazet wrote:
> In case we compute a software skb->rxhash, we can generate a consistent
> hash : Its value will be the same in both flow directions.
>
> This helps some workloads, like conntracking, since the same state needs
> to be accessed in both directions.
>
> tbench + RFS + this patch gives better results than tbench with default
> kernel configuration (no RPS, no RFS)
>
> Also fixed some sparse warnings.
>
> Signed-off-by: Eric Dumazet <eric.dumazet@gmail.com>
> ---

I thought about this for some time...

Do we really need the port numbers here at all? A simple
addr1^addr2 can provide a good enough pointer for
distribution amongst CPUs.

The real connection tracking is better done locally at the
corresponding CPU. That way a potential cache miss can be
avoided and the still needed hash calculation for
connection tracking will be offloaded.


Franco


^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 12:48                           ` Franco Fichtner
@ 2010-04-20 13:16                             ` Eric Dumazet
  2010-04-20 14:03                               ` Franco Fichtner
  2010-04-20 15:09                             ` Tom Herbert
  1 sibling, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-20 13:16 UTC (permalink / raw)
  To: Franco Fichtner; +Cc: Changli Gao, David Miller, therbert, netdev

Le mardi 20 avril 2010 à 14:48 +0200, Franco Fichtner a écrit :

> 
> I thought about this for some time...
> 
> Do we really need the port numbers here at all? A simple
> addr1^addr2 can provide a good enough pointer for
> distribution amongst CPUs.
> 
> The real connection tracking is better done locally at the
> corresponding CPU. That way a potential cache miss can be
> avoided and the still needed hash calculation for
> connection tracking will be offloaded.
> 

Yes, doing the port test/swap is useful in the loopback case 
(addr1 == addr2).

This is probably a bit convoluted, but David (and me) found this
funny ;)



^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 13:16                             ` Eric Dumazet
@ 2010-04-20 14:03                               ` Franco Fichtner
  2010-04-20 14:57                                 ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: Franco Fichtner @ 2010-04-20 14:03 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: Changli Gao, David Miller, therbert, netdev

Eric Dumazet wrote:
> Le mardi 20 avril 2010 à 14:48 +0200, Franco Fichtner a écrit :
> 
>> I thought about this for some time...
>>
>> Do we really need the port numbers here at all? A simple
>> addr1^addr2 can provide a good enough pointer for
>> distribution amongst CPUs.
>>
>> The real connection tracking is better done locally at the
>> corresponding CPU. That way a potential cache miss can be
>> avoided and the still needed hash calculation for
>> connection tracking will be offloaded.
>>
> 
> Yes, doing the port test/swap is useful in the loopback case 
> (addr1 == addr2).
> 
> This is probably a bit convoluted, but David (and me) found this
> funny ;)
> 
> 

It is funny, but I fail to see the big picture of the
firewall / conntrack application here. It looks like
this is needed for local netperf tests to impress, but
it's a quite special use case, isn't it?

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 14:03                               ` Franco Fichtner
@ 2010-04-20 14:57                                 ` Eric Dumazet
  2010-04-20 21:41                                   ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-20 14:57 UTC (permalink / raw)
  To: Franco Fichtner; +Cc: Changli Gao, David Miller, therbert, netdev

Le mardi 20 avril 2010 à 16:03 +0200, Franco Fichtner a écrit :

> 
> It is funny, but I fail to see the big picture of the
> firewall / conntrack application here. It looks like
> this is needed for local netperf tests to impress, but
> it's a quite special use case, isn't it?

I know many applications using TCP on loopback, they are real :)

What I find 'funny' are not the tbench results, but the fact that RFS
can give pretty good hints to process scheduler, something that might be
good to investigate by scheduler specialists.

In the meantime, if some admin finds that setting RFS on loopback can
boost by 10% its application, why not ?




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-20  5:59                       ` Eric Dumazet
  2010-04-20  7:56                         ` [PATCH net-next-2.6] rps: consistent rxhash Eric Dumazet
@ 2010-04-20 15:04                         ` Tom Herbert
  2010-04-20 15:39                           ` Eric Dumazet
  1 sibling, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-20 15:04 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: Changli Gao, David Miller, netdev

On Mon, Apr 19, 2010 at 10:59 PM, Eric Dumazet <eric.dumazet@gmail.com> wrote:
> Le mardi 20 avril 2010 à 07:38 +0800, Changli Gao a écrit :
>
>> Does this problem has relationship with your patch? No. If the rxhash
>> isn't provided by hardware, we can get more throughput from you patch,
>> and on the other side, we don't lose anything but potential more hash
>> collision.
>>
>
> I am not sure what you call hash collision. There is no hash chain here.
>
> This 32bit hash is a jhash one, and we only need 1 to 12 bits in it, I
> am pretty sure its OK.
>
Maybe for the purposes of RPS, but hash collisions could definitely be
an issue in RFS.  If two active connections hit the same rps_flow
entry this may cause thrashing of those connections between CPUs.  I
think your patch may increase the probability of this happening.

>
>
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 12:48                           ` Franco Fichtner
  2010-04-20 13:16                             ` Eric Dumazet
@ 2010-04-20 15:09                             ` Tom Herbert
  2010-04-21  9:29                               ` Franco Fichtner
  1 sibling, 1 reply; 60+ messages in thread
From: Tom Herbert @ 2010-04-20 15:09 UTC (permalink / raw)
  To: Franco Fichtner; +Cc: Eric Dumazet, Changli Gao, David Miller, netdev

> I thought about this for some time...
>
> Do we really need the port numbers here at all? A simple
> addr1^addr2 can provide a good enough pointer for
> distribution amongst CPUs.
>

What about a server behind a TCP proxy?  Also, need to minimize
collisions for RPS to be effective.

Tom

> The real connection tracking is better done locally at the
> corresponding CPU. That way a potential cache miss can be
> avoided and the still needed hash calculation for
> connection tracking will be offloaded.
>
>
> Franco
>
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-20 15:04                         ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
@ 2010-04-20 15:39                           ` Eric Dumazet
  0 siblings, 0 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-20 15:39 UTC (permalink / raw)
  To: Tom Herbert; +Cc: Changli Gao, David Miller, netdev

Le mardi 20 avril 2010 à 08:04 -0700, Tom Herbert a écrit :

> Maybe for the purposes of RPS, but hash collisions could definitely be
> an issue in RFS.  If two active connections hit the same rps_flow
> entry this may cause thrashing of those connections between CPUs.  I
> think your patch may increase the probability of this happening.
> 

Good point.

I'll make a gathering of tcp tuples on a busy server over a day and try
to compute number of clashes we can get with and without the addr/port
swapping.



^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 14:57                                 ` Eric Dumazet
@ 2010-04-20 21:41                                   ` David Miller
  2010-04-20 23:35                                     ` Changli Gao
  2010-04-21 19:12                                     ` Tom Herbert
  0 siblings, 2 replies; 60+ messages in thread
From: David Miller @ 2010-04-20 21:41 UTC (permalink / raw)
  To: eric.dumazet; +Cc: franco, xiaosuo, therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Tue, 20 Apr 2010 16:57:01 +0200

> I know many applications using TCP on loopback, they are real :)

This is all true and I support your hashing patch and all of that.

But if we really want TCP over loopback to go fast, there are much
better ways to do this.

Eric, do you remember that "TCP friends" rough patch I sent you last
year that essentailly made TCP sockets over loopback behave like
AF_UNIX ones and just queue the SKBs directly to the destination
socket without doing any protocol work?

If we ever got that working, tbench performance would become
impressive :)

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 21:41                                   ` David Miller
@ 2010-04-20 23:35                                     ` Changli Gao
  2010-04-20 23:38                                       ` David Miller
  2010-04-21 19:12                                     ` Tom Herbert
  1 sibling, 1 reply; 60+ messages in thread
From: Changli Gao @ 2010-04-20 23:35 UTC (permalink / raw)
  To: David Miller; +Cc: eric.dumazet, franco, therbert, netdev

On Wed, Apr 21, 2010 at 5:41 AM, David Miller <davem@davemloft.net> wrote:
> Eric, do you remember that "TCP friends" rough patch I sent you last
> year that essentailly made TCP sockets over loopback behave like
> AF_UNIX ones and just queue the SKBs directly to the destination
> socket without doing any protocol work?

I think it will break some benchmark tools. The loopback device is for
testing networking protocol stacks, so we shouldn't bypass the
protocol processing. And anyone who has a performance problem of
loopback device should turn to UNIX domain socket.

For routers, how about letting users choose whether RPS mixes layer 4 info in?

-- 
Regards,
Changli Gao(xiaosuo@gmail.com)

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 23:35                                     ` Changli Gao
@ 2010-04-20 23:38                                       ` David Miller
  0 siblings, 0 replies; 60+ messages in thread
From: David Miller @ 2010-04-20 23:38 UTC (permalink / raw)
  To: xiaosuo; +Cc: eric.dumazet, franco, therbert, netdev

From: Changli Gao <xiaosuo@gmail.com>
Date: Wed, 21 Apr 2010 07:35:48 +0800

> On Wed, Apr 21, 2010 at 5:41 AM, David Miller <davem@davemloft.net> wrote:
>> Eric, do you remember that "TCP friends" rough patch I sent you last
>> year that essentailly made TCP sockets over loopback behave like
>> AF_UNIX ones and just queue the SKBs directly to the destination
>> socket without doing any protocol work?
> 
> I think it will break some benchmark tools.

Other systems already do this optimization, so if things break, this
breakage is already pervasive.

We should be able to tell people that they can use TCP solely in their
applications and it will perform optimally regardless of transport.

People already code their applications this way, and ignoring
this issue would just makes us stupid.

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 15:09                             ` Tom Herbert
@ 2010-04-21  9:29                               ` Franco Fichtner
  2010-04-21  9:39                                 ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: Franco Fichtner @ 2010-04-21  9:29 UTC (permalink / raw)
  To: Tom Herbert; +Cc: Eric Dumazet, Changli Gao, David Miller, netdev

Tom Herbert wrote:
>> I thought about this for some time...
>>
>> Do we really need the port numbers here at all? A simple
>> addr1^addr2 can provide a good enough pointer for
>> distribution amongst CPUs.
>>
> 
> What about a server behind a TCP proxy?  Also, need to minimize
> collisions for RPS to be effective

What about routers? What about loopback? This all boils down to
the same issue of obscuring IP data by "magical" means and then
reattaching functionality by reaching for upper layer information.
It is necessary in some cases, but it can cripple performance
for other cases.

The interesting thing is you don't need to deal with collisions
while distributing amonst cpus at all. You just need to make sure
the distribution algorithm keeps every single flow attached to
the correct cpu.

All of the actual flow hashing, tracking and whatever else the
traffic needs to go through can be done locally by cpu x which
helps a lot with load distribution and cache issues in mind. It
also helps locking because there is no global flow lookup table.
Oh, and it also reduces collisions with every cpu you add for
receiving.

I work with a lot of plain office and ISP traffic in mind daily,
so please don't misunderstand my motivation here. I'd hate to
see poor performance in scenarios in which there is a lot of
potential improvement.


Franco

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-21  9:29                               ` Franco Fichtner
@ 2010-04-21  9:39                                 ` Eric Dumazet
  2010-04-21 11:06                                   ` Franco Fichtner
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-21  9:39 UTC (permalink / raw)
  To: Franco Fichtner; +Cc: Tom Herbert, Changli Gao, David Miller, netdev

Le mercredi 21 avril 2010 à 11:29 +0200, Franco Fichtner a écrit :
> Tom Herbert wrote:
> >> I thought about this for some time...
> >>
> >> Do we really need the port numbers here at all? A simple
> >> addr1^addr2 can provide a good enough pointer for
> >> distribution amongst CPUs.
> >>
> > 
> > What about a server behind a TCP proxy?  Also, need to minimize
> > collisions for RPS to be effective
> 
> What about routers? What about loopback? This all boils down to
> the same issue of obscuring IP data by "magical" means and then
> reattaching functionality by reaching for upper layer information.
> It is necessary in some cases, but it can cripple performance
> for other cases.
> 
> The interesting thing is you don't need to deal with collisions
> while distributing amonst cpus at all. You just need to make sure
> the distribution algorithm keeps every single flow attached to
> the correct cpu.
> 
> All of the actual flow hashing, tracking and whatever else the
> traffic needs to go through can be done locally by cpu x which
> helps a lot with load distribution and cache issues in mind. It
> also helps locking because there is no global flow lookup table.
> Oh, and it also reduces collisions with every cpu you add for
> receiving.
> 
> I work with a lot of plain office and ISP traffic in mind daily,
> so please don't misunderstand my motivation here. I'd hate to
> see poor performance in scenarios in which there is a lot of
> potential improvement.
> 

I am a bit lost by this conversation.

Are you saying something is wrong with current schem ?

What are exactly your suggestions ?

Tom replied to you that a hash derived from (addr1 ^ addr2) would not
work in situations where all flows goes from machine A to machine B
(all hashes would be the same)

Current hash is probably more than enough to cover all situations.






^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-21  9:39                                 ` Eric Dumazet
@ 2010-04-21 11:06                                   ` Franco Fichtner
  2010-04-21 11:16                                     ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: Franco Fichtner @ 2010-04-21 11:06 UTC (permalink / raw)
  To: Eric Dumazet; +Cc: Tom Herbert, Changli Gao, David Miller, netdev

Eric Dumazet wrote:
> Le mercredi 21 avril 2010 à 11:29 +0200, Franco Fichtner a écrit :
>> Tom Herbert wrote:
>>>> I thought about this for some time...
>>>>
>>>> Do we really need the port numbers here at all? A simple
>>>> addr1^addr2 can provide a good enough pointer for
>>>> distribution amongst CPUs.
>>>>
>>> What about a server behind a TCP proxy?  Also, need to minimize
>>> collisions for RPS to be effective
>> What about routers? What about loopback? This all boils down to
>> the same issue of obscuring IP data by "magical" means and then
>> reattaching functionality by reaching for upper layer information.
>> It is necessary in some cases, but it can cripple performance
>> for other cases.
>>
>> The interesting thing is you don't need to deal with collisions
>> while distributing amonst cpus at all. You just need to make sure
>> the distribution algorithm keeps every single flow attached to
>> the correct cpu.
>>
>> All of the actual flow hashing, tracking and whatever else the
>> traffic needs to go through can be done locally by cpu x which
>> helps a lot with load distribution and cache issues in mind. It
>> also helps locking because there is no global flow lookup table.
>> Oh, and it also reduces collisions with every cpu you add for
>> receiving.
>>
>> I work with a lot of plain office and ISP traffic in mind daily,
>> so please don't misunderstand my motivation here. I'd hate to
>> see poor performance in scenarios in which there is a lot of
>> potential improvement.
>>
> 
> I am a bit lost by this conversation.
> 
> Are you saying something is wrong with current schem ?
> 
> What are exactly your suggestions ?
> 

Hashing for cpu distribution should be as minimal as it could possibly
be with the least number operations needed to compute a hash, which 
normally involves touching one cold cache line (ip header). If you add 
the ports to your mix you have the luxury of solving static ip mappings,
but only for protocols that support it. Usage of the destination port
may also prove to be more or less pointless with a lot of http traffic,
because it's most likely static. And you add another potential cold
cache line access. For a lot of traffic scenarios, we'll have a bunch of
internal ips and the internet on the other side, so having a simple hash
based on a flavor if internal/external ip is more than enough to work
with for distribution. If the network card can provide a complete hash
all the better. Then this part of my point is void.

But then, hashing for cpu distribution should have nothing todo with
real flow tracking with lookup tables for let's say a firewall or dpi
application, because that data is only needed by local cpu and can
be gathered after distribution. Simply put, the lookup for the flow, if
it is needed, does not belong to distribution. It can be outsourced to
the destination cpu or just simply be ignored, if the application
doesn't care.

> Tom replied to you that a hash derived from (addr1 ^ addr2) would not
> work in situations where all flows goes from machine A to machine B
> (all hashes would be the same)
> 

Yes, I see the point. And all I'm just asking if it's wise to optimize
for this particular scenario.

If you spin this idea further beyond flow tracking, maybe an application
also needs to do some kind of user tracking by ip. Wouldn't it make
sense to have user based flows on a more local basis, not a global one
because ports will get in the way?

> Current hash is probably more than enough to cover all situations.
> 

I agree with this, but would like to point out the phrasing "probably
more than enough". :)


Franco

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-21 11:06                                   ` Franco Fichtner
@ 2010-04-21 11:16                                     ` Eric Dumazet
  0 siblings, 0 replies; 60+ messages in thread
From: Eric Dumazet @ 2010-04-21 11:16 UTC (permalink / raw)
  To: Franco Fichtner; +Cc: Tom Herbert, Changli Gao, David Miller, netdev

Le mercredi 21 avril 2010 à 13:06 +0200, Franco Fichtner a écrit :
> > 
> 
> Hashing for cpu distribution should be as minimal as it could possibly
> be with the least number operations needed to compute a hash, which 
> normally involves touching one cold cache line (ip header). If you add 
> the ports to your mix you have the luxury of solving static ip mappings,
> but only for protocols that support it. Usage of the destination port
> may also prove to be more or less pointless with a lot of http traffic,
> because it's most likely static. And you add another potential cold
> cache line access. For a lot of traffic scenarios, we'll have a bunch of
> internal ips and the internet on the other side, so having a simple hash
> based on a flavor if internal/external ip is more than enough to work
> with for distribution. If the network card can provide a complete hash
> all the better. Then this part of my point is void.
> 

But we already have to bring into our cpu cache one cache line, needed
in eth_type_trans() : (12+2 bytes of ethernet header)

TCP/UDP tuples are included into this cache line (64 bytes on current
popular arches)

Cost of rxhash is absolute noise into the picture.
A device provided hash, to be effective, would also make
eth_type_trans() call not done.




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-20 21:41                                   ` David Miller
  2010-04-20 23:35                                     ` Changli Gao
@ 2010-04-21 19:12                                     ` Tom Herbert
  2010-04-23 20:44                                       ` David Miller
  2010-05-06  8:06                                       ` David Miller
  1 sibling, 2 replies; 60+ messages in thread
From: Tom Herbert @ 2010-04-21 19:12 UTC (permalink / raw)
  To: David Miller; +Cc: eric.dumazet, franco, xiaosuo, netdev

On Tue, Apr 20, 2010 at 2:41 PM, David Miller <davem@davemloft.net> wrote:
> From: Eric Dumazet <eric.dumazet@gmail.com>
> Date: Tue, 20 Apr 2010 16:57:01 +0200
>
>> I know many applications using TCP on loopback, they are real :)
>
> This is all true and I support your hashing patch and all of that.
>
> But if we really want TCP over loopback to go fast, there are much
> better ways to do this.
>
> Eric, do you remember that "TCP friends" rough patch I sent you last
> year that essentailly made TCP sockets over loopback behave like
> AF_UNIX ones and just queue the SKBs directly to the destination
> socket without doing any protocol work?
>

This is sounds very interesting!  Could you post a patch? :-)

> If we ever got that working, tbench performance would become
> impressive :)
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-21 19:12                                     ` Tom Herbert
@ 2010-04-23 20:44                                       ` David Miller
  2010-05-06  8:06                                       ` David Miller
  1 sibling, 0 replies; 60+ messages in thread
From: David Miller @ 2010-04-23 20:44 UTC (permalink / raw)
  To: therbert; +Cc: eric.dumazet, franco, xiaosuo, netdev

From: Tom Herbert <therbert@google.com>
Date: Wed, 21 Apr 2010 12:12:41 -0700

> On Tue, Apr 20, 2010 at 2:41 PM, David Miller <davem@davemloft.net> wrote:
>> Eric, do you remember that "TCP friends" rough patch I sent you last
>> year that essentailly made TCP sockets over loopback behave like
>> AF_UNIX ones and just queue the SKBs directly to the destination
>> socket without doing any protocol work?
>>
> 
> This is sounds very interesting!  Could you post a patch? :-)

I'll see if I can find it, I sent it to Eric more than a year
ago...

The basic scheme was pretty simple:

1) Add "struct sock *friend" to struct sk_buff

2) TCP initial handshake SYN and SYN+ACK transmits set "skb->friend =
   sk" and TCP receive path notices this and stores this 'friend'
   socket pointer locally in the newly created connection socket.

   The purpose of skb->friend is to let the receiving socket on
   loopback see that the other end is on the local system and
   can be directly communicated to.

3) TCP sendmsg queues data directly to sk->friend's receive queue
   instead sending TCP protocol packets.

The only complications come from making sendmsg and recvmsg not
try to do all of the sequence handling and checking, stuff like
that.  Also, URG would need to be dealt with somehow too.

I'm sure someone suitably motivated could get a working patch
going in no time :-)

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-19 21:19                     ` David Miller
@ 2010-04-26  8:41                       ` Eric Dumazet
  2010-04-27 21:59                         ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-26  8:41 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le lundi 19 avril 2010 à 14:19 -0700, David Miller a écrit :

> 
> I was thinking also about how we could compute rxhash in the
> loopback driver :-)

This would be easy if rxhash was not a "struct inet_sock" field but a
"struct sock" one

sock_alloc_send_pskb() (or skb_set_owner_w())

skb->rxhash = sk->rxhash;




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-26  8:41                       ` Eric Dumazet
@ 2010-04-27 21:59                         ` David Miller
  2010-04-27 22:08                           ` Eric Dumazet
  0 siblings, 1 reply; 60+ messages in thread
From: David Miller @ 2010-04-27 21:59 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Mon, 26 Apr 2010 10:41:11 +0200

> Le lundi 19 avril 2010 à 14:19 -0700, David Miller a écrit :
> 
>> 
>> I was thinking also about how we could compute rxhash in the
>> loopback driver :-)
> 
> This would be easy if rxhash was not a "struct inet_sock" field but a
> "struct sock" one
> 
> sock_alloc_send_pskb() (or skb_set_owner_w())
> 
> skb->rxhash = sk->rxhash;

Agreed.  I'll commit the following to net-next-2.6 after some build
testing.

net: Make RFS socket operations not be inet specific.

Idea from Eric Dumazet.

As for placement inside of struct sock, I tried to choose a place
that otherwise has a 32-bit hole on 64-bit systems.

Signed-off-by: David S. Miller <davem@davemloft.net>

diff --git a/include/net/inet_sock.h b/include/net/inet_sock.h
index c1d4295..1653de5 100644
--- a/include/net/inet_sock.h
+++ b/include/net/inet_sock.h
@@ -102,7 +102,6 @@ struct rtable;
  * @uc_ttl - Unicast TTL
  * @inet_sport - Source port
  * @inet_id - ID counter for DF pkts
- * @rxhash - flow hash received from netif layer
  * @tos - TOS
  * @mc_ttl - Multicasting TTL
  * @is_icsk - is this an inet_connection_sock?
@@ -126,9 +125,6 @@ struct inet_sock {
 	__u16			cmsg_flags;
 	__be16			inet_sport;
 	__u16			inet_id;
-#ifdef CONFIG_RPS
-	__u32			rxhash;
-#endif
 
 	struct ip_options	*opt;
 	__u8			tos;
@@ -224,37 +220,4 @@ static inline __u8 inet_sk_flowi_flags(const struct sock *sk)
 	return inet_sk(sk)->transparent ? FLOWI_FLAG_ANYSRC : 0;
 }
 
-static inline void inet_rps_record_flow(const struct sock *sk)
-{
-#ifdef CONFIG_RPS
-	struct rps_sock_flow_table *sock_flow_table;
-
-	rcu_read_lock();
-	sock_flow_table = rcu_dereference(rps_sock_flow_table);
-	rps_record_sock_flow(sock_flow_table, inet_sk(sk)->rxhash);
-	rcu_read_unlock();
-#endif
-}
-
-static inline void inet_rps_reset_flow(const struct sock *sk)
-{
-#ifdef CONFIG_RPS
-	struct rps_sock_flow_table *sock_flow_table;
-
-	rcu_read_lock();
-	sock_flow_table = rcu_dereference(rps_sock_flow_table);
-	rps_reset_sock_flow(sock_flow_table, inet_sk(sk)->rxhash);
-	rcu_read_unlock();
-#endif
-}
-
-static inline void inet_rps_save_rxhash(struct sock *sk, u32 rxhash)
-{
-#ifdef CONFIG_RPS
-	if (unlikely(inet_sk(sk)->rxhash != rxhash)) {
-		inet_rps_reset_flow(sk);
-		inet_sk(sk)->rxhash = rxhash;
-	}
-#endif
-}
 #endif	/* _INET_SOCK_H */
diff --git a/include/net/sock.h b/include/net/sock.h
index ef2f875..cf12b1e 100644
--- a/include/net/sock.h
+++ b/include/net/sock.h
@@ -198,6 +198,7 @@ struct sock_common {
   *	@sk_rcvlowat: %SO_RCVLOWAT setting
   *	@sk_rcvtimeo: %SO_RCVTIMEO setting
   *	@sk_sndtimeo: %SO_SNDTIMEO setting
+  *	@sk_rxhash: flow hash received from netif layer
   *	@sk_filter: socket filtering instructions
   *	@sk_protinfo: private area, net family specific, when not using slab
   *	@sk_timer: sock cleanup timer
@@ -278,6 +279,9 @@ struct sock {
 	int			sk_gso_type;
 	unsigned int		sk_gso_max_size;
 	int			sk_rcvlowat;
+#ifdef CONFIG_RPS
+	__u32			sk_rxhash;
+#endif
 	unsigned long 		sk_flags;
 	unsigned long	        sk_lingertime;
 	struct sk_buff_head	sk_error_queue;
@@ -629,6 +633,40 @@ static inline int sk_backlog_rcv(struct sock *sk, struct sk_buff *skb)
 	return sk->sk_backlog_rcv(sk, skb);
 }
 
+static inline void sock_rps_record_flow(const struct sock *sk)
+{
+#ifdef CONFIG_RPS
+	struct rps_sock_flow_table *sock_flow_table;
+
+	rcu_read_lock();
+	sock_flow_table = rcu_dereference(rps_sock_flow_table);
+	rps_record_sock_flow(sock_flow_table, sk->sk_rxhash);
+	rcu_read_unlock();
+#endif
+}
+
+static inline void sock_rps_reset_flow(const struct sock *sk)
+{
+#ifdef CONFIG_RPS
+	struct rps_sock_flow_table *sock_flow_table;
+
+	rcu_read_lock();
+	sock_flow_table = rcu_dereference(rps_sock_flow_table);
+	rps_reset_sock_flow(sock_flow_table, sk->sk_rxhash);
+	rcu_read_unlock();
+#endif
+}
+
+static inline void sock_rps_save_rxhash(struct sock *sk, u32 rxhash)
+{
+#ifdef CONFIG_RPS
+	if (unlikely(sk->sk_rxhash != rxhash)) {
+		sock_rps_reset_flow(sk);
+		sk->sk_rxhash = rxhash;
+	}
+#endif
+}
+
 #define sk_wait_event(__sk, __timeo, __condition)			\
 	({	int __rc;						\
 		release_sock(__sk);					\
diff --git a/net/ipv4/af_inet.c b/net/ipv4/af_inet.c
index 9f52880..c6c43bc 100644
--- a/net/ipv4/af_inet.c
+++ b/net/ipv4/af_inet.c
@@ -419,7 +419,7 @@ int inet_release(struct socket *sock)
 	if (sk) {
 		long timeout;
 
-		inet_rps_reset_flow(sk);
+		sock_rps_reset_flow(sk);
 
 		/* Applications forget to leave groups before exiting */
 		ip_mc_drop_socket(sk);
@@ -722,7 +722,7 @@ int inet_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
 {
 	struct sock *sk = sock->sk;
 
-	inet_rps_record_flow(sk);
+	sock_rps_record_flow(sk);
 
 	/* We may need to bind the socket. */
 	if (!inet_sk(sk)->inet_num && inet_autobind(sk))
@@ -737,7 +737,7 @@ static ssize_t inet_sendpage(struct socket *sock, struct page *page, int offset,
 {
 	struct sock *sk = sock->sk;
 
-	inet_rps_record_flow(sk);
+	sock_rps_record_flow(sk);
 
 	/* We may need to bind the socket. */
 	if (!inet_sk(sk)->inet_num && inet_autobind(sk))
@@ -755,7 +755,7 @@ int inet_recvmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
 	int addr_len = 0;
 	int err;
 
-	inet_rps_record_flow(sk);
+	sock_rps_record_flow(sk);
 
 	err = sk->sk_prot->recvmsg(iocb, sk, msg, size, flags & MSG_DONTWAIT,
 				   flags & ~MSG_DONTWAIT, &addr_len);
diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
index 4d6717d..771f814 100644
--- a/net/ipv4/tcp_ipv4.c
+++ b/net/ipv4/tcp_ipv4.c
@@ -1672,7 +1672,7 @@ process:
 
 	skb->dev = NULL;
 
-	inet_rps_save_rxhash(sk, skb->rxhash);
+	sock_rps_save_rxhash(sk, skb->rxhash);
 
 	bh_lock_sock_nested(sk);
 	ret = 0;
diff --git a/net/ipv4/udp.c b/net/ipv4/udp.c
index 776c844..63eb56b 100644
--- a/net/ipv4/udp.c
+++ b/net/ipv4/udp.c
@@ -1217,7 +1217,7 @@ int udp_disconnect(struct sock *sk, int flags)
 	sk->sk_state = TCP_CLOSE;
 	inet->inet_daddr = 0;
 	inet->inet_dport = 0;
-	inet_rps_save_rxhash(sk, 0);
+	sock_rps_save_rxhash(sk, 0);
 	sk->sk_bound_dev_if = 0;
 	if (!(sk->sk_userlocks & SOCK_BINDADDR_LOCK))
 		inet_reset_saddr(sk);
@@ -1262,7 +1262,7 @@ static int __udp_queue_rcv_skb(struct sock *sk, struct sk_buff *skb)
 	int rc;
 
 	if (inet_sk(sk)->inet_daddr)
-		inet_rps_save_rxhash(sk, skb->rxhash);
+		sock_rps_save_rxhash(sk, skb->rxhash);
 
 	rc = sock_queue_rcv_skb(sk, skb);
 	if (rc < 0) {

^ permalink raw reply related	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-27 21:59                         ` David Miller
@ 2010-04-27 22:08                           ` Eric Dumazet
  2010-04-27 22:10                             ` David Miller
  0 siblings, 1 reply; 60+ messages in thread
From: Eric Dumazet @ 2010-04-27 22:08 UTC (permalink / raw)
  To: David Miller; +Cc: therbert, netdev

Le mardi 27 avril 2010 à 14:59 -0700, David Miller a écrit :
> From: Eric Dumazet <eric.dumazet@gmail.com>
> Date: Mon, 26 Apr 2010 10:41:11 +0200
> 
> > Le lundi 19 avril 2010 à 14:19 -0700, David Miller a écrit :
> > 
> >> 
> >> I was thinking also about how we could compute rxhash in the
> >> loopback driver :-)
> > 
> > This would be easy if rxhash was not a "struct inet_sock" field but a
> > "struct sock" one
> > 
> > sock_alloc_send_pskb() (or skb_set_owner_w())
> > 
> > skb->rxhash = sk->rxhash;
> 
> Agreed.  I'll commit the following to net-next-2.6 after some build
> testing.
> 
> net: Make RFS socket operations not be inet specific.
> 
> Idea from Eric Dumazet.
> 
> As for placement inside of struct sock, I tried to choose a place
> that otherwise has a 32-bit hole on 64-bit systems.
> 
> Signed-off-by: David S. Miller <davem@davemloft.net>

Acked-by: Eric Dumazet <eric.dumazet@gmail.com>

I tested same patch today (plus the skb->rxhash = sk->sk_rxhash) and got
a very small speedup on my Nehalem machine, where get_rps_cpus() was
using 1 % of cpu, now 0.25 %, on a tbench.




^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH v5] rfs: Receive Flow Steering
  2010-04-27 22:08                           ` Eric Dumazet
@ 2010-04-27 22:10                             ` David Miller
  0 siblings, 0 replies; 60+ messages in thread
From: David Miller @ 2010-04-27 22:10 UTC (permalink / raw)
  To: eric.dumazet; +Cc: therbert, netdev

From: Eric Dumazet <eric.dumazet@gmail.com>
Date: Wed, 28 Apr 2010 00:08:46 +0200

> Le mardi 27 avril 2010 à 14:59 -0700, David Miller a écrit :
>> net: Make RFS socket operations not be inet specific.
>> 
>> Idea from Eric Dumazet.
>> 
>> As for placement inside of struct sock, I tried to choose a place
>> that otherwise has a 32-bit hole on 64-bit systems.
>> 
>> Signed-off-by: David S. Miller <davem@davemloft.net>
> 
> Acked-by: Eric Dumazet <eric.dumazet@gmail.com>
> 
> I tested same patch today (plus the skb->rxhash = sk->sk_rxhash) and got
> a very small speedup on my Nehalem machine, where get_rps_cpus() was
> using 1 % of cpu, now 0.25 %, on a tbench.

Great, I've added your ACK.

Thanks!

^ permalink raw reply	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-04-21 19:12                                     ` Tom Herbert
  2010-04-23 20:44                                       ` David Miller
@ 2010-05-06  8:06                                       ` David Miller
  2010-05-06 14:45                                         ` Tom Herbert
  1 sibling, 1 reply; 60+ messages in thread
From: David Miller @ 2010-05-06  8:06 UTC (permalink / raw)
  To: therbert; +Cc: eric.dumazet, franco, xiaosuo, netdev

From: Tom Herbert <therbert@google.com>
Date: Wed, 21 Apr 2010 12:12:41 -0700

> On Tue, Apr 20, 2010 at 2:41 PM, David Miller <davem@davemloft.net> wrote:
>> From: Eric Dumazet <eric.dumazet@gmail.com>
>> Date: Tue, 20 Apr 2010 16:57:01 +0200
>>
>>> I know many applications using TCP on loopback, they are real :)
>>
>> This is all true and I support your hashing patch and all of that.
>>
>> But if we really want TCP over loopback to go fast, there are much
>> better ways to do this.
>>
>> Eric, do you remember that "TCP friends" rough patch I sent you last
>> year that essentailly made TCP sockets over loopback behave like
>> AF_UNIX ones and just queue the SKBs directly to the destination
>> socket without doing any protocol work?
>>
> This is sounds very interesting!  Could you post a patch? :-)

I was finally able to unearth a copy, it's completely raw, it's at least
a year old, and it's not fully implemented at all.

But you asked for it :-)

diff --git a/include/linux/skbuff.h b/include/linux/skbuff.h
index 299ec4b..7f855d3 100644
--- a/include/linux/skbuff.h
+++ b/include/linux/skbuff.h
@@ -206,6 +206,7 @@ typedef unsigned char *sk_buff_data_t;
  *	@mac_header: Link layer header
  *	@dst: destination entry
  *	@sp: the security path, used for xfrm
+ *	@friend: loopback friend socket
  *	@cb: Control buffer. Free for use by every layer. Put private vars here
  *	@len: Length of actual data
  *	@data_len: Data length
@@ -262,6 +263,7 @@ struct sk_buff {
 		struct  rtable		*rtable;
 	};
 	struct	sec_path	*sp;
+	struct sock		*friend;
 
 	/*
 	 * This is the control buffer. It is free to use for every
diff --git a/include/net/request_sock.h b/include/net/request_sock.h
index b220b5f..52b2f7a 100644
--- a/include/net/request_sock.h
+++ b/include/net/request_sock.h
@@ -53,6 +53,7 @@ struct request_sock {
 	unsigned long			expires;
 	const struct request_sock_ops	*rsk_ops;
 	struct sock			*sk;
+	struct sock			*friend;
 	u32				secid;
 	u32				peer_secid;
 };
diff --git a/include/net/sock.h b/include/net/sock.h
index dc42b44..3e86190 100644
--- a/include/net/sock.h
+++ b/include/net/sock.h
@@ -137,6 +137,7 @@ struct sock_common {
   *	@sk_userlocks: %SO_SNDBUF and %SO_RCVBUF settings
   *	@sk_lock:	synchronizer
   *	@sk_rcvbuf: size of receive buffer in bytes
+  *	@sk_friend: loopback friend socket
   *	@sk_sleep: sock wait queue
   *	@sk_dst_cache: destination cache
   *	@sk_dst_lock: destination cache lock
@@ -227,6 +228,7 @@ struct sock {
 		struct sk_buff *head;
 		struct sk_buff *tail;
 	} sk_backlog;
+	struct sock		*sk_friend;
 	wait_queue_head_t	*sk_sleep;
 	struct dst_entry	*sk_dst_cache;
 	struct xfrm_policy	*sk_policy[2];
diff --git a/net/core/skbuff.c b/net/core/skbuff.c
index 4fe605f..0eef90a 100644
--- a/net/core/skbuff.c
+++ b/net/core/skbuff.c
@@ -435,6 +435,7 @@ static void __copy_skb_header(struct sk_buff *new, const struct sk_buff *old)
 #ifdef CONFIG_INET
 	new->sp			= secpath_get(old->sp);
 #endif
+	new->friend		= old->friend;
 	memcpy(new->cb, old->cb, sizeof(old->cb));
 	new->csum_start		= old->csum_start;
 	new->csum_offset	= old->csum_offset;
diff --git a/net/ipv4/inet_connection_sock.c b/net/ipv4/inet_connection_sock.c
index 828ea21..375dc2e 100644
--- a/net/ipv4/inet_connection_sock.c
+++ b/net/ipv4/inet_connection_sock.c
@@ -503,6 +503,8 @@ struct sock *inet_csk_clone(struct sock *sk, const struct request_sock *req,
 	if (newsk != NULL) {
 		struct inet_connection_sock *newicsk = inet_csk(newsk);
 
+		newsk->sk_friend = req->friend;
+
 		newsk->sk_state = TCP_SYN_RECV;
 		newicsk->icsk_bind_hash = NULL;
 
diff --git a/net/ipv4/tcp.c b/net/ipv4/tcp.c
index 58ac838..042ee1d 100644
--- a/net/ipv4/tcp.c
+++ b/net/ipv4/tcp.c
@@ -474,7 +474,8 @@ static inline int forced_push(struct tcp_sock *tp)
 	return after(tp->write_seq, tp->pushed_seq + (tp->max_window >> 1));
 }
 
-static inline void skb_entail(struct sock *sk, struct sk_buff *skb)
+static inline void skb_entail(struct sock *sk, struct sk_buff *skb,
+			      struct sk_buff_head *friend_queue)
 {
 	struct tcp_sock *tp = tcp_sk(sk);
 	struct tcp_skb_cb *tcb = TCP_SKB_CB(skb);
@@ -484,7 +485,10 @@ static inline void skb_entail(struct sock *sk, struct sk_buff *skb)
 	tcb->flags   = TCPCB_FLAG_ACK;
 	tcb->sacked  = 0;
 	skb_header_release(skb);
-	tcp_add_write_queue_tail(sk, skb);
+	if (sk->sk_friend)
+		__skb_queue_tail(friend_queue, skb);
+	else
+		tcp_add_write_queue_tail(sk, skb);
 	sk->sk_wmem_queued += skb->truesize;
 	sk_mem_charge(sk, skb->truesize);
 	if (tp->nonagle & TCP_NAGLE_PUSH)
@@ -501,7 +505,7 @@ static inline void tcp_mark_urg(struct tcp_sock *tp, int flags,
 }
 
 static inline void tcp_push(struct sock *sk, int flags, int mss_now,
-			    int nonagle)
+			    int nonagle, struct sk_buff_head *friend_queue)
 {
 	struct tcp_sock *tp = tcp_sk(sk);
 
@@ -512,6 +516,19 @@ static inline void tcp_push(struct sock *sk, int flags, int mss_now,
 		tcp_mark_urg(tp, flags, skb);
 		__tcp_push_pending_frames(sk, mss_now,
 					  (flags & MSG_MORE) ? TCP_NAGLE_CORK : nonagle);
+	} else if (sk->sk_friend) {
+		struct sock *friend = sk->sk_friend;
+		struct sk_buff *skb;
+		unsigned int len;
+
+		spin_lock_bh(&friend->sk_lock.slock);
+		len = 0;
+		while ((skb = __skb_dequeue(friend_queue)) != NULL) {
+			len += skb->len;
+			__skb_queue_tail(&sk->sk_receive_queue, skb);
+		}
+		sk->sk_data_ready(friend, len);
+		spin_unlock_bh(&friend->sk_lock.slock);
 	}
 }
 
@@ -658,6 +675,7 @@ static ssize_t do_tcp_sendpages(struct sock *sk, struct page **pages, int poffse
 			 size_t psize, int flags)
 {
 	struct tcp_sock *tp = tcp_sk(sk);
+	struct sk_buff_head friend_queue;
 	int mss_now, size_goal;
 	int err;
 	ssize_t copied;
@@ -674,6 +692,8 @@ static ssize_t do_tcp_sendpages(struct sock *sk, struct page **pages, int poffse
 	size_goal = tp->xmit_size_goal;
 	copied = 0;
 
+	skb_queue_head_init(&friend_queue);
+
 	err = -EPIPE;
 	if (sk->sk_err || (sk->sk_shutdown & SEND_SHUTDOWN))
 		goto do_error;
@@ -694,7 +714,7 @@ new_segment:
 			if (!skb)
 				goto wait_for_memory;
 
-			skb_entail(sk, skb);
+			skb_entail(sk, skb, &friend_queue);
 			copy = size_goal;
 		}
 
@@ -749,7 +769,8 @@ wait_for_sndbuf:
 		set_bit(SOCK_NOSPACE, &sk->sk_socket->flags);
 wait_for_memory:
 		if (copied)
-			tcp_push(sk, flags & ~MSG_MORE, mss_now, TCP_NAGLE_PUSH);
+			tcp_push(sk, flags & ~MSG_MORE, mss_now, TCP_NAGLE_PUSH,
+				 &friend_queue);
 
 		if ((err = sk_stream_wait_memory(sk, &timeo)) != 0)
 			goto do_error;
@@ -760,7 +781,7 @@ wait_for_memory:
 
 out:
 	if (copied)
-		tcp_push(sk, flags, mss_now, tp->nonagle);
+		tcp_push(sk, flags, mss_now, tp->nonagle, &friend_queue);
 	return copied;
 
 do_error:
@@ -817,6 +838,7 @@ int tcp_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
 	struct sock *sk = sock->sk;
 	struct iovec *iov;
 	struct tcp_sock *tp = tcp_sk(sk);
+	struct sk_buff_head friend_queue;
 	struct sk_buff *skb;
 	int iovlen, flags;
 	int mss_now, size_goal;
@@ -849,6 +871,7 @@ int tcp_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
 	if (sk->sk_err || (sk->sk_shutdown & SEND_SHUTDOWN))
 		goto do_error;
 
+	skb_queue_head_init(&friend_queue);
 	while (--iovlen >= 0) {
 		int seglen = iov->iov_len;
 		unsigned char __user *from = iov->iov_base;
@@ -881,7 +904,7 @@ new_segment:
 				if (sk->sk_route_caps & NETIF_F_ALL_CSUM)
 					skb->ip_summed = CHECKSUM_PARTIAL;
 
-				skb_entail(sk, skb);
+				skb_entail(sk, skb, &friend_queue);
 				copy = size_goal;
 			}
 
@@ -995,7 +1018,8 @@ wait_for_sndbuf:
 			set_bit(SOCK_NOSPACE, &sk->sk_socket->flags);
 wait_for_memory:
 			if (copied)
-				tcp_push(sk, flags & ~MSG_MORE, mss_now, TCP_NAGLE_PUSH);
+				tcp_push(sk, flags & ~MSG_MORE, mss_now,
+					 TCP_NAGLE_PUSH, &friend_queue);
 
 			if ((err = sk_stream_wait_memory(sk, &timeo)) != 0)
 				goto do_error;
@@ -1007,7 +1031,7 @@ wait_for_memory:
 
 out:
 	if (copied)
-		tcp_push(sk, flags, mss_now, tp->nonagle);
+		tcp_push(sk, flags, mss_now, tp->nonagle, &friend_queue);
 	TCP_CHECK_TIMER(sk);
 	release_sock(sk);
 	return copied;
diff --git a/net/ipv4/tcp_input.c b/net/ipv4/tcp_input.c
index cdc051b..eb6f914 100644
--- a/net/ipv4/tcp_input.c
+++ b/net/ipv4/tcp_input.c
@@ -4998,6 +4998,7 @@ static int tcp_rcv_synsent_state_process(struct sock *sk, struct sk_buff *skb,
 		 *    state to ESTABLISHED..."
 		 */
 
+		sk->sk_friend = skb->friend;
 		TCP_ECN_rcv_synack(tp, th);
 
 		tp->snd_wl1 = TCP_SKB_CB(skb)->seq;
diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
index 7766151..4d91ff4 100644
--- a/net/ipv4/tcp_ipv4.c
+++ b/net/ipv4/tcp_ipv4.c
@@ -1289,6 +1289,7 @@ int tcp_v4_conn_request(struct sock *sk, struct sk_buff *skb)
 	if (!req)
 		goto drop;
 
+	req->friend = skb->friend;
 #ifdef CONFIG_TCP_MD5SIG
 	tcp_rsk(req)->af_specific = &tcp_request_sock_ipv4_ops;
 #endif
diff --git a/net/ipv4/tcp_output.c b/net/ipv4/tcp_output.c
index debf235..a4d4c14 100644
--- a/net/ipv4/tcp_output.c
+++ b/net/ipv4/tcp_output.c
@@ -577,6 +577,7 @@ static int tcp_transmit_skb(struct sock *sk, struct sk_buff *skb, int clone_it,
 	}
 
 	if (unlikely(tcb->flags & TCPCB_FLAG_SYN)) {
+		skb->friend = sk;
 		tcp_syn_build_options((__be32 *)(th + 1),
 				      tcp_advertise_mss(sk),
 				      (sysctl_flags & SYSCTL_FLAG_TSTAMPS),
@@ -1006,6 +1007,8 @@ unsigned int tcp_current_mss(struct sock *sk, int large_allowed)
 		xmit_size_goal = tcp_bound_to_half_wnd(tp, xmit_size_goal);
 		xmit_size_goal -= (xmit_size_goal % mss_now);
 	}
+	if (sk->sk_friend)
+		xmit_size_goal = ~(u16)0;
 	tp->xmit_size_goal = xmit_size_goal;
 
 	return mss_now;
diff --git a/net/ipv6/tcp_ipv6.c b/net/ipv6/tcp_ipv6.c
index 715965f..c79d3ea 100644
--- a/net/ipv6/tcp_ipv6.c
+++ b/net/ipv6/tcp_ipv6.c
@@ -1280,6 +1280,7 @@ static int tcp_v6_conn_request(struct sock *sk, struct sk_buff *skb)
 	if (req == NULL)
 		goto drop;
 
+	req->friend = skb->friend;
 #ifdef CONFIG_TCP_MD5SIG
 	tcp_rsk(req)->af_specific = &tcp_request_sock_ipv6_ops;
 #endif

^ permalink raw reply related	[flat|nested] 60+ messages in thread

* Re: [PATCH net-next-2.6] rps: consistent rxhash
  2010-05-06  8:06                                       ` David Miller
@ 2010-05-06 14:45                                         ` Tom Herbert
  0 siblings, 0 replies; 60+ messages in thread
From: Tom Herbert @ 2010-05-06 14:45 UTC (permalink / raw)
  To: David Miller; +Cc: eric.dumazet, franco, xiaosuo, netdev

On Thu, May 6, 2010 at 1:06 AM, David Miller <davem@davemloft.net> wrote:
> From: Tom Herbert <therbert@google.com>
> Date: Wed, 21 Apr 2010 12:12:41 -0700
>
>> On Tue, Apr 20, 2010 at 2:41 PM, David Miller <davem@davemloft.net> wrote:
>>> From: Eric Dumazet <eric.dumazet@gmail.com>
>>> Date: Tue, 20 Apr 2010 16:57:01 +0200
>>>
>>>> I know many applications using TCP on loopback, they are real :)
>>>
>>> This is all true and I support your hashing patch and all of that.
>>>
>>> But if we really want TCP over loopback to go fast, there are much
>>> better ways to do this.
>>>
>>> Eric, do you remember that "TCP friends" rough patch I sent you last
>>> year that essentailly made TCP sockets over loopback behave like
>>> AF_UNIX ones and just queue the SKBs directly to the destination
>>> socket without doing any protocol work?
>>>
>> This is sounds very interesting!  Could you post a patch? :-)
>
> I was finally able to unearth a copy, it's completely raw, it's at least
> a year old, and it's not fully implemented at all.
>
> But you asked for it :-)
>
Thanks!  We'll take a look... I've always thought sockets should have
friends :-)

> diff --git a/include/linux/skbuff.h b/include/linux/skbuff.h
> index 299ec4b..7f855d3 100644
> --- a/include/linux/skbuff.h
> +++ b/include/linux/skbuff.h
> @@ -206,6 +206,7 @@ typedef unsigned char *sk_buff_data_t;
>  *     @mac_header: Link layer header
>  *     @dst: destination entry
>  *     @sp: the security path, used for xfrm
> + *     @friend: loopback friend socket
>  *     @cb: Control buffer. Free for use by every layer. Put private vars here
>  *     @len: Length of actual data
>  *     @data_len: Data length
> @@ -262,6 +263,7 @@ struct sk_buff {
>                struct  rtable          *rtable;
>        };
>        struct  sec_path        *sp;
> +       struct sock             *friend;
>
>        /*
>         * This is the control buffer. It is free to use for every
> diff --git a/include/net/request_sock.h b/include/net/request_sock.h
> index b220b5f..52b2f7a 100644
> --- a/include/net/request_sock.h
> +++ b/include/net/request_sock.h
> @@ -53,6 +53,7 @@ struct request_sock {
>        unsigned long                   expires;
>        const struct request_sock_ops   *rsk_ops;
>        struct sock                     *sk;
> +       struct sock                     *friend;
>        u32                             secid;
>        u32                             peer_secid;
>  };
> diff --git a/include/net/sock.h b/include/net/sock.h
> index dc42b44..3e86190 100644
> --- a/include/net/sock.h
> +++ b/include/net/sock.h
> @@ -137,6 +137,7 @@ struct sock_common {
>   *    @sk_userlocks: %SO_SNDBUF and %SO_RCVBUF settings
>   *    @sk_lock:       synchronizer
>   *    @sk_rcvbuf: size of receive buffer in bytes
> +  *    @sk_friend: loopback friend socket
>   *    @sk_sleep: sock wait queue
>   *    @sk_dst_cache: destination cache
>   *    @sk_dst_lock: destination cache lock
> @@ -227,6 +228,7 @@ struct sock {
>                struct sk_buff *head;
>                struct sk_buff *tail;
>        } sk_backlog;
> +       struct sock             *sk_friend;
>        wait_queue_head_t       *sk_sleep;
>        struct dst_entry        *sk_dst_cache;
>        struct xfrm_policy      *sk_policy[2];
> diff --git a/net/core/skbuff.c b/net/core/skbuff.c
> index 4fe605f..0eef90a 100644
> --- a/net/core/skbuff.c
> +++ b/net/core/skbuff.c
> @@ -435,6 +435,7 @@ static void __copy_skb_header(struct sk_buff *new, const struct sk_buff *old)
>  #ifdef CONFIG_INET
>        new->sp                 = secpath_get(old->sp);
>  #endif
> +       new->friend             = old->friend;
>        memcpy(new->cb, old->cb, sizeof(old->cb));
>        new->csum_start         = old->csum_start;
>        new->csum_offset        = old->csum_offset;
> diff --git a/net/ipv4/inet_connection_sock.c b/net/ipv4/inet_connection_sock.c
> index 828ea21..375dc2e 100644
> --- a/net/ipv4/inet_connection_sock.c
> +++ b/net/ipv4/inet_connection_sock.c
> @@ -503,6 +503,8 @@ struct sock *inet_csk_clone(struct sock *sk, const struct request_sock *req,
>        if (newsk != NULL) {
>                struct inet_connection_sock *newicsk = inet_csk(newsk);
>
> +               newsk->sk_friend = req->friend;
> +
>                newsk->sk_state = TCP_SYN_RECV;
>                newicsk->icsk_bind_hash = NULL;
>
> diff --git a/net/ipv4/tcp.c b/net/ipv4/tcp.c
> index 58ac838..042ee1d 100644
> --- a/net/ipv4/tcp.c
> +++ b/net/ipv4/tcp.c
> @@ -474,7 +474,8 @@ static inline int forced_push(struct tcp_sock *tp)
>        return after(tp->write_seq, tp->pushed_seq + (tp->max_window >> 1));
>  }
>
> -static inline void skb_entail(struct sock *sk, struct sk_buff *skb)
> +static inline void skb_entail(struct sock *sk, struct sk_buff *skb,
> +                             struct sk_buff_head *friend_queue)
>  {
>        struct tcp_sock *tp = tcp_sk(sk);
>        struct tcp_skb_cb *tcb = TCP_SKB_CB(skb);
> @@ -484,7 +485,10 @@ static inline void skb_entail(struct sock *sk, struct sk_buff *skb)
>        tcb->flags   = TCPCB_FLAG_ACK;
>        tcb->sacked  = 0;
>        skb_header_release(skb);
> -       tcp_add_write_queue_tail(sk, skb);
> +       if (sk->sk_friend)
> +               __skb_queue_tail(friend_queue, skb);
> +       else
> +               tcp_add_write_queue_tail(sk, skb);
>        sk->sk_wmem_queued += skb->truesize;
>        sk_mem_charge(sk, skb->truesize);
>        if (tp->nonagle & TCP_NAGLE_PUSH)
> @@ -501,7 +505,7 @@ static inline void tcp_mark_urg(struct tcp_sock *tp, int flags,
>  }
>
>  static inline void tcp_push(struct sock *sk, int flags, int mss_now,
> -                           int nonagle)
> +                           int nonagle, struct sk_buff_head *friend_queue)
>  {
>        struct tcp_sock *tp = tcp_sk(sk);
>
> @@ -512,6 +516,19 @@ static inline void tcp_push(struct sock *sk, int flags, int mss_now,
>                tcp_mark_urg(tp, flags, skb);
>                __tcp_push_pending_frames(sk, mss_now,
>                                          (flags & MSG_MORE) ? TCP_NAGLE_CORK : nonagle);
> +       } else if (sk->sk_friend) {
> +               struct sock *friend = sk->sk_friend;
> +               struct sk_buff *skb;
> +               unsigned int len;
> +
> +               spin_lock_bh(&friend->sk_lock.slock);
> +               len = 0;
> +               while ((skb = __skb_dequeue(friend_queue)) != NULL) {
> +                       len += skb->len;
> +                       __skb_queue_tail(&sk->sk_receive_queue, skb);
> +               }
> +               sk->sk_data_ready(friend, len);
> +               spin_unlock_bh(&friend->sk_lock.slock);
>        }
>  }
>
> @@ -658,6 +675,7 @@ static ssize_t do_tcp_sendpages(struct sock *sk, struct page **pages, int poffse
>                         size_t psize, int flags)
>  {
>        struct tcp_sock *tp = tcp_sk(sk);
> +       struct sk_buff_head friend_queue;
>        int mss_now, size_goal;
>        int err;
>        ssize_t copied;
> @@ -674,6 +692,8 @@ static ssize_t do_tcp_sendpages(struct sock *sk, struct page **pages, int poffse
>        size_goal = tp->xmit_size_goal;
>        copied = 0;
>
> +       skb_queue_head_init(&friend_queue);
> +
>        err = -EPIPE;
>        if (sk->sk_err || (sk->sk_shutdown & SEND_SHUTDOWN))
>                goto do_error;
> @@ -694,7 +714,7 @@ new_segment:
>                        if (!skb)
>                                goto wait_for_memory;
>
> -                       skb_entail(sk, skb);
> +                       skb_entail(sk, skb, &friend_queue);
>                        copy = size_goal;
>                }
>
> @@ -749,7 +769,8 @@ wait_for_sndbuf:
>                set_bit(SOCK_NOSPACE, &sk->sk_socket->flags);
>  wait_for_memory:
>                if (copied)
> -                       tcp_push(sk, flags & ~MSG_MORE, mss_now, TCP_NAGLE_PUSH);
> +                       tcp_push(sk, flags & ~MSG_MORE, mss_now, TCP_NAGLE_PUSH,
> +                                &friend_queue);
>
>                if ((err = sk_stream_wait_memory(sk, &timeo)) != 0)
>                        goto do_error;
> @@ -760,7 +781,7 @@ wait_for_memory:
>
>  out:
>        if (copied)
> -               tcp_push(sk, flags, mss_now, tp->nonagle);
> +               tcp_push(sk, flags, mss_now, tp->nonagle, &friend_queue);
>        return copied;
>
>  do_error:
> @@ -817,6 +838,7 @@ int tcp_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
>        struct sock *sk = sock->sk;
>        struct iovec *iov;
>        struct tcp_sock *tp = tcp_sk(sk);
> +       struct sk_buff_head friend_queue;
>        struct sk_buff *skb;
>        int iovlen, flags;
>        int mss_now, size_goal;
> @@ -849,6 +871,7 @@ int tcp_sendmsg(struct kiocb *iocb, struct socket *sock, struct msghdr *msg,
>        if (sk->sk_err || (sk->sk_shutdown & SEND_SHUTDOWN))
>                goto do_error;
>
> +       skb_queue_head_init(&friend_queue);
>        while (--iovlen >= 0) {
>                int seglen = iov->iov_len;
>                unsigned char __user *from = iov->iov_base;
> @@ -881,7 +904,7 @@ new_segment:
>                                if (sk->sk_route_caps & NETIF_F_ALL_CSUM)
>                                        skb->ip_summed = CHECKSUM_PARTIAL;
>
> -                               skb_entail(sk, skb);
> +                               skb_entail(sk, skb, &friend_queue);
>                                copy = size_goal;
>                        }
>
> @@ -995,7 +1018,8 @@ wait_for_sndbuf:
>                        set_bit(SOCK_NOSPACE, &sk->sk_socket->flags);
>  wait_for_memory:
>                        if (copied)
> -                               tcp_push(sk, flags & ~MSG_MORE, mss_now, TCP_NAGLE_PUSH);
> +                               tcp_push(sk, flags & ~MSG_MORE, mss_now,
> +                                        TCP_NAGLE_PUSH, &friend_queue);
>
>                        if ((err = sk_stream_wait_memory(sk, &timeo)) != 0)
>                                goto do_error;
> @@ -1007,7 +1031,7 @@ wait_for_memory:
>
>  out:
>        if (copied)
> -               tcp_push(sk, flags, mss_now, tp->nonagle);
> +               tcp_push(sk, flags, mss_now, tp->nonagle, &friend_queue);
>        TCP_CHECK_TIMER(sk);
>        release_sock(sk);
>        return copied;
> diff --git a/net/ipv4/tcp_input.c b/net/ipv4/tcp_input.c
> index cdc051b..eb6f914 100644
> --- a/net/ipv4/tcp_input.c
> +++ b/net/ipv4/tcp_input.c
> @@ -4998,6 +4998,7 @@ static int tcp_rcv_synsent_state_process(struct sock *sk, struct sk_buff *skb,
>                 *    state to ESTABLISHED..."
>                 */
>
> +               sk->sk_friend = skb->friend;
>                TCP_ECN_rcv_synack(tp, th);
>
>                tp->snd_wl1 = TCP_SKB_CB(skb)->seq;
> diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
> index 7766151..4d91ff4 100644
> --- a/net/ipv4/tcp_ipv4.c
> +++ b/net/ipv4/tcp_ipv4.c
> @@ -1289,6 +1289,7 @@ int tcp_v4_conn_request(struct sock *sk, struct sk_buff *skb)
>        if (!req)
>                goto drop;
>
> +       req->friend = skb->friend;
>  #ifdef CONFIG_TCP_MD5SIG
>        tcp_rsk(req)->af_specific = &tcp_request_sock_ipv4_ops;
>  #endif
> diff --git a/net/ipv4/tcp_output.c b/net/ipv4/tcp_output.c
> index debf235..a4d4c14 100644
> --- a/net/ipv4/tcp_output.c
> +++ b/net/ipv4/tcp_output.c
> @@ -577,6 +577,7 @@ static int tcp_transmit_skb(struct sock *sk, struct sk_buff *skb, int clone_it,
>        }
>
>        if (unlikely(tcb->flags & TCPCB_FLAG_SYN)) {
> +               skb->friend = sk;
>                tcp_syn_build_options((__be32 *)(th + 1),
>                                      tcp_advertise_mss(sk),
>                                      (sysctl_flags & SYSCTL_FLAG_TSTAMPS),
> @@ -1006,6 +1007,8 @@ unsigned int tcp_current_mss(struct sock *sk, int large_allowed)
>                xmit_size_goal = tcp_bound_to_half_wnd(tp, xmit_size_goal);
>                xmit_size_goal -= (xmit_size_goal % mss_now);
>        }
> +       if (sk->sk_friend)
> +               xmit_size_goal = ~(u16)0;
>        tp->xmit_size_goal = xmit_size_goal;
>
>        return mss_now;
> diff --git a/net/ipv6/tcp_ipv6.c b/net/ipv6/tcp_ipv6.c
> index 715965f..c79d3ea 100644
> --- a/net/ipv6/tcp_ipv6.c
> +++ b/net/ipv6/tcp_ipv6.c
> @@ -1280,6 +1280,7 @@ static int tcp_v6_conn_request(struct sock *sk, struct sk_buff *skb)
>        if (req == NULL)
>                goto drop;
>
> +       req->friend = skb->friend;
>  #ifdef CONFIG_TCP_MD5SIG
>        tcp_rsk(req)->af_specific = &tcp_request_sock_ipv6_ops;
>  #endif
>

^ permalink raw reply	[flat|nested] 60+ messages in thread

end of thread, other threads:[~2010-05-06 14:45 UTC | newest]

Thread overview: 60+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2010-04-16  5:47 [PATCH v5] rfs: Receive Flow Steering Tom Herbert
2010-04-16  6:33 ` David Miller
2010-04-16  6:56   ` Eric Dumazet
2010-04-16  7:18     ` Eric Dumazet
2010-04-16  7:26       ` David Miller
2010-04-16  7:48         ` Eric Dumazet
2010-04-17  7:52           ` [PATCH net-next-2.6] rps: rps_sock_flow_table is mostly read Eric Dumazet
2010-04-17  7:57             ` David Miller
2010-04-16 15:35     ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
2010-04-16 18:15       ` Eric Dumazet
2010-04-16 18:35     ` Tom Herbert
2010-04-16 18:53       ` Eric Dumazet
2010-04-16 20:42         ` Tom Herbert
2010-04-16 21:12           ` Eric Dumazet
2010-04-16 21:25             ` Eric Dumazet
2010-04-17 16:10             ` Eric Dumazet
2010-04-17 17:38               ` Tom Herbert
2010-04-18  0:06                 ` Changli Gao
2010-04-18 11:06                   ` Franco Fichtner
2010-04-19 20:09               ` David Miller
2010-04-19 20:23                 ` David Miller
2010-04-19 20:32                   ` Eric Dumazet
2010-04-19 21:19                     ` David Miller
2010-04-26  8:41                       ` Eric Dumazet
2010-04-27 21:59                         ` David Miller
2010-04-27 22:08                           ` Eric Dumazet
2010-04-27 22:10                             ` David Miller
2010-04-19 23:38                     ` Changli Gao
2010-04-20  5:59                       ` Eric Dumazet
2010-04-20  7:56                         ` [PATCH net-next-2.6] rps: consistent rxhash Eric Dumazet
2010-04-20  8:18                           ` David Miller
2010-04-20 12:48                           ` Franco Fichtner
2010-04-20 13:16                             ` Eric Dumazet
2010-04-20 14:03                               ` Franco Fichtner
2010-04-20 14:57                                 ` Eric Dumazet
2010-04-20 21:41                                   ` David Miller
2010-04-20 23:35                                     ` Changli Gao
2010-04-20 23:38                                       ` David Miller
2010-04-21 19:12                                     ` Tom Herbert
2010-04-23 20:44                                       ` David Miller
2010-05-06  8:06                                       ` David Miller
2010-05-06 14:45                                         ` Tom Herbert
2010-04-20 15:09                             ` Tom Herbert
2010-04-21  9:29                               ` Franco Fichtner
2010-04-21  9:39                                 ` Eric Dumazet
2010-04-21 11:06                                   ` Franco Fichtner
2010-04-21 11:16                                     ` Eric Dumazet
2010-04-20 15:04                         ` [PATCH v5] rfs: Receive Flow Steering Tom Herbert
2010-04-20 15:39                           ` Eric Dumazet
2010-04-16 19:37   ` Eric Dumazet
2010-04-16 22:49     ` David Miller
2010-04-16 22:53       ` David Miller
2010-04-16 22:57         ` David Miller
2010-04-17  0:22           ` Tom Herbert
2010-04-17  0:58             ` David Miller
2010-04-16 11:57 ` Andi Kleen
2010-04-16 13:32   ` jamal
2010-04-16 13:42     ` Andi Kleen
2010-04-16 14:05       ` jamal
2010-04-16 15:28         ` Andi Kleen

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.