All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC PATCH net-next 0/6] implement kthread based napi poll
@ 2020-09-14 17:24 Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support Wei Wang
                   ` (7 more replies)
  0 siblings, 8 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

The idea of moving the napi poll process out of softirq context to a
kernel thread based context is not new.
Paolo Abeni and Hannes Frederic Sowa has proposed patches to move napi
poll to kthread back in 2016. And Felix Fietkau has also proposed
patches of similar ideas to use workqueue to process napi poll just a
few weeks ago.

The main reason we'd like to push forward with this idea is that the
scheduler has poor visibility into cpu cycles spent in softirq context,
and is not able to make optimal scheduling decisions of the user threads.
For example, we see in one of the application benchmark where network
load is high, the CPUs handling network softirqs has ~80% cpu util. And
user threads are still scheduled on those CPUs, despite other more idle
cpus available in the system. And we see very high tail latencies. In this
case, we have to explicitly pin away user threads from the CPUs handling
network softirqs to ensure good performance.
With napi poll moved to kthread, scheduler is in charge of scheduling both
the kthreads handling network load, and the user threads, and is able to
make better decisions. In the previous benchmark, if we do this and we
pin the kthreads processing napi poll to specific CPUs, scheduler is
able to schedule user threads away from these CPUs automatically.

And the reason we prefer 1 kthread per napi, instead of 1 workqueue
entity per host, is that kthread is more configurable than workqueue,
and we could leverage existing tuning tools for threads, like taskset,
chrt, etc to tune scheduling class and cpu set, etc. Another reason is
if we eventually want to provide busy poll feature using kernel threads
for napi poll, kthread seems to be more suitable than workqueue. 

In this patch series, I revived Paolo and Hannes's patch in 2016 and
left them as the first 2 patches. Then there are changes proposed by
Felix, Jakub, Paolo and myself on top of those, with suggestions from
Eric Dumazet.

In terms of performance, I ran tcp_rr tests with 1000 flows with
various request/response sizes, with RFS/RPS disabled, and compared
performance between softirq vs kthread. Host has 56 hyper threads and
100Gbps nic.

        req/resp   QPS   50%tile    90%tile    99%tile    99.9%tile
softirq   1B/1B   2.19M   284us       987us      1.1ms      1.56ms
kthread   1B/1B   2.14M   295us       987us      1.0ms      1.17ms

softirq 5KB/5KB   1.31M   869us      1.06ms     1.28ms      2.38ms
kthread 5KB/5KB   1.32M   878us      1.06ms     1.26ms      1.66ms

softirq 1MB/1MB  10.78K   84ms       166ms      234ms       294ms
kthread 1MB/1MB  10.83K   82ms       173ms      262ms       320ms

I also ran one application benchmark where the user threads have more
work to do. We do see good amount of tail latency reductions with the
kthread model. 

Paolo Abeni (2):
  net: implement threaded-able napi poll loop support
  net: add sysfs attribute to control napi threaded mode
Felix Fietkau (1):
  net: extract napi poll functionality to __napi_poll()
Jakub Kicinski (1):
  net: modify kthread handler to use __napi_poll()
Paolo Abeni (1):
  net: process RPS/RFS work in kthread context
Wei Wang (1):
  net: improve napi threaded config

 include/linux/netdevice.h |   6 ++
 net/core/dev.c            | 146 +++++++++++++++++++++++++++++++++++---
 net/core/net-sysfs.c      |  99 ++++++++++++++++++++++++++
 3 files changed, 242 insertions(+), 9 deletions(-)

-- 
2.28.0.618.gf4bc123cb7-goog


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

* [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
@ 2020-09-14 17:24 ` Wei Wang
  2020-09-25 19:45   ` Hannes Frederic Sowa
  2020-09-14 17:24 ` [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode Wei Wang
                   ` (6 subsequent siblings)
  7 siblings, 1 reply; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Paolo Abeni <pabeni@redhat.com> 

This patch allows running each napi poll loop inside its
own kernel thread.
The rx mode can be enabled per napi instance via the
newly addded napi_set_threaded() api; the requested kthread
will be created on demand and shut down on device stop.

Once that threaded mode is enabled and the kthread is
started, napi_schedule() will wake-up such thread instead
of scheduling the softirq.

The threaded poll loop behaves quite likely the net_rx_action,
but it does not have to manipulate local irqs and uses
an explicit scheduling point based on netdev_budget.

Signed-off-by: Paolo Abeni <pabeni@redhat.com>
Signed-off-by: Hannes Frederic Sowa <hannes@stressinduktion.org>
Signed-off-by: Wei Wang <weiwan@google.com>
---
 include/linux/netdevice.h |   5 ++
 net/core/dev.c            | 113 ++++++++++++++++++++++++++++++++++++++
 2 files changed, 118 insertions(+)

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index 157e0242e9ee..6797eb356e2e 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -348,6 +348,7 @@ struct napi_struct {
 	struct list_head	dev_list;
 	struct hlist_node	napi_hash_node;
 	unsigned int		napi_id;
+	struct task_struct	*thread;
 };
 
 enum {
@@ -358,6 +359,7 @@ enum {
 	NAPI_STATE_LISTED,	/* NAPI added to system lists */
 	NAPI_STATE_NO_BUSY_POLL,/* Do not add in napi_hash, no busy polling */
 	NAPI_STATE_IN_BUSY_POLL,/* sk_busy_loop() owns this NAPI */
+	NAPI_STATE_THREADED,	/* The poll is performed inside its own thread*/
 };
 
 enum {
@@ -368,6 +370,7 @@ enum {
 	NAPIF_STATE_LISTED	 = BIT(NAPI_STATE_LISTED),
 	NAPIF_STATE_NO_BUSY_POLL = BIT(NAPI_STATE_NO_BUSY_POLL),
 	NAPIF_STATE_IN_BUSY_POLL = BIT(NAPI_STATE_IN_BUSY_POLL),
+	NAPIF_STATE_THREADED	 = BIT(NAPI_STATE_THREADED),
 };
 
 enum gro_result {
@@ -489,6 +492,8 @@ static inline bool napi_complete(struct napi_struct *n)
 	return napi_complete_done(n, 0);
 }
 
+int napi_set_threaded(struct napi_struct *n, bool threded);
+
 /**
  *	napi_disable - prevent NAPI from scheduling
  *	@n: NAPI context
diff --git a/net/core/dev.c b/net/core/dev.c
index 03624192862a..0fe4c531b682 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -91,6 +91,7 @@
 #include <linux/etherdevice.h>
 #include <linux/ethtool.h>
 #include <linux/skbuff.h>
+#include <linux/kthread.h>
 #include <linux/bpf.h>
 #include <linux/bpf_trace.h>
 #include <net/net_namespace.h>
@@ -1486,9 +1487,19 @@ void netdev_notify_peers(struct net_device *dev)
 }
 EXPORT_SYMBOL(netdev_notify_peers);
 
+static int napi_threaded_poll(void *data);
+
+static void napi_thread_start(struct napi_struct *n)
+{
+	if (test_bit(NAPI_STATE_THREADED, &n->state) && !n->thread)
+		n->thread = kthread_create(napi_threaded_poll, n, "%s-%d",
+					   n->dev->name, n->napi_id);
+}
+
 static int __dev_open(struct net_device *dev, struct netlink_ext_ack *extack)
 {
 	const struct net_device_ops *ops = dev->netdev_ops;
+	struct napi_struct *n;
 	int ret;
 
 	ASSERT_RTNL();
@@ -1520,6 +1531,9 @@ static int __dev_open(struct net_device *dev, struct netlink_ext_ack *extack)
 	if (!ret && ops->ndo_open)
 		ret = ops->ndo_open(dev);
 
+	list_for_each_entry(n, &dev->napi_list, dev_list)
+		napi_thread_start(n);
+
 	netpoll_poll_enable(dev);
 
 	if (ret)
@@ -1565,6 +1579,14 @@ int dev_open(struct net_device *dev, struct netlink_ext_ack *extack)
 }
 EXPORT_SYMBOL(dev_open);
 
+static void napi_thread_stop(struct napi_struct *n)
+{
+	if (!n->thread)
+		return;
+	kthread_stop(n->thread);
+	n->thread = NULL;
+}
+
 static void __dev_close_many(struct list_head *head)
 {
 	struct net_device *dev;
@@ -1593,6 +1615,7 @@ static void __dev_close_many(struct list_head *head)
 
 	list_for_each_entry(dev, head, close_list) {
 		const struct net_device_ops *ops = dev->netdev_ops;
+		struct napi_struct *n;
 
 		/*
 		 *	Call the device specific close. This cannot fail.
@@ -1604,6 +1627,9 @@ static void __dev_close_many(struct list_head *head)
 		if (ops->ndo_stop)
 			ops->ndo_stop(dev);
 
+		list_for_each_entry(n, &dev->napi_list, dev_list)
+			napi_thread_stop(n);
+
 		dev->flags &= ~IFF_UP;
 		netpoll_poll_enable(dev);
 	}
@@ -4240,6 +4266,11 @@ int gro_normal_batch __read_mostly = 8;
 static inline void ____napi_schedule(struct softnet_data *sd,
 				     struct napi_struct *napi)
 {
+	if (napi->thread) {
+		wake_up_process(napi->thread);
+		return;
+	}
+
 	list_add_tail(&napi->poll_list, &sd->poll_list);
 	__raise_softirq_irqoff(NET_RX_SOFTIRQ);
 }
@@ -6590,6 +6621,30 @@ static void init_gro_hash(struct napi_struct *napi)
 	napi->gro_bitmask = 0;
 }
 
+int napi_set_threaded(struct napi_struct *n, bool threaded)
+{
+	ASSERT_RTNL();
+
+	if (n->dev->flags & IFF_UP)
+		return -EBUSY;
+
+	if (threaded == !!test_bit(NAPI_STATE_THREADED, &n->state))
+		return 0;
+	if (threaded)
+		set_bit(NAPI_STATE_THREADED, &n->state);
+	else
+		clear_bit(NAPI_STATE_THREADED, &n->state);
+
+	/* if the device is initializing, nothing todo */
+	if (test_bit(__LINK_STATE_START, &n->dev->state))
+		return 0;
+
+	napi_thread_stop(n);
+	napi_thread_start(n);
+	return 0;
+}
+EXPORT_SYMBOL(napi_set_threaded);
+
 void netif_napi_add(struct net_device *dev, struct napi_struct *napi,
 		    int (*poll)(struct napi_struct *, int), int weight)
 {
@@ -6730,6 +6785,64 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
 	return work;
 }
 
+static int napi_thread_wait(struct napi_struct *napi)
+{
+	set_current_state(TASK_INTERRUPTIBLE);
+
+	while (!kthread_should_stop() && !napi_disable_pending(napi)) {
+		if (test_bit(NAPI_STATE_SCHED, &napi->state)) {
+			__set_current_state(TASK_RUNNING);
+			return 0;
+		}
+
+		schedule();
+		set_current_state(TASK_INTERRUPTIBLE);
+	}
+	__set_current_state(TASK_RUNNING);
+	return -1;
+}
+
+static int napi_threaded_poll(void *data)
+{
+	struct napi_struct *napi = data;
+
+	while (!napi_thread_wait(napi)) {
+		struct list_head dummy_repoll;
+		int budget = netdev_budget;
+		unsigned long time_limit;
+		bool again = true;
+
+		INIT_LIST_HEAD(&dummy_repoll);
+		local_bh_disable();
+		time_limit = jiffies + 2;
+		do {
+			/* ensure that the poll list is not empty */
+			if (list_empty(&dummy_repoll))
+				list_add(&napi->poll_list, &dummy_repoll);
+
+			budget -= napi_poll(napi, &dummy_repoll);
+			if (unlikely(budget <= 0 ||
+				     time_after_eq(jiffies, time_limit))) {
+				cond_resched();
+
+				/* refresh the budget */
+				budget = netdev_budget;
+				__kfree_skb_flush();
+				time_limit = jiffies + 2;
+			}
+
+			if (napi_disable_pending(napi))
+				again = false;
+			else if (!test_bit(NAPI_STATE_SCHED, &napi->state))
+				again = false;
+		} while (again);
+
+		__kfree_skb_flush();
+		local_bh_enable();
+	}
+	return 0;
+}
+
 static __latent_entropy void net_rx_action(struct softirq_action *h)
 {
 	struct softnet_data *sd = this_cpu_ptr(&softnet_data);
-- 
2.28.0.618.gf4bc123cb7-goog


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

* [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support Wei Wang
@ 2020-09-14 17:24 ` Wei Wang
  2020-09-15  2:50   ` kernel test robot
  2020-09-15  3:47   ` kernel test robot
  2020-09-14 17:24 ` [RFC PATCH net-next 3/6] net: extract napi poll functionality to __napi_poll() Wei Wang
                   ` (5 subsequent siblings)
  7 siblings, 2 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Paolo Abeni <pabeni@redhat.com> 

this patch adds a new sysfs attribute to the network
device class. Said attribute is a bitmask that allows controlling
the threaded mode for all the napi instances of the given
network device.

The threaded mode can be switched only if related network device
is down.

Signed-off-by: Paolo Abeni <pabeni@redhat.com>
Signed-off-by: Hannes Frederic Sowa <hannes@stressinduktion.org>
Signed-off-by: Wei Wang <weiwan@google.com>
---
 include/linux/netdevice.h |   1 +
 net/core/net-sysfs.c      | 102 ++++++++++++++++++++++++++++++++++++++
 2 files changed, 103 insertions(+)

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index 6797eb356e2e..37941d6a911d 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -1965,6 +1965,7 @@ struct net_device {
 	spinlock_t		addr_list_lock;
 	unsigned char		name_assign_type;
 	bool			uc_promisc;
+	bool			napi_threaded;
 	struct netdev_hw_addr_list	uc;
 	struct netdev_hw_addr_list	mc;
 	struct netdev_hw_addr_list	dev_addrs;
diff --git a/net/core/net-sysfs.c b/net/core/net-sysfs.c
index efec66fa78b7..0172457a1bfe 100644
--- a/net/core/net-sysfs.c
+++ b/net/core/net-sysfs.c
@@ -538,6 +538,107 @@ static ssize_t phys_switch_id_show(struct device *dev,
 }
 static DEVICE_ATTR_RO(phys_switch_id);
 
+unsigned long *__alloc_thread_bitmap(struct net_device *netdev, int *bits)
+{
+	struct napi_struct *n;
+
+	*bits = 0;
+	list_for_each_entry(n, &netdev->napi_list, dev_list)
+		(*bits)++;
+
+	return kmalloc_array(BITS_TO_LONGS(*bits), sizeof(unsigned long),
+			     GFP_ATOMIC | __GFP_ZERO);
+}
+
+static ssize_t threaded_show(struct device *dev,
+			     struct device_attribute *attr, char *buf)
+{
+	struct net_device *netdev = to_net_dev(dev);
+	struct napi_struct *n;
+	unsigned long *bmap;
+	size_t count = 0;
+	int i, bits;
+
+	if (!rtnl_trylock())
+		return restart_syscall();
+
+	if (!dev_isalive(netdev))
+		goto unlock;
+
+	bmap = __alloc_thread_bitmap(netdev, &bits);
+	if (!bmap) {
+		count = -ENOMEM;
+		goto unlock;
+	}
+
+	i = 0;
+	list_for_each_entry(n, &netdev->napi_list, dev_list) {
+		if (test_bit(NAPI_STATE_THREADED, &n->state))
+			set_bit(i, bmap);
+		i++;
+	}
+
+	count = bitmap_print_to_pagebuf(true, buf, bmap, bits);
+	kfree(bmap);
+
+unlock:
+	rtnl_unlock();
+
+	return count;
+}
+
+static ssize_t threaded_store(struct device *dev,
+			      struct device_attribute *attr,
+			      const char *buf, size_t len)
+{
+	struct net_device *netdev = to_net_dev(dev);
+	struct napi_struct *n;
+	unsigned long *bmap;
+	int i, bits;
+	size_t ret;
+
+	if (!capable(CAP_NET_ADMIN))
+		return -EPERM;
+
+	if (!rtnl_trylock())
+		return restart_syscall();
+
+	if (!dev_isalive(netdev)) {
+		ret = len;
+		goto unlock;
+	}
+
+	if (netdev->flags & IFF_UP) {
+		ret = -EBUSY;
+		goto unlock;
+	}
+
+	bmap = __alloc_thread_bitmap(netdev, &bits);
+	if (!bmap) {
+		ret = -ENOMEM;
+		goto unlock;
+	}
+
+	ret = bitmap_parselist(buf, bmap, bits);
+	if (ret)
+		goto free_unlock;
+
+	i = 0;
+	list_for_each_entry(n, &netdev->napi_list, dev_list) {
+		napi_set_threaded(n, test_bit(i, bmap));
+		i++;
+	}
+	ret = len;
+
+free_unlock:
+	kfree(bmap);
+
+unlock:
+	rtnl_unlock();
+	return ret;
+}
+static DEVICE_ATTR_RW(threaded);
+
 static struct attribute *net_class_attrs[] __ro_after_init = {
 	&dev_attr_netdev_group.attr,
 	&dev_attr_type.attr,
@@ -570,6 +671,7 @@ static struct attribute *net_class_attrs[] __ro_after_init = {
 	&dev_attr_proto_down.attr,
 	&dev_attr_carrier_up_count.attr,
 	&dev_attr_carrier_down_count.attr,
+	&dev_attr_threaded.attr,
 	NULL,
 };
 ATTRIBUTE_GROUPS(net_class);
-- 
2.28.0.618.gf4bc123cb7-goog


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

* [RFC PATCH net-next 3/6] net: extract napi poll functionality to __napi_poll()
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode Wei Wang
@ 2020-09-14 17:24 ` Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 4/6] net: modify kthread handler to use __napi_poll() Wei Wang
                   ` (4 subsequent siblings)
  7 siblings, 0 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Felix Fietkau <nbd@nbd.name> 

This commit introduces a new function __napi_poll() which does the main
logic of the existing napi_poll() function, and will be called by other
functions in later commits.
This idea and implementation is done by Felix Fietkau <nbd@nbd.name> and
is proposed as part of the patch to move napi work to work_queue
context.
This commit by itself is a code restructure.

Signed-off-by: Felix Fietkau <nbd@nbd.name>
Signed-off-by: Wei Wang <weiwan@google.com>
---
 net/core/dev.c | 31 +++++++++++++++++++++++--------
 1 file changed, 23 insertions(+), 8 deletions(-)

diff --git a/net/core/dev.c b/net/core/dev.c
index 0fe4c531b682..bc2a7681b239 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -6719,15 +6719,10 @@ void __netif_napi_del(struct napi_struct *napi)
 }
 EXPORT_SYMBOL(__netif_napi_del);
 
-static int napi_poll(struct napi_struct *n, struct list_head *repoll)
+static int __napi_poll(struct napi_struct *n, bool *repoll)
 {
-	void *have;
 	int work, weight;
 
-	list_del_init(&n->poll_list);
-
-	have = netpoll_poll_lock(n);
-
 	weight = n->weight;
 
 	/* This NAPI_STATE_SCHED test is for avoiding a race
@@ -6747,7 +6742,7 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
 			    n->poll, work, weight);
 
 	if (likely(work < weight))
-		goto out_unlock;
+		return work;
 
 	/* Drivers must not modify the NAPI state if they
 	 * consume the entire weight.  In such cases this code
@@ -6756,7 +6751,7 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
 	 */
 	if (unlikely(napi_disable_pending(n))) {
 		napi_complete(n);
-		goto out_unlock;
+		return work;
 	}
 
 	if (n->gro_bitmask) {
@@ -6768,6 +6763,26 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
 
 	gro_normal_list(n);
 
+	*repoll = true;
+
+	return work;
+}
+
+static int napi_poll(struct napi_struct *n, struct list_head *repoll)
+{
+	bool do_repoll = false;
+	void *have;
+	int work;
+
+	list_del_init(&n->poll_list);
+
+	have = netpoll_poll_lock(n);
+
+	work = __napi_poll(n, &do_repoll);
+
+	if (!do_repoll)
+		goto out_unlock;
+
 	/* Some drivers may have called napi_schedule
 	 * prior to exhausting their budget.
 	 */
-- 
2.28.0.618.gf4bc123cb7-goog


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

* [RFC PATCH net-next 4/6] net: modify kthread handler to use __napi_poll()
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
                   ` (2 preceding siblings ...)
  2020-09-14 17:24 ` [RFC PATCH net-next 3/6] net: extract napi poll functionality to __napi_poll() Wei Wang
@ 2020-09-14 17:24 ` Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context Wei Wang
                   ` (3 subsequent siblings)
  7 siblings, 0 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Jakub Kicinski <kuba@kernel.org> 

The current kthread handler calls napi_poll() and has to pass a dummy
repoll list to the function, which seems redundent. The new proposed
kthread handler calls the newly proposed __napi_poll(), and respects
napi->weight as before. If repoll is needed, cond_resched() is called
first to give other tasks a chance to run before repolling.
This change is proposed by Jakub Kicinski <kuba@kernel.org> on top of
the previous patch.

Signed-off-by: Jakub Kicinski <kuba@kernel.org>
Signed-off-by: Wei Wang <weiwan@google.com>
---
 net/core/dev.c | 62 +++++++++++++++++++-------------------------------
 1 file changed, 24 insertions(+), 38 deletions(-)

diff --git a/net/core/dev.c b/net/core/dev.c
index bc2a7681b239..be676c21bdc4 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -6763,6 +6763,15 @@ static int __napi_poll(struct napi_struct *n, bool *repoll)
 
 	gro_normal_list(n);
 
+	/* Some drivers may have called napi_schedule
+	 * prior to exhausting their budget.
+	 */
+	if (unlikely(!list_empty(&n->poll_list))) {
+		pr_warn_once("%s: Budget exhausted after napi rescheduled\n",
+			     n->dev ? n->dev->name : "backlog");
+		return work;
+	}
+
 	*repoll = true;
 
 	return work;
@@ -6783,15 +6792,6 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
 	if (!do_repoll)
 		goto out_unlock;
 
-	/* Some drivers may have called napi_schedule
-	 * prior to exhausting their budget.
-	 */
-	if (unlikely(!list_empty(&n->poll_list))) {
-		pr_warn_once("%s: Budget exhausted after napi rescheduled\n",
-			     n->dev ? n->dev->name : "backlog");
-		goto out_unlock;
-	}
-
 	list_add_tail(&n->poll_list, repoll);
 
 out_unlock:
@@ -6820,40 +6820,26 @@ static int napi_thread_wait(struct napi_struct *napi)
 static int napi_threaded_poll(void *data)
 {
 	struct napi_struct *napi = data;
+	void *have;
 
 	while (!napi_thread_wait(napi)) {
-		struct list_head dummy_repoll;
-		int budget = netdev_budget;
-		unsigned long time_limit;
-		bool again = true;
+		for (;;) {
+			bool repoll = false;
 
-		INIT_LIST_HEAD(&dummy_repoll);
-		local_bh_disable();
-		time_limit = jiffies + 2;
-		do {
-			/* ensure that the poll list is not empty */
-			if (list_empty(&dummy_repoll))
-				list_add(&napi->poll_list, &dummy_repoll);
-
-			budget -= napi_poll(napi, &dummy_repoll);
-			if (unlikely(budget <= 0 ||
-				     time_after_eq(jiffies, time_limit))) {
-				cond_resched();
-
-				/* refresh the budget */
-				budget = netdev_budget;
-				__kfree_skb_flush();
-				time_limit = jiffies + 2;
-			}
+			local_bh_disable();
 
-			if (napi_disable_pending(napi))
-				again = false;
-			else if (!test_bit(NAPI_STATE_SCHED, &napi->state))
-				again = false;
-		} while (again);
+			have = netpoll_poll_lock(napi);
+			__napi_poll(napi, &repoll);
+			netpoll_poll_unlock(have);
 
-		__kfree_skb_flush();
-		local_bh_enable();
+			__kfree_skb_flush();
+			local_bh_enable();
+
+			if (!repoll)
+				break;
+
+			cond_resched();
+		}
 	}
 	return 0;
 }
-- 
2.28.0.618.gf4bc123cb7-goog


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

* [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
                   ` (3 preceding siblings ...)
  2020-09-14 17:24 ` [RFC PATCH net-next 4/6] net: modify kthread handler to use __napi_poll() Wei Wang
@ 2020-09-14 17:24 ` Wei Wang
  2020-09-18 22:44   ` Wei Wang
  2020-09-14 17:24 ` [RFC PATCH net-next 6/6] net: improve napi threaded config Wei Wang
                   ` (2 subsequent siblings)
  7 siblings, 1 reply; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Paolo Abeni <pabeni@redhat.com>

This patch adds the missing part to handle RFS/RPS in the napi thread
handler and makes sure RPS/RFS works properly when using kthread to do
napi poll.

Signed-off-by: Paolo Abeni <pabeni@redhat.com>
Signed-off-by: Wei Wang <weiwan@google.com>
---
 net/core/dev.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/net/core/dev.c b/net/core/dev.c
index be676c21bdc4..ab8af727058b 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -6820,6 +6820,7 @@ static int napi_thread_wait(struct napi_struct *napi)
 static int napi_threaded_poll(void *data)
 {
 	struct napi_struct *napi = data;
+	struct softnet_data *sd;
 	void *have;
 
 	while (!napi_thread_wait(napi)) {
@@ -6835,6 +6836,12 @@ static int napi_threaded_poll(void *data)
 			__kfree_skb_flush();
 			local_bh_enable();
 
+			sd = this_cpu_ptr(&softnet_data);
+			if (sd_has_rps_ipi_waiting(sd)) {
+				local_irq_disable();
+				net_rps_action_and_irq_enable(sd);
+			}
+
 			if (!repoll)
 				break;
 
-- 
2.28.0.618.gf4bc123cb7-goog


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

* [RFC PATCH net-next 6/6] net: improve napi threaded config
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
                   ` (4 preceding siblings ...)
  2020-09-14 17:24 ` [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context Wei Wang
@ 2020-09-14 17:24 ` Wei Wang
  2020-09-25 13:48 ` [RFC PATCH net-next 0/6] implement kthread based napi poll Magnus Karlsson
  2020-09-28 17:43 ` Eric Dumazet
  7 siblings, 0 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-14 17:24 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

This commit mainly addresses the threaded config to make the switch
between softirq based and kthread based NAPI processing not require
a device down/up.
It also moves the kthread_create() call to the sysfs handler when user
tries to enable "threaded" on napi, and properly handles the
kthread_create() failure. This is because certain drivers do not have
the napi created and linked to the dev when dev_open() is called. So
the previous implementation does not work properly there.

Signed-off-by: Wei Wang <weiwan@google.com>
---
 net/core/dev.c       | 49 +++++++++++++++++++++++++-------------------
 net/core/net-sysfs.c |  9 +++-----
 2 files changed, 31 insertions(+), 27 deletions(-)

diff --git a/net/core/dev.c b/net/core/dev.c
index ab8af727058b..9f7df61f7c9a 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -1489,17 +1489,24 @@ EXPORT_SYMBOL(netdev_notify_peers);
 
 static int napi_threaded_poll(void *data);
 
-static void napi_thread_start(struct napi_struct *n)
+static int napi_kthread_create(struct napi_struct *n)
 {
-	if (test_bit(NAPI_STATE_THREADED, &n->state) && !n->thread)
-		n->thread = kthread_create(napi_threaded_poll, n, "%s-%d",
-					   n->dev->name, n->napi_id);
+	int err = 0;
+
+	n->thread = kthread_create(napi_threaded_poll, n, "%s-%d",
+				   n->dev->name, n->napi_id);
+	if (IS_ERR(n->thread)) {
+		err = PTR_ERR(n->thread);
+		pr_err("kthread_create failed with err %d\n", err);
+		n->thread = NULL;
+	}
+
+	return err;
 }
 
 static int __dev_open(struct net_device *dev, struct netlink_ext_ack *extack)
 {
 	const struct net_device_ops *ops = dev->netdev_ops;
-	struct napi_struct *n;
 	int ret;
 
 	ASSERT_RTNL();
@@ -1531,9 +1538,6 @@ static int __dev_open(struct net_device *dev, struct netlink_ext_ack *extack)
 	if (!ret && ops->ndo_open)
 		ret = ops->ndo_open(dev);
 
-	list_for_each_entry(n, &dev->napi_list, dev_list)
-		napi_thread_start(n);
-
 	netpoll_poll_enable(dev);
 
 	if (ret)
@@ -1584,6 +1588,7 @@ static void napi_thread_stop(struct napi_struct *n)
 	if (!n->thread)
 		return;
 	kthread_stop(n->thread);
+	clear_bit(NAPI_STATE_THREADED, &n->state);
 	n->thread = NULL;
 }
 
@@ -4266,7 +4271,7 @@ int gro_normal_batch __read_mostly = 8;
 static inline void ____napi_schedule(struct softnet_data *sd,
 				     struct napi_struct *napi)
 {
-	if (napi->thread) {
+	if (test_bit(NAPI_STATE_THREADED, &napi->state)) {
 		wake_up_process(napi->thread);
 		return;
 	}
@@ -6623,25 +6628,25 @@ static void init_gro_hash(struct napi_struct *napi)
 
 int napi_set_threaded(struct napi_struct *n, bool threaded)
 {
-	ASSERT_RTNL();
+	int err = 0;
 
-	if (n->dev->flags & IFF_UP)
-		return -EBUSY;
+	ASSERT_RTNL();
 
 	if (threaded == !!test_bit(NAPI_STATE_THREADED, &n->state))
 		return 0;
-	if (threaded)
+	if (threaded) {
+		if (!n->thread) {
+			err = napi_kthread_create(n);
+			if (err)
+				goto out;
+		}
 		set_bit(NAPI_STATE_THREADED, &n->state);
-	else
+	} else {
 		clear_bit(NAPI_STATE_THREADED, &n->state);
+	}
 
-	/* if the device is initializing, nothing todo */
-	if (test_bit(__LINK_STATE_START, &n->dev->state))
-		return 0;
-
-	napi_thread_stop(n);
-	napi_thread_start(n);
-	return 0;
+out:
+	return err;
 }
 EXPORT_SYMBOL(napi_set_threaded);
 
@@ -6686,6 +6691,7 @@ void napi_disable(struct napi_struct *n)
 		msleep(1);
 
 	hrtimer_cancel(&n->timer);
+	napi_thread_stop(n);
 
 	clear_bit(NAPI_STATE_DISABLE, &n->state);
 }
@@ -6806,6 +6812,7 @@ static int napi_thread_wait(struct napi_struct *napi)
 
 	while (!kthread_should_stop() && !napi_disable_pending(napi)) {
 		if (test_bit(NAPI_STATE_SCHED, &napi->state)) {
+			WARN_ON(!list_empty(&napi->poll_list));
 			__set_current_state(TASK_RUNNING);
 			return 0;
 		}
diff --git a/net/core/net-sysfs.c b/net/core/net-sysfs.c
index 0172457a1bfe..48b7582a0372 100644
--- a/net/core/net-sysfs.c
+++ b/net/core/net-sysfs.c
@@ -608,11 +608,6 @@ static ssize_t threaded_store(struct device *dev,
 		goto unlock;
 	}
 
-	if (netdev->flags & IFF_UP) {
-		ret = -EBUSY;
-		goto unlock;
-	}
-
 	bmap = __alloc_thread_bitmap(netdev, &bits);
 	if (!bmap) {
 		ret = -ENOMEM;
@@ -625,7 +620,9 @@ static ssize_t threaded_store(struct device *dev,
 
 	i = 0;
 	list_for_each_entry(n, &netdev->napi_list, dev_list) {
-		napi_set_threaded(n, test_bit(i, bmap));
+		ret = napi_set_threaded(n, test_bit(i, bmap));
+		if (ret)
+			goto free_unlock;
 		i++;
 	}
 	ret = len;
-- 
2.28.0.618.gf4bc123cb7-goog


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

* Re: [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode
  2020-09-14 17:24 ` [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode Wei Wang
@ 2020-09-15  2:50   ` kernel test robot
  2020-09-15  3:47   ` kernel test robot
  1 sibling, 0 replies; 32+ messages in thread
From: kernel test robot @ 2020-09-15  2:50 UTC (permalink / raw)
  To: kbuild-all

[-- Attachment #1: Type: text/plain, Size: 2442 bytes --]

Hi Wei,

[FYI, it's a private test report for your RFC patch.]
[auto build test WARNING on net-next/master]

url:    https://github.com/0day-ci/linux/commits/Wei-Wang/implement-kthread-based-napi-poll/20200915-013651
base:   https://git.kernel.org/pub/scm/linux/kernel/git/davem/net-next.git e059c6f340f6fccadd3db9993f06d4cc51305804
config: arm64-randconfig-r024-20200914 (attached as .config)
compiler: clang version 12.0.0 (https://github.com/llvm/llvm-project b2c32c90bab09a6e2c1f370429db26017a182143)
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # install arm64 cross compiling tool for clang build
        # apt-get install binutils-aarch64-linux-gnu
        # save the attached .config to linux build tree
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=clang make.cross ARCH=arm64 

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All warnings (new ones prefixed by >>):

>> net/core/net-sysfs.c:541:16: warning: no previous prototype for function '__alloc_thread_bitmap' [-Wmissing-prototypes]
   unsigned long *__alloc_thread_bitmap(struct net_device *netdev, int *bits)
                  ^
   net/core/net-sysfs.c:541:1: note: declare 'static' if the function is not intended to be used outside of this translation unit
   unsigned long *__alloc_thread_bitmap(struct net_device *netdev, int *bits)
   ^
   static 
   1 warning generated.

# https://github.com/0day-ci/linux/commit/ba7d63f1040aedba86f70713af085d86322fee62
git remote add linux-review https://github.com/0day-ci/linux
git fetch --no-tags linux-review Wei-Wang/implement-kthread-based-napi-poll/20200915-013651
git checkout ba7d63f1040aedba86f70713af085d86322fee62
vim +/__alloc_thread_bitmap +541 net/core/net-sysfs.c

   540	
 > 541	unsigned long *__alloc_thread_bitmap(struct net_device *netdev, int *bits)
   542	{
   543		struct napi_struct *n;
   544	
   545		*bits = 0;
   546		list_for_each_entry(n, &netdev->napi_list, dev_list)
   547			(*bits)++;
   548	
   549		return kmalloc_array(BITS_TO_LONGS(*bits), sizeof(unsigned long),
   550				     GFP_ATOMIC | __GFP_ZERO);
   551	}
   552	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

[-- Attachment #2: config.gz --]
[-- Type: application/gzip, Size: 37469 bytes --]

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

* Re: [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode
  2020-09-14 17:24 ` [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode Wei Wang
  2020-09-15  2:50   ` kernel test robot
@ 2020-09-15  3:47   ` kernel test robot
  1 sibling, 0 replies; 32+ messages in thread
From: kernel test robot @ 2020-09-15  3:47 UTC (permalink / raw)
  To: kbuild-all

[-- Attachment #1: Type: text/plain, Size: 1835 bytes --]

Hi Wei,

[FYI, it's a private test report for your RFC patch.]
[auto build test WARNING on net-next/master]

url:    https://github.com/0day-ci/linux/commits/Wei-Wang/implement-kthread-based-napi-poll/20200915-013651
base:   https://git.kernel.org/pub/scm/linux/kernel/git/davem/net-next.git e059c6f340f6fccadd3db9993f06d4cc51305804
config: i386-randconfig-r035-20200913 (attached as .config)
compiler: gcc-9 (Debian 9.3.0-15) 9.3.0
reproduce (this is a W=1 build):
        # save the attached .config to linux build tree
        make W=1 ARCH=i386 

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All warnings (new ones prefixed by >>):

>> net/core/net-sysfs.c:541:16: warning: no previous prototype for '__alloc_thread_bitmap' [-Wmissing-prototypes]
     541 | unsigned long *__alloc_thread_bitmap(struct net_device *netdev, int *bits)
         |                ^~~~~~~~~~~~~~~~~~~~~

# https://github.com/0day-ci/linux/commit/ba7d63f1040aedba86f70713af085d86322fee62
git remote add linux-review https://github.com/0day-ci/linux
git fetch --no-tags linux-review Wei-Wang/implement-kthread-based-napi-poll/20200915-013651
git checkout ba7d63f1040aedba86f70713af085d86322fee62
vim +/__alloc_thread_bitmap +541 net/core/net-sysfs.c

   540	
 > 541	unsigned long *__alloc_thread_bitmap(struct net_device *netdev, int *bits)
   542	{
   543		struct napi_struct *n;
   544	
   545		*bits = 0;
   546		list_for_each_entry(n, &netdev->napi_list, dev_list)
   547			(*bits)++;
   548	
   549		return kmalloc_array(BITS_TO_LONGS(*bits), sizeof(unsigned long),
   550				     GFP_ATOMIC | __GFP_ZERO);
   551	}
   552	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

[-- Attachment #2: config.gz --]
[-- Type: application/gzip, Size: 30498 bytes --]

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

* Re: [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context
  2020-09-14 17:24 ` [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context Wei Wang
@ 2020-09-18 22:44   ` Wei Wang
  2020-09-21  8:11     ` Eric Dumazet
  0 siblings, 1 reply; 32+ messages in thread
From: Wei Wang @ 2020-09-18 22:44 UTC (permalink / raw)
  To: David S . Miller, Linux Kernel Network Developers
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau

On Mon, Sep 14, 2020 at 10:26 AM Wei Wang <weiwan@google.com> wrote:
>
> From: Paolo Abeni <pabeni@redhat.com>
>
> This patch adds the missing part to handle RFS/RPS in the napi thread
> handler and makes sure RPS/RFS works properly when using kthread to do
> napi poll.
>
> Signed-off-by: Paolo Abeni <pabeni@redhat.com>
> Signed-off-by: Wei Wang <weiwan@google.com>
> ---

With some more thoughts, I think this patch is not needed. RPS/RFS
uses its own napi (sd->backlog) which currently does not have
NAPI_STATE_THREADED set. So it is still being handled in softirq
context by net_rx_action().
I will remove this patch in the next version if no one objects.


>  net/core/dev.c | 7 +++++++
>  1 file changed, 7 insertions(+)
>
> diff --git a/net/core/dev.c b/net/core/dev.c
> index be676c21bdc4..ab8af727058b 100644
> --- a/net/core/dev.c
> +++ b/net/core/dev.c
> @@ -6820,6 +6820,7 @@ static int napi_thread_wait(struct napi_struct *napi)
>  static int napi_threaded_poll(void *data)
>  {
>         struct napi_struct *napi = data;
> +       struct softnet_data *sd;
>         void *have;
>
>         while (!napi_thread_wait(napi)) {
> @@ -6835,6 +6836,12 @@ static int napi_threaded_poll(void *data)
>                         __kfree_skb_flush();
>                         local_bh_enable();
>
> +                       sd = this_cpu_ptr(&softnet_data);
> +                       if (sd_has_rps_ipi_waiting(sd)) {
> +                               local_irq_disable();
> +                               net_rps_action_and_irq_enable(sd);
> +                       }
> +
>                         if (!repoll)
>                                 break;
>
> --
> 2.28.0.618.gf4bc123cb7-goog
>

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

* Re: [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context
  2020-09-18 22:44   ` Wei Wang
@ 2020-09-21  8:11     ` Eric Dumazet
  0 siblings, 0 replies; 32+ messages in thread
From: Eric Dumazet @ 2020-09-21  8:11 UTC (permalink / raw)
  To: Wei Wang
  Cc: David S . Miller, Linux Kernel Network Developers,
	Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau

On Sat, Sep 19, 2020 at 12:45 AM Wei Wang <weiwan@google.com> wrote:
>
> On Mon, Sep 14, 2020 at 10:26 AM Wei Wang <weiwan@google.com> wrote:
> >
> > From: Paolo Abeni <pabeni@redhat.com>
> >
> > This patch adds the missing part to handle RFS/RPS in the napi thread
> > handler and makes sure RPS/RFS works properly when using kthread to do
> > napi poll.
> >
> > Signed-off-by: Paolo Abeni <pabeni@redhat.com>
> > Signed-off-by: Wei Wang <weiwan@google.com>
> > ---
>
> With some more thoughts, I think this patch is not needed. RPS/RFS
> uses its own napi (sd->backlog) which currently does not have
> NAPI_STATE_THREADED set. So it is still being handled in softirq
> context by net_rx_action().
> I will remove this patch in the next version if no one objects.

The purpose of the patch was to make sure to kick the IPI

I think we need it, otherwise RPS/RFS might add a lot of jitter.


>
>
> >  net/core/dev.c | 7 +++++++
> >  1 file changed, 7 insertions(+)
> >
> > diff --git a/net/core/dev.c b/net/core/dev.c
> > index be676c21bdc4..ab8af727058b 100644
> > --- a/net/core/dev.c
> > +++ b/net/core/dev.c
> > @@ -6820,6 +6820,7 @@ static int napi_thread_wait(struct napi_struct *napi)
> >  static int napi_threaded_poll(void *data)
> >  {
> >         struct napi_struct *napi = data;
> > +       struct softnet_data *sd;
> >         void *have;
> >
> >         while (!napi_thread_wait(napi)) {
> > @@ -6835,6 +6836,12 @@ static int napi_threaded_poll(void *data)
> >                         __kfree_skb_flush();
> >                         local_bh_enable();
> >
> > +                       sd = this_cpu_ptr(&softnet_data);
> > +                       if (sd_has_rps_ipi_waiting(sd)) {
> > +                               local_irq_disable();
> > +                               net_rps_action_and_irq_enable(sd);
> > +                       }
> > +
> >                         if (!repoll)
> >                                 break;
> >
> > --
> > 2.28.0.618.gf4bc123cb7-goog
> >

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
                   ` (5 preceding siblings ...)
  2020-09-14 17:24 ` [RFC PATCH net-next 6/6] net: improve napi threaded config Wei Wang
@ 2020-09-25 13:48 ` Magnus Karlsson
  2020-09-25 17:15   ` Wei Wang
  2020-09-25 19:06   ` Jakub Kicinski
  2020-09-28 17:43 ` Eric Dumazet
  7 siblings, 2 replies; 32+ messages in thread
From: Magnus Karlsson @ 2020-09-25 13:48 UTC (permalink / raw)
  To: Wei Wang
  Cc: David S . Miller, Network Development, Jakub Kicinski,
	Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel

On Mon, Sep 14, 2020 at 7:26 PM Wei Wang <weiwan@google.com> wrote:
>
> The idea of moving the napi poll process out of softirq context to a
> kernel thread based context is not new.
> Paolo Abeni and Hannes Frederic Sowa has proposed patches to move napi
> poll to kthread back in 2016. And Felix Fietkau has also proposed
> patches of similar ideas to use workqueue to process napi poll just a
> few weeks ago.
>
> The main reason we'd like to push forward with this idea is that the
> scheduler has poor visibility into cpu cycles spent in softirq context,
> and is not able to make optimal scheduling decisions of the user threads.
> For example, we see in one of the application benchmark where network
> load is high, the CPUs handling network softirqs has ~80% cpu util. And
> user threads are still scheduled on those CPUs, despite other more idle
> cpus available in the system. And we see very high tail latencies. In this
> case, we have to explicitly pin away user threads from the CPUs handling
> network softirqs to ensure good performance.
> With napi poll moved to kthread, scheduler is in charge of scheduling both
> the kthreads handling network load, and the user threads, and is able to
> make better decisions. In the previous benchmark, if we do this and we
> pin the kthreads processing napi poll to specific CPUs, scheduler is
> able to schedule user threads away from these CPUs automatically.
>
> And the reason we prefer 1 kthread per napi, instead of 1 workqueue
> entity per host, is that kthread is more configurable than workqueue,
> and we could leverage existing tuning tools for threads, like taskset,
> chrt, etc to tune scheduling class and cpu set, etc. Another reason is
> if we eventually want to provide busy poll feature using kernel threads
> for napi poll, kthread seems to be more suitable than workqueue.
>
> In this patch series, I revived Paolo and Hannes's patch in 2016 and
> left them as the first 2 patches. Then there are changes proposed by
> Felix, Jakub, Paolo and myself on top of those, with suggestions from
> Eric Dumazet.
>
> In terms of performance, I ran tcp_rr tests with 1000 flows with
> various request/response sizes, with RFS/RPS disabled, and compared
> performance between softirq vs kthread. Host has 56 hyper threads and
> 100Gbps nic.
>
>         req/resp   QPS   50%tile    90%tile    99%tile    99.9%tile
> softirq   1B/1B   2.19M   284us       987us      1.1ms      1.56ms
> kthread   1B/1B   2.14M   295us       987us      1.0ms      1.17ms
>
> softirq 5KB/5KB   1.31M   869us      1.06ms     1.28ms      2.38ms
> kthread 5KB/5KB   1.32M   878us      1.06ms     1.26ms      1.66ms
>
> softirq 1MB/1MB  10.78K   84ms       166ms      234ms       294ms
> kthread 1MB/1MB  10.83K   82ms       173ms      262ms       320ms
>
> I also ran one application benchmark where the user threads have more
> work to do. We do see good amount of tail latency reductions with the
> kthread model.

I really like this RFC and would encourage you to submit it as a
patch. Would love to see it make it into the kernel.

I see the same positive effects as you when trying it out with AF_XDP
sockets. Made some simple experiments where I sent 64-byte packets to
a single AF_XDP socket. Have not managed to figure out how to do
percentiles on my load generator, so this is going to be min, avg and
max only. The application using the AF_XDP socket just performs a mac
swap on the packet and sends it back to the load generator that then
measures the round trip latency. The kthread is taskset to the same
core as ksoftirqd would run on. So in each experiment, they always run
on the same core id (which is not the same as the application).

Rate 12 Mpps with 0% loss.
              Latencies (us)         Delay Variation between packets
          min    avg    max      avg   max
sofirq  11.0  17.1   78.4      0.116  63.0
kthread 11.2  17.1   35.0     0.116  20.9

Rate ~58 Mpps (Line rate at 40 Gbit/s) with substantial loss
              Latencies (us)         Delay Variation between packets
          min    avg    max      avg   max
softirq  87.6  194.9  282.6    0.062  25.9
kthread  86.5  185.2  271.8    0.061  22.5

For the last experiment, I also get 1.5% to 2% higher throughput with
your kthread approach. Moreover, just from the per-second throughput
printouts from my application, I can see that the kthread numbers are
more stable. The softirq numbers can vary quite a lot between each
second, around +-3%. But for the kthread approach, they are nice and
stable. Have not examined why.

One thing I noticed though, and I do not know if this is an issue, is
that the switching between the two modes does not occur at high packet
rates. I have to lower the packet rate to something that makes the
core work less than 100% for it to switch between ksoftirqd to kthread
and vice versa. They just seem too busy to switch at 100% load when
changing the "threaded" sysfs variable.

Thank you for working on this feature.


/Magnus


> Paolo Abeni (2):
>   net: implement threaded-able napi poll loop support
>   net: add sysfs attribute to control napi threaded mode
> Felix Fietkau (1):
>   net: extract napi poll functionality to __napi_poll()
> Jakub Kicinski (1):
>   net: modify kthread handler to use __napi_poll()
> Paolo Abeni (1):
>   net: process RPS/RFS work in kthread context
> Wei Wang (1):
>   net: improve napi threaded config
>
>  include/linux/netdevice.h |   6 ++
>  net/core/dev.c            | 146 +++++++++++++++++++++++++++++++++++---
>  net/core/net-sysfs.c      |  99 ++++++++++++++++++++++++++
>  3 files changed, 242 insertions(+), 9 deletions(-)
>
> --
> 2.28.0.618.gf4bc123cb7-goog
>

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 13:48 ` [RFC PATCH net-next 0/6] implement kthread based napi poll Magnus Karlsson
@ 2020-09-25 17:15   ` Wei Wang
  2020-09-25 17:30     ` Eric Dumazet
  2020-09-25 18:16     ` Stephen Hemminger
  2020-09-25 19:06   ` Jakub Kicinski
  1 sibling, 2 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-25 17:15 UTC (permalink / raw)
  To: Magnus Karlsson
  Cc: David S . Miller, Network Development, Jakub Kicinski,
	Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel

On Fri, Sep 25, 2020 at 6:48 AM Magnus Karlsson
<magnus.karlsson@gmail.com> wrote:
>
> On Mon, Sep 14, 2020 at 7:26 PM Wei Wang <weiwan@google.com> wrote:
> >
> > The idea of moving the napi poll process out of softirq context to a
> > kernel thread based context is not new.
> > Paolo Abeni and Hannes Frederic Sowa has proposed patches to move napi
> > poll to kthread back in 2016. And Felix Fietkau has also proposed
> > patches of similar ideas to use workqueue to process napi poll just a
> > few weeks ago.
> >
> > The main reason we'd like to push forward with this idea is that the
> > scheduler has poor visibility into cpu cycles spent in softirq context,
> > and is not able to make optimal scheduling decisions of the user threads.
> > For example, we see in one of the application benchmark where network
> > load is high, the CPUs handling network softirqs has ~80% cpu util. And
> > user threads are still scheduled on those CPUs, despite other more idle
> > cpus available in the system. And we see very high tail latencies. In this
> > case, we have to explicitly pin away user threads from the CPUs handling
> > network softirqs to ensure good performance.
> > With napi poll moved to kthread, scheduler is in charge of scheduling both
> > the kthreads handling network load, and the user threads, and is able to
> > make better decisions. In the previous benchmark, if we do this and we
> > pin the kthreads processing napi poll to specific CPUs, scheduler is
> > able to schedule user threads away from these CPUs automatically.
> >
> > And the reason we prefer 1 kthread per napi, instead of 1 workqueue
> > entity per host, is that kthread is more configurable than workqueue,
> > and we could leverage existing tuning tools for threads, like taskset,
> > chrt, etc to tune scheduling class and cpu set, etc. Another reason is
> > if we eventually want to provide busy poll feature using kernel threads
> > for napi poll, kthread seems to be more suitable than workqueue.
> >
> > In this patch series, I revived Paolo and Hannes's patch in 2016 and
> > left them as the first 2 patches. Then there are changes proposed by
> > Felix, Jakub, Paolo and myself on top of those, with suggestions from
> > Eric Dumazet.
> >
> > In terms of performance, I ran tcp_rr tests with 1000 flows with
> > various request/response sizes, with RFS/RPS disabled, and compared
> > performance between softirq vs kthread. Host has 56 hyper threads and
> > 100Gbps nic.
> >
> >         req/resp   QPS   50%tile    90%tile    99%tile    99.9%tile
> > softirq   1B/1B   2.19M   284us       987us      1.1ms      1.56ms
> > kthread   1B/1B   2.14M   295us       987us      1.0ms      1.17ms
> >
> > softirq 5KB/5KB   1.31M   869us      1.06ms     1.28ms      2.38ms
> > kthread 5KB/5KB   1.32M   878us      1.06ms     1.26ms      1.66ms
> >
> > softirq 1MB/1MB  10.78K   84ms       166ms      234ms       294ms
> > kthread 1MB/1MB  10.83K   82ms       173ms      262ms       320ms
> >
> > I also ran one application benchmark where the user threads have more
> > work to do. We do see good amount of tail latency reductions with the
> > kthread model.
>
> I really like this RFC and would encourage you to submit it as a
> patch. Would love to see it make it into the kernel.
>

Thanks for the feedback! I am preparing an official patchset for this
and will send them out soon.

> I see the same positive effects as you when trying it out with AF_XDP
> sockets. Made some simple experiments where I sent 64-byte packets to
> a single AF_XDP socket. Have not managed to figure out how to do
> percentiles on my load generator, so this is going to be min, avg and
> max only. The application using the AF_XDP socket just performs a mac
> swap on the packet and sends it back to the load generator that then
> measures the round trip latency. The kthread is taskset to the same
> core as ksoftirqd would run on. So in each experiment, they always run
> on the same core id (which is not the same as the application).
>
> Rate 12 Mpps with 0% loss.
>               Latencies (us)         Delay Variation between packets
>           min    avg    max      avg   max
> sofirq  11.0  17.1   78.4      0.116  63.0
> kthread 11.2  17.1   35.0     0.116  20.9
>
> Rate ~58 Mpps (Line rate at 40 Gbit/s) with substantial loss
>               Latencies (us)         Delay Variation between packets
>           min    avg    max      avg   max
> softirq  87.6  194.9  282.6    0.062  25.9
> kthread  86.5  185.2  271.8    0.061  22.5
>
> For the last experiment, I also get 1.5% to 2% higher throughput with
> your kthread approach. Moreover, just from the per-second throughput
> printouts from my application, I can see that the kthread numbers are
> more stable. The softirq numbers can vary quite a lot between each
> second, around +-3%. But for the kthread approach, they are nice and
> stable. Have not examined why.
>

Thanks for sharing the results!

> One thing I noticed though, and I do not know if this is an issue, is
> that the switching between the two modes does not occur at high packet
> rates. I have to lower the packet rate to something that makes the
> core work less than 100% for it to switch between ksoftirqd to kthread
> and vice versa. They just seem too busy to switch at 100% load when
> changing the "threaded" sysfs variable.
>

I think the reason for this is when load is high, napi_poll() probably
always exhausts the predefined napi->weight. So it will keep
re-polling in the current context. The switch could only happen the
next time ___napi_schedule() is called.

> Thank you for working on this feature.
>
>
> /Magnus
>
>
> > Paolo Abeni (2):
> >   net: implement threaded-able napi poll loop support
> >   net: add sysfs attribute to control napi threaded mode
> > Felix Fietkau (1):
> >   net: extract napi poll functionality to __napi_poll()
> > Jakub Kicinski (1):
> >   net: modify kthread handler to use __napi_poll()
> > Paolo Abeni (1):
> >   net: process RPS/RFS work in kthread context
> > Wei Wang (1):
> >   net: improve napi threaded config
> >
> >  include/linux/netdevice.h |   6 ++
> >  net/core/dev.c            | 146 +++++++++++++++++++++++++++++++++++---
> >  net/core/net-sysfs.c      |  99 ++++++++++++++++++++++++++
> >  3 files changed, 242 insertions(+), 9 deletions(-)
> >
> > --
> > 2.28.0.618.gf4bc123cb7-goog
> >

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 17:15   ` Wei Wang
@ 2020-09-25 17:30     ` Eric Dumazet
  2020-09-25 18:16     ` Stephen Hemminger
  1 sibling, 0 replies; 32+ messages in thread
From: Eric Dumazet @ 2020-09-25 17:30 UTC (permalink / raw)
  To: Wei Wang, Magnus Karlsson
  Cc: David S . Miller, Network Development, Jakub Kicinski,
	Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel



On 9/25/20 7:15 PM, Wei Wang wrote:
> On Fri, Sep 25, 2020 at 6:48 AM Magnus Karlsson
> <magnus.karlsson@gmail.com> wrote:
>>
>> On Mon, Sep 14, 2020 at 7:26 PM Wei Wang <weiwan@google.com> wrote:
>>>
>>> The idea of moving the napi poll process out of softirq context to a
>>> kernel thread based context is not new.
>>> Paolo Abeni and Hannes Frederic Sowa has proposed patches to move napi
>>> poll to kthread back in 2016. And Felix Fietkau has also proposed
>>> patches of similar ideas to use workqueue to process napi poll just a
>>> few weeks ago.
>>>
>>> The main reason we'd like to push forward with this idea is that the
>>> scheduler has poor visibility into cpu cycles spent in softirq context,
>>> and is not able to make optimal scheduling decisions of the user threads.
>>> For example, we see in one of the application benchmark where network
>>> load is high, the CPUs handling network softirqs has ~80% cpu util. And
>>> user threads are still scheduled on those CPUs, despite other more idle
>>> cpus available in the system. And we see very high tail latencies. In this
>>> case, we have to explicitly pin away user threads from the CPUs handling
>>> network softirqs to ensure good performance.
>>> With napi poll moved to kthread, scheduler is in charge of scheduling both
>>> the kthreads handling network load, and the user threads, and is able to
>>> make better decisions. In the previous benchmark, if we do this and we
>>> pin the kthreads processing napi poll to specific CPUs, scheduler is
>>> able to schedule user threads away from these CPUs automatically.
>>>
>>> And the reason we prefer 1 kthread per napi, instead of 1 workqueue
>>> entity per host, is that kthread is more configurable than workqueue,
>>> and we could leverage existing tuning tools for threads, like taskset,
>>> chrt, etc to tune scheduling class and cpu set, etc. Another reason is
>>> if we eventually want to provide busy poll feature using kernel threads
>>> for napi poll, kthread seems to be more suitable than workqueue.
>>>
>>> In this patch series, I revived Paolo and Hannes's patch in 2016 and
>>> left them as the first 2 patches. Then there are changes proposed by
>>> Felix, Jakub, Paolo and myself on top of those, with suggestions from
>>> Eric Dumazet.
>>>
>>> In terms of performance, I ran tcp_rr tests with 1000 flows with
>>> various request/response sizes, with RFS/RPS disabled, and compared
>>> performance between softirq vs kthread. Host has 56 hyper threads and
>>> 100Gbps nic.
>>>
>>>         req/resp   QPS   50%tile    90%tile    99%tile    99.9%tile
>>> softirq   1B/1B   2.19M   284us       987us      1.1ms      1.56ms
>>> kthread   1B/1B   2.14M   295us       987us      1.0ms      1.17ms
>>>
>>> softirq 5KB/5KB   1.31M   869us      1.06ms     1.28ms      2.38ms
>>> kthread 5KB/5KB   1.32M   878us      1.06ms     1.26ms      1.66ms
>>>
>>> softirq 1MB/1MB  10.78K   84ms       166ms      234ms       294ms
>>> kthread 1MB/1MB  10.83K   82ms       173ms      262ms       320ms
>>>
>>> I also ran one application benchmark where the user threads have more
>>> work to do. We do see good amount of tail latency reductions with the
>>> kthread model.
>>
>> I really like this RFC and would encourage you to submit it as a
>> patch. Would love to see it make it into the kernel.
>>
> 
> Thanks for the feedback! I am preparing an official patchset for this
> and will send them out soon.
> 
>> I see the same positive effects as you when trying it out with AF_XDP
>> sockets. Made some simple experiments where I sent 64-byte packets to
>> a single AF_XDP socket. Have not managed to figure out how to do
>> percentiles on my load generator, so this is going to be min, avg and
>> max only. The application using the AF_XDP socket just performs a mac
>> swap on the packet and sends it back to the load generator that then
>> measures the round trip latency. The kthread is taskset to the same
>> core as ksoftirqd would run on. So in each experiment, they always run
>> on the same core id (which is not the same as the application).
>>
>> Rate 12 Mpps with 0% loss.
>>               Latencies (us)         Delay Variation between packets
>>           min    avg    max      avg   max
>> sofirq  11.0  17.1   78.4      0.116  63.0
>> kthread 11.2  17.1   35.0     0.116  20.9
>>
>> Rate ~58 Mpps (Line rate at 40 Gbit/s) with substantial loss
>>               Latencies (us)         Delay Variation between packets
>>           min    avg    max      avg   max
>> softirq  87.6  194.9  282.6    0.062  25.9
>> kthread  86.5  185.2  271.8    0.061  22.5
>>
>> For the last experiment, I also get 1.5% to 2% higher throughput with
>> your kthread approach. Moreover, just from the per-second throughput
>> printouts from my application, I can see that the kthread numbers are
>> more stable. The softirq numbers can vary quite a lot between each
>> second, around +-3%. But for the kthread approach, they are nice and
>> stable. Have not examined why.
>>
> 
> Thanks for sharing the results!
> 
>> One thing I noticed though, and I do not know if this is an issue, is
>> that the switching between the two modes does not occur at high packet
>> rates. I have to lower the packet rate to something that makes the
>> core work less than 100% for it to switch between ksoftirqd to kthread
>> and vice versa. They just seem too busy to switch at 100% load when
>> changing the "threaded" sysfs variable.
>>
> 
> I think the reason for this is when load is high, napi_poll() probably
> always exhausts the predefined napi->weight. So it will keep
> re-polling in the current context. The switch could only happen the
> next time ___napi_schedule() is called.

A similar problem happens when /proc/irq/{..}/smp_affinity is changed.

Few drivers actually detect the affinity has changed (and does not include
current cpu), and force an napi poll complete/exit, so that a new hardware
interrupt is allowed and routed to another cpu.

Presumably the softirq -> kthread transition could be enforced if really needed.

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 17:15   ` Wei Wang
  2020-09-25 17:30     ` Eric Dumazet
@ 2020-09-25 18:16     ` Stephen Hemminger
  2020-09-25 18:23       ` Eric Dumazet
  1 sibling, 1 reply; 32+ messages in thread
From: Stephen Hemminger @ 2020-09-25 18:16 UTC (permalink / raw)
  To: Wei Wang
  Cc: Magnus Karlsson, David S . Miller, Network Development,
	Jakub Kicinski, Eric Dumazet, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Björn Töpel

On Fri, 25 Sep 2020 10:15:25 -0700
Wei Wang <weiwan@google.com> wrote:

> > > In terms of performance, I ran tcp_rr tests with 1000 flows with
> > > various request/response sizes, with RFS/RPS disabled, and compared
> > > performance between softirq vs kthread. Host has 56 hyper threads and
> > > 100Gbps nic.

It would be good to similar tests on othere hardware. Not everyone has
server class hardware. There are people running web servers on untuned
servers over 10 years old; this may cause a regression there.

Not to mention the slower CPU's in embedded systems. How would this
impact OpenWrt or Android?

Another potential problem is that if you run real time (SCH_FIFO)
threads they have higher priority than kthread. So for that use
case, moving networking to kthread would break them.

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 18:16     ` Stephen Hemminger
@ 2020-09-25 18:23       ` Eric Dumazet
  2020-09-25 19:00         ` Stephen Hemminger
  0 siblings, 1 reply; 32+ messages in thread
From: Eric Dumazet @ 2020-09-25 18:23 UTC (permalink / raw)
  To: Stephen Hemminger
  Cc: Wei Wang, Magnus Karlsson, David S . Miller, Network Development,
	Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel

On Fri, Sep 25, 2020 at 8:16 PM Stephen Hemminger
<stephen@networkplumber.org> wrote:
>
> On Fri, 25 Sep 2020 10:15:25 -0700
> Wei Wang <weiwan@google.com> wrote:
>
> > > > In terms of performance, I ran tcp_rr tests with 1000 flows with
> > > > various request/response sizes, with RFS/RPS disabled, and compared
> > > > performance between softirq vs kthread. Host has 56 hyper threads and
> > > > 100Gbps nic.
>
> It would be good to similar tests on othere hardware. Not everyone has
> server class hardware. There are people running web servers on untuned
> servers over 10 years old; this may cause a regression there.
>
> Not to mention the slower CPU's in embedded systems. How would this
> impact OpenWrt or Android?

Most probably you won't notice a significant difference.

Switching to a kthread is quite cheap, since you have no MMU games to play with.

>
> Another potential problem is that if you run real time (SCH_FIFO)
> threads they have higher priority than kthread. So for that use
> case, moving networking to kthread would break them.

Sure, playing with FIFO threads is dangerous.

Note that our plan is still to have softirqs by default.

If an admin chose to use kthreads, it is its choice, not ours.

This is also why I very much prefer the kthread approach to the work
queue, since the work queue could not be fine tuned.

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 18:23       ` Eric Dumazet
@ 2020-09-25 19:00         ` Stephen Hemminger
  0 siblings, 0 replies; 32+ messages in thread
From: Stephen Hemminger @ 2020-09-25 19:00 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: Wei Wang, Magnus Karlsson, David S . Miller, Network Development,
	Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel

On Fri, 25 Sep 2020 20:23:37 +0200
Eric Dumazet <edumazet@google.com> wrote:

> On Fri, Sep 25, 2020 at 8:16 PM Stephen Hemminger
> <stephen@networkplumber.org> wrote:
> >
> > On Fri, 25 Sep 2020 10:15:25 -0700
> > Wei Wang <weiwan@google.com> wrote:
> >  
> > > > > In terms of performance, I ran tcp_rr tests with 1000 flows with
> > > > > various request/response sizes, with RFS/RPS disabled, and compared
> > > > > performance between softirq vs kthread. Host has 56 hyper threads and
> > > > > 100Gbps nic.  
> >
> > It would be good to similar tests on othere hardware. Not everyone has
> > server class hardware. There are people running web servers on untuned
> > servers over 10 years old; this may cause a regression there.
> >
> > Not to mention the slower CPU's in embedded systems. How would this
> > impact OpenWrt or Android?  
> 
> Most probably you won't notice a significant difference.
> 
> Switching to a kthread is quite cheap, since you have no MMU games to play with.

That makes sense, and in the past when doing stress tests the napi
work was mostly on the kthread already.

> >
> > Another potential problem is that if you run real time (SCH_FIFO)
> > threads they have higher priority than kthread. So for that use
> > case, moving networking to kthread would break them.  
> 
> Sure, playing with FIFO threads is dangerous.
> 
> Note that our plan is still to have softirqs by default.
> 
> If an admin chose to use kthreads, it is its choice, not ours.
> 
> This is also why I very much prefer the kthread approach to the work
> queue, since the work queue could not be fine tuned.

Agree with you, best to keep this as opt-in.

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 13:48 ` [RFC PATCH net-next 0/6] implement kthread based napi poll Magnus Karlsson
  2020-09-25 17:15   ` Wei Wang
@ 2020-09-25 19:06   ` Jakub Kicinski
  2020-09-28 14:07     ` Magnus Karlsson
  1 sibling, 1 reply; 32+ messages in thread
From: Jakub Kicinski @ 2020-09-25 19:06 UTC (permalink / raw)
  To: Magnus Karlsson
  Cc: Wei Wang, David S . Miller, Network Development, Eric Dumazet,
	Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel

On Fri, 25 Sep 2020 15:48:35 +0200 Magnus Karlsson wrote:
> I really like this RFC and would encourage you to submit it as a
> patch. Would love to see it make it into the kernel.
> 
> I see the same positive effects as you when trying it out with AF_XDP
> sockets. Made some simple experiments where I sent 64-byte packets to
> a single AF_XDP socket. Have not managed to figure out how to do
> percentiles on my load generator, so this is going to be min, avg and
> max only. The application using the AF_XDP socket just performs a mac
> swap on the packet and sends it back to the load generator that then
> measures the round trip latency. The kthread is taskset to the same
> core as ksoftirqd would run on. So in each experiment, they always run
> on the same core id (which is not the same as the application).
> 
> Rate 12 Mpps with 0% loss.
>               Latencies (us)         Delay Variation between packets
>           min    avg    max      avg   max
> sofirq  11.0  17.1   78.4      0.116  63.0
> kthread 11.2  17.1   35.0     0.116  20.9
> 
> Rate ~58 Mpps (Line rate at 40 Gbit/s) with substantial loss
>               Latencies (us)         Delay Variation between packets
>           min    avg    max      avg   max
> softirq  87.6  194.9  282.6    0.062  25.9
> kthread  86.5  185.2  271.8    0.061  22.5
> 
> For the last experiment, I also get 1.5% to 2% higher throughput with
> your kthread approach. Moreover, just from the per-second throughput
> printouts from my application, I can see that the kthread numbers are
> more stable. The softirq numbers can vary quite a lot between each
> second, around +-3%. But for the kthread approach, they are nice and
> stable. Have not examined why.

Sure, it's better than status quo for AF_XDP but it's going to be far
inferior to well implemented busy polling.

We already discussed the potential scheme with Bjorn, since you prompted
me again, let me shoot some code from the hip at ya:

diff --git a/net/core/dev.c b/net/core/dev.c
index 74ce8b253ed6..8dbdfaeb0183 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -6668,6 +6668,7 @@ static struct napi_struct *napi_by_id(unsigned int napi_id)
 
 static void busy_poll_stop(struct napi_struct *napi, void *have_poll_lock)
 {
+       unsigned long to;
        int rc;
 
        /* Busy polling means there is a high chance device driver hard irq
@@ -6682,6 +6683,13 @@ static void busy_poll_stop(struct napi_struct *napi, void *have_poll_lock)
        clear_bit(NAPI_STATE_MISSED, &napi->state);
        clear_bit(NAPI_STATE_IN_BUSY_POLL, &napi->state);
 
+       if (READ_ONCE(napi->dev->napi_defer_hard_irqs)) {
+               netpoll_poll_unlock(have_poll_lock);
+               to = ns_to_ktime(READ_ONCE(napi->dev->gro_flush_timeout));
+               hrtimer_start(&n->timer, to, HRTIMER_MODE_REL_PINNED);
+               return;
+       }
+
        local_bh_disable();
 
        /* All we really want here is to re-enable device interrupts.


With basic busy polling implemented for AF_XDP this is all** you need
to make busy polling work very well.

** once bugs are fixed :D I haven't even compiled this

Eric & co. already implemented hard IRQ deferral. All we need to do is
push the timer away when application picks up frames. I think.

Please, no loose threads for AF_XDP apps (or other busy polling apps).
Let the application burn 100% of the core :(

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

* Re: [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support
  2020-09-14 17:24 ` [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support Wei Wang
@ 2020-09-25 19:45   ` Hannes Frederic Sowa
  2020-09-25 23:50     ` Wei Wang
  0 siblings, 1 reply; 32+ messages in thread
From: Hannes Frederic Sowa @ 2020-09-25 19:45 UTC (permalink / raw)
  To: Wei Wang, David Miller, netdev
  Cc: Jakub Kicinski, Eric Dumazet, Paolo Abeni, Felix Fietkau

Hello,

Happy to see this work being resurrected (in case it is useful). :)

On Mon, Sep 14, 2020, at 19:24, Wei Wang wrote:
>
> [...]
>
> +static void napi_thread_start(struct napi_struct *n)
> +{
> +	if (test_bit(NAPI_STATE_THREADED, &n->state) && !n->thread)
> +		n->thread = kthread_create(napi_threaded_poll, n, "%s-%d",
> +					   n->dev->name, n->napi_id);
> +}
> +

The format string is only based on variable strings. To ease a quick
grep for napi threads with ps I would propose to use "napi-%s-%d" or
something alike to distinguish all threads created that way.

Some other comments and questions:

Back then my plan was to get this somewhat integrated with the
`threadirqs` kernel boot option because triggering the softirq from
threaded context (if this option is set) seemed wrong to me. Maybe in
theory the existing interrupt thread could already be used in this case.
This would also allow for fine tuning the corresponding threads as in
this patch series.

Maybe the whole approach of threaded irqs plus the already existing
infrastructure could also be used for this series if it wouldn't be an
all or nothing opt-in based on the kernel cmd line parameter? napi would
then be able to just poll directly inline in the interrupt thread.

The difference for those kthreads and the extra threads created here
would be that fifo scheduling policy is set by default and they seem to
automatically get steered to the appropriate CPUs via the IRQTF_AFFINITY
mechanism. Maybe this approach is useful here as well?

I hadn't had a look at the code for a while thus my memories might be
wrong here.

Thanks,
Hannes

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

* Re: [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support
  2020-09-25 19:45   ` Hannes Frederic Sowa
@ 2020-09-25 23:50     ` Wei Wang
  2020-09-26 14:22       ` Hannes Frederic Sowa
  0 siblings, 1 reply; 32+ messages in thread
From: Wei Wang @ 2020-09-25 23:50 UTC (permalink / raw)
  To: Hannes Frederic Sowa
  Cc: David Miller, Linux Kernel Network Developers, Jakub Kicinski,
	Eric Dumazet, Paolo Abeni, Felix Fietkau

On Fri, Sep 25, 2020 at 12:46 PM Hannes Frederic Sowa
<hannes@stressinduktion.org> wrote:
>
> Hello,
>
> Happy to see this work being resurrected (in case it is useful). :)
>
> On Mon, Sep 14, 2020, at 19:24, Wei Wang wrote:
> >
> > [...]
> >
> > +static void napi_thread_start(struct napi_struct *n)
> > +{
> > +     if (test_bit(NAPI_STATE_THREADED, &n->state) && !n->thread)
> > +             n->thread = kthread_create(napi_threaded_poll, n, "%s-%d",
> > +                                        n->dev->name, n->napi_id);
> > +}
> > +
>
> The format string is only based on variable strings. To ease a quick
> grep for napi threads with ps I would propose to use "napi-%s-%d" or
> something alike to distinguish all threads created that way.
>

Ack. Will add this in the next version.

> Some other comments and questions:
>
> Back then my plan was to get this somewhat integrated with the
> `threadirqs` kernel boot option because triggering the softirq from
> threaded context (if this option is set) seemed wrong to me. Maybe in
> theory the existing interrupt thread could already be used in this case.
> This would also allow for fine tuning the corresponding threads as in
> this patch series.
>
> Maybe the whole approach of threaded irqs plus the already existing
> infrastructure could also be used for this series if it wouldn't be an
> all or nothing opt-in based on the kernel cmd line parameter? napi would
> then be able to just poll directly inline in the interrupt thread.
>

I took a look at the current "threadirqs" implementation. From my
understanding, the kthread used there is to handle irq from the
driver, and needs driver-specific thread_fn to be used. It is not as
generic as in the napi layer where a common napi_poll() related
function could be used as the thread handler. Or did I misunderstand
your point?


> The difference for those kthreads and the extra threads created here
> would be that fifo scheduling policy is set by default and they seem to
> automatically get steered to the appropriate CPUs via the IRQTF_AFFINITY
> mechanism. Maybe this approach is useful here as well?
>
> I hadn't had a look at the code for a while thus my memories might be
> wrong here.

Yes. Using a higher priority thread policy and doing pinning could be
beneficial in certain workloads. But I think this should be left to
the user/admin to do the tuning accordingly.

>
> Thanks,
> Hannes

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

* Re: [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support
  2020-09-25 23:50     ` Wei Wang
@ 2020-09-26 14:22       ` Hannes Frederic Sowa
  2020-09-28  8:45         ` Paolo Abeni
  0 siblings, 1 reply; 32+ messages in thread
From: Hannes Frederic Sowa @ 2020-09-26 14:22 UTC (permalink / raw)
  To: Wei Wang
  Cc: David Miller, Linux Kernel Network Developers, Jakub Kicinski,
	Eric Dumazet, Paolo Abeni, Felix Fietkau

Hello,

On Sat, Sep 26, 2020, at 01:50, Wei Wang wrote:
> On Fri, Sep 25, 2020 at 12:46 PM Hannes Frederic Sowa
> <hannes@stressinduktion.org> wrote:
> > The format string is only based on variable strings. To ease a quick
> > grep for napi threads with ps I would propose to use "napi-%s-%d" or
> > something alike to distinguish all threads created that way.
> >
> 
> Ack. Will add this in the next version.

I think the convention would be to use "napi/%s-%d".

> > Some other comments and questions:
> >
> > Back then my plan was to get this somewhat integrated with the
> > `threadirqs` kernel boot option because triggering the softirq from
> > threaded context (if this option is set) seemed wrong to me. Maybe in
> > theory the existing interrupt thread could already be used in this case.
> > This would also allow for fine tuning the corresponding threads as in
> > this patch series.
> >
> > Maybe the whole approach of threaded irqs plus the already existing
> > infrastructure could also be used for this series if it wouldn't be an
> > all or nothing opt-in based on the kernel cmd line parameter? napi would
> > then be able to just poll directly inline in the interrupt thread.
> >
> 
> I took a look at the current "threadirqs" implementation. From my
> understanding, the kthread used there is to handle irq from the
> driver, and needs driver-specific thread_fn to be used. It is not as
> generic as in the napi layer where a common napi_poll() related
> function could be used as the thread handler. Or did I misunderstand
> your point?

Based on my memories: We had napi_schedule & co being invoked inside
the threads without touching any driver code when we specified
threadirqs. But this would need a double check. The idea of the napi
threads came out of the observation that the threaded irq would merely
kick softirq net-rx (thread). Maybe Paolo has better memories and what
we tried back then?

Thus the idea is to add a flag NAPI_INLINE, which could run
the napi loop from within the threaded irq handler directly and thus
just build on top of the current irq management framework.

This would require to make the single-shot kernel boot parameter
configurable per device (and probably during run-time). I have
absolutely no idea if that's feasible and how complicated that is and
thus might be a dead end.

> > The difference for those kthreads and the extra threads created here
> > would be that fifo scheduling policy is set by default and they seem to
> > automatically get steered to the appropriate CPUs via the IRQTF_AFFINITY
> > mechanism. Maybe this approach is useful here as well?
> >
> > I hadn't had a look at the code for a while thus my memories might be
> > wrong here.
> 
> Yes. Using a higher priority thread policy and doing pinning could be
> beneficial in certain workloads. But I think this should be left to
> the user/admin to do the tuning accordingly.

I agree in general, but if the common case necessarily requires to set
various scheduling options it might still be worthwhile? Administrators
are free to change them later anyway. It might be the same argument that
added the default scheduling parameters to the thread irqs in the first
place, but I can be wrong here and they got added because of
correctness.

Bye,
Hannes

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

* Re: [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support
  2020-09-26 14:22       ` Hannes Frederic Sowa
@ 2020-09-28  8:45         ` Paolo Abeni
  2020-09-28 18:13           ` Wei Wang
  0 siblings, 1 reply; 32+ messages in thread
From: Paolo Abeni @ 2020-09-28  8:45 UTC (permalink / raw)
  To: Hannes Frederic Sowa, Wei Wang
  Cc: David Miller, Linux Kernel Network Developers, Jakub Kicinski,
	Eric Dumazet, Felix Fietkau

Hello,

On Sat, 2020-09-26 at 16:22 +0200, Hannes Frederic Sowa wrote:
> On Sat, Sep 26, 2020, at 01:50, Wei Wang wrote:
> > I took a look at the current "threadirqs" implementation. From my
> > understanding, the kthread used there is to handle irq from the
> > driver, and needs driver-specific thread_fn to be used. It is not
> > as
> > generic as in the napi layer where a common napi_poll() related
> > function could be used as the thread handler. Or did I
> > misunderstand
> > your point?
> 
> Based on my memories: We had napi_schedule & co being invoked inside
> the threads 

I just looked at the code - I really forgot most details. The above is
correct...

> without touching any driver code when we specified
> threadirqs. But this would need a double check. 

... but still that code needed some per device driver modification: the
irq subsystem handled the switch to/from threaded mode, and needed some
callback, provided from the device driver, to notify the network code
about the change (specifically, to mark the threaded status inside the
relevant napi struct).

Cheers,

Paolo


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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-25 19:06   ` Jakub Kicinski
@ 2020-09-28 14:07     ` Magnus Karlsson
  0 siblings, 0 replies; 32+ messages in thread
From: Magnus Karlsson @ 2020-09-28 14:07 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: Wei Wang, David S . Miller, Network Development, Eric Dumazet,
	Paolo Abeni, Hannes Frederic Sowa, Felix Fietkau,
	Björn Töpel

On Fri, Sep 25, 2020 at 9:06 PM Jakub Kicinski <kuba@kernel.org> wrote:
>
> On Fri, 25 Sep 2020 15:48:35 +0200 Magnus Karlsson wrote:
> > I really like this RFC and would encourage you to submit it as a
> > patch. Would love to see it make it into the kernel.
> >
> > I see the same positive effects as you when trying it out with AF_XDP
> > sockets. Made some simple experiments where I sent 64-byte packets to
> > a single AF_XDP socket. Have not managed to figure out how to do
> > percentiles on my load generator, so this is going to be min, avg and
> > max only. The application using the AF_XDP socket just performs a mac
> > swap on the packet and sends it back to the load generator that then
> > measures the round trip latency. The kthread is taskset to the same
> > core as ksoftirqd would run on. So in each experiment, they always run
> > on the same core id (which is not the same as the application).
> >
> > Rate 12 Mpps with 0% loss.
> >               Latencies (us)         Delay Variation between packets
> >           min    avg    max      avg   max
> > sofirq  11.0  17.1   78.4      0.116  63.0
> > kthread 11.2  17.1   35.0     0.116  20.9
> >
> > Rate ~58 Mpps (Line rate at 40 Gbit/s) with substantial loss
> >               Latencies (us)         Delay Variation between packets
> >           min    avg    max      avg   max
> > softirq  87.6  194.9  282.6    0.062  25.9
> > kthread  86.5  185.2  271.8    0.061  22.5
> >
> > For the last experiment, I also get 1.5% to 2% higher throughput with
> > your kthread approach. Moreover, just from the per-second throughput
> > printouts from my application, I can see that the kthread numbers are
> > more stable. The softirq numbers can vary quite a lot between each
> > second, around +-3%. But for the kthread approach, they are nice and
> > stable. Have not examined why.
>
> Sure, it's better than status quo for AF_XDP but it's going to be far
> inferior to well implemented busy polling.

Agree completely. Björn is looking into this at the moment, so I will
let him comment on it and post some patches.

> We already discussed the potential scheme with Bjorn, since you prompted
> me again, let me shoot some code from the hip at ya:
>
> diff --git a/net/core/dev.c b/net/core/dev.c
> index 74ce8b253ed6..8dbdfaeb0183 100644
> --- a/net/core/dev.c
> +++ b/net/core/dev.c
> @@ -6668,6 +6668,7 @@ static struct napi_struct *napi_by_id(unsigned int napi_id)
>
>  static void busy_poll_stop(struct napi_struct *napi, void *have_poll_lock)
>  {
> +       unsigned long to;
>         int rc;
>
>         /* Busy polling means there is a high chance device driver hard irq
> @@ -6682,6 +6683,13 @@ static void busy_poll_stop(struct napi_struct *napi, void *have_poll_lock)
>         clear_bit(NAPI_STATE_MISSED, &napi->state);
>         clear_bit(NAPI_STATE_IN_BUSY_POLL, &napi->state);
>
> +       if (READ_ONCE(napi->dev->napi_defer_hard_irqs)) {
> +               netpoll_poll_unlock(have_poll_lock);
> +               to = ns_to_ktime(READ_ONCE(napi->dev->gro_flush_timeout));
> +               hrtimer_start(&n->timer, to, HRTIMER_MODE_REL_PINNED);
> +               return;
> +       }
> +
>         local_bh_disable();
>
>         /* All we really want here is to re-enable device interrupts.
>
>
> With basic busy polling implemented for AF_XDP this is all** you need
> to make busy polling work very well.
>
> ** once bugs are fixed :D I haven't even compiled this
>
> Eric & co. already implemented hard IRQ deferral. All we need to do is
> push the timer away when application picks up frames. I think.
>
> Please, no loose threads for AF_XDP apps (or other busy polling apps).
> Let the application burn 100% of the core :(

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
                   ` (6 preceding siblings ...)
  2020-09-25 13:48 ` [RFC PATCH net-next 0/6] implement kthread based napi poll Magnus Karlsson
@ 2020-09-28 17:43 ` Eric Dumazet
  2020-09-28 18:15   ` Wei Wang
  2020-09-29 19:19   ` Jakub Kicinski
  7 siblings, 2 replies; 32+ messages in thread
From: Eric Dumazet @ 2020-09-28 17:43 UTC (permalink / raw)
  To: Wei Wang
  Cc: David S . Miller, netdev, Jakub Kicinski, Paolo Abeni,
	Hannes Frederic Sowa, Felix Fietkau

On Mon, Sep 14, 2020 at 7:26 PM Wei Wang <weiwan@google.com> wrote:
>
> The idea of moving the napi poll process out of softirq context to a
> kernel thread based context is not new.
> Paolo Abeni and Hannes Frederic Sowa has proposed patches to move napi
> poll to kthread back in 2016. And Felix Fietkau has also proposed
> patches of similar ideas to use workqueue to process napi poll just a
> few weeks ago.
>
> The main reason we'd like to push forward with this idea is that the
> scheduler has poor visibility into cpu cycles spent in softirq context,
> and is not able to make optimal scheduling decisions of the user threads.
> For example, we see in one of the application benchmark where network
> load is high, the CPUs handling network softirqs has ~80% cpu util. And
> user threads are still scheduled on those CPUs, despite other more idle
> cpus available in the system. And we see very high tail latencies. In this
> case, we have to explicitly pin away user threads from the CPUs handling
> network softirqs to ensure good performance.
> With napi poll moved to kthread, scheduler is in charge of scheduling both
> the kthreads handling network load, and the user threads, and is able to
> make better decisions. In the previous benchmark, if we do this and we
> pin the kthreads processing napi poll to specific CPUs, scheduler is
> able to schedule user threads away from these CPUs automatically.
>
> And the reason we prefer 1 kthread per napi, instead of 1 workqueue
> entity per host, is that kthread is more configurable than workqueue,
> and we could leverage existing tuning tools for threads, like taskset,
> chrt, etc to tune scheduling class and cpu set, etc. Another reason is
> if we eventually want to provide busy poll feature using kernel threads
> for napi poll, kthread seems to be more suitable than workqueue.
>
> In this patch series, I revived Paolo and Hannes's patch in 2016 and
> left them as the first 2 patches. Then there are changes proposed by
> Felix, Jakub, Paolo and myself on top of those, with suggestions from
> Eric Dumazet.
>
> In terms of performance, I ran tcp_rr tests with 1000 flows with
> various request/response sizes, with RFS/RPS disabled, and compared
> performance between softirq vs kthread. Host has 56 hyper threads and
> 100Gbps nic.
>
>         req/resp   QPS   50%tile    90%tile    99%tile    99.9%tile
> softirq   1B/1B   2.19M   284us       987us      1.1ms      1.56ms
> kthread   1B/1B   2.14M   295us       987us      1.0ms      1.17ms
>
> softirq 5KB/5KB   1.31M   869us      1.06ms     1.28ms      2.38ms
> kthread 5KB/5KB   1.32M   878us      1.06ms     1.26ms      1.66ms
>
> softirq 1MB/1MB  10.78K   84ms       166ms      234ms       294ms
> kthread 1MB/1MB  10.83K   82ms       173ms      262ms       320ms
>
> I also ran one application benchmark where the user threads have more
> work to do. We do see good amount of tail latency reductions with the
> kthread model.



Wei, this is a very nice work.

Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.

Thanks !

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

* Re: [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support
  2020-09-28  8:45         ` Paolo Abeni
@ 2020-09-28 18:13           ` Wei Wang
  0 siblings, 0 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-28 18:13 UTC (permalink / raw)
  To: Paolo Abeni
  Cc: Hannes Frederic Sowa, David Miller,
	Linux Kernel Network Developers, Jakub Kicinski, Eric Dumazet,
	Felix Fietkau

On Mon, Sep 28, 2020 at 1:45 AM Paolo Abeni <pabeni@redhat.com> wrote:
>
> Hello,
>
> On Sat, 2020-09-26 at 16:22 +0200, Hannes Frederic Sowa wrote:
> > On Sat, Sep 26, 2020, at 01:50, Wei Wang wrote:
> > > I took a look at the current "threadirqs" implementation. From my
> > > understanding, the kthread used there is to handle irq from the
> > > driver, and needs driver-specific thread_fn to be used. It is not
> > > as
> > > generic as in the napi layer where a common napi_poll() related
> > > function could be used as the thread handler. Or did I
> > > misunderstand
> > > your point?
> >
> > Based on my memories: We had napi_schedule & co being invoked inside
> > the threads
>
> I just looked at the code - I really forgot most details. The above is
> correct...
>
> > without touching any driver code when we specified
> > threadirqs. But this would need a double check.
>
> ... but still that code needed some per device driver modification: the
> irq subsystem handled the switch to/from threaded mode, and needed some
> callback, provided from the device driver, to notify the network code
> about the change (specifically, to mark the threaded status inside the
> relevant napi struct).

Thanks for the clarification. This corresponds with my understanding as well.

>
> Cheers,
>
> Paolo
>

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-28 17:43 ` Eric Dumazet
@ 2020-09-28 18:15   ` Wei Wang
  2020-09-29 19:19   ` Jakub Kicinski
  1 sibling, 0 replies; 32+ messages in thread
From: Wei Wang @ 2020-09-28 18:15 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: David S . Miller, netdev, Jakub Kicinski, Paolo Abeni,
	Hannes Frederic Sowa, Felix Fietkau

On Mon, Sep 28, 2020 at 10:43 AM Eric Dumazet <edumazet@google.com> wrote:
>
> On Mon, Sep 14, 2020 at 7:26 PM Wei Wang <weiwan@google.com> wrote:
> >
> > The idea of moving the napi poll process out of softirq context to a
> > kernel thread based context is not new.
> > Paolo Abeni and Hannes Frederic Sowa has proposed patches to move napi
> > poll to kthread back in 2016. And Felix Fietkau has also proposed
> > patches of similar ideas to use workqueue to process napi poll just a
> > few weeks ago.
> >
> > The main reason we'd like to push forward with this idea is that the
> > scheduler has poor visibility into cpu cycles spent in softirq context,
> > and is not able to make optimal scheduling decisions of the user threads.
> > For example, we see in one of the application benchmark where network
> > load is high, the CPUs handling network softirqs has ~80% cpu util. And
> > user threads are still scheduled on those CPUs, despite other more idle
> > cpus available in the system. And we see very high tail latencies. In this
> > case, we have to explicitly pin away user threads from the CPUs handling
> > network softirqs to ensure good performance.
> > With napi poll moved to kthread, scheduler is in charge of scheduling both
> > the kthreads handling network load, and the user threads, and is able to
> > make better decisions. In the previous benchmark, if we do this and we
> > pin the kthreads processing napi poll to specific CPUs, scheduler is
> > able to schedule user threads away from these CPUs automatically.
> >
> > And the reason we prefer 1 kthread per napi, instead of 1 workqueue
> > entity per host, is that kthread is more configurable than workqueue,
> > and we could leverage existing tuning tools for threads, like taskset,
> > chrt, etc to tune scheduling class and cpu set, etc. Another reason is
> > if we eventually want to provide busy poll feature using kernel threads
> > for napi poll, kthread seems to be more suitable than workqueue.
> >
> > In this patch series, I revived Paolo and Hannes's patch in 2016 and
> > left them as the first 2 patches. Then there are changes proposed by
> > Felix, Jakub, Paolo and myself on top of those, with suggestions from
> > Eric Dumazet.
> >
> > In terms of performance, I ran tcp_rr tests with 1000 flows with
> > various request/response sizes, with RFS/RPS disabled, and compared
> > performance between softirq vs kthread. Host has 56 hyper threads and
> > 100Gbps nic.
> >
> >         req/resp   QPS   50%tile    90%tile    99%tile    99.9%tile
> > softirq   1B/1B   2.19M   284us       987us      1.1ms      1.56ms
> > kthread   1B/1B   2.14M   295us       987us      1.0ms      1.17ms
> >
> > softirq 5KB/5KB   1.31M   869us      1.06ms     1.28ms      2.38ms
> > kthread 5KB/5KB   1.32M   878us      1.06ms     1.26ms      1.66ms
> >
> > softirq 1MB/1MB  10.78K   84ms       166ms      234ms       294ms
> > kthread 1MB/1MB  10.83K   82ms       173ms      262ms       320ms
> >
> > I also ran one application benchmark where the user threads have more
> > work to do. We do see good amount of tail latency reductions with the
> > kthread model.
>
>
>
> Wei, this is a very nice work.
>
> Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.
>
> Thanks !

Thank you Eric! Will prepare the official patch series and send it out soon.

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-28 17:43 ` Eric Dumazet
  2020-09-28 18:15   ` Wei Wang
@ 2020-09-29 19:19   ` Jakub Kicinski
  2020-09-29 20:16     ` Wei Wang
  1 sibling, 1 reply; 32+ messages in thread
From: Jakub Kicinski @ 2020-09-29 19:19 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: Wei Wang, David S . Miller, netdev, Paolo Abeni,
	Hannes Frederic Sowa, Felix Fietkau

On Mon, 28 Sep 2020 19:43:36 +0200 Eric Dumazet wrote:
> Wei, this is a very nice work.
> 
> Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.

The problem is for the application I'm testing with this implementation
is significantly slower (in terms of RPS) than Felix's code:

              |        L  A  T  E  N  C  Y       |  App   |     C P U     |
       |  RPS |   AVG  |  P50  |   P99  |   P999 | Overld |  busy |  PSI  |
thread | 1.1% | -15.6% | -0.3% | -42.5% |  -8.1% | -83.4% | -2.3% | 60.6% |
work q | 4.3% | -13.1% |  0.1% | -44.4% |  -1.1% |   2.3% | -1.2% | 90.1% |
TAPI   | 4.4% | -17.1% | -1.4% | -43.8% | -11.0% | -60.2% | -2.3% | 46.7% |

thread is this code, "work q" is Felix's code, TAPI is my hacks.

The numbers are comparing performance to normal NAPI.

In all cases (but not the baseline) I configured timer-based polling
(defer_hard_irqs), with around 100us timeout. Without deferring hard
IRQs threaded NAPI is actually slower for this app. Also I'm not
modifying niceness, this again causes application performance
regression here.

1 NUMA node. 18 NAPI instances each is around 25% of a single CPU.

I was initially hoping that TAPI would fit nicely as an extension 
of this code, but I don't think that will be the case.

Are there any assumptions you're making about the configuration that 
I should try to replicate?

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-29 19:19   ` Jakub Kicinski
@ 2020-09-29 20:16     ` Wei Wang
  2020-09-29 21:48       ` Jakub Kicinski
  0 siblings, 1 reply; 32+ messages in thread
From: Wei Wang @ 2020-09-29 20:16 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: Eric Dumazet, David S . Miller, netdev, Paolo Abeni,
	Hannes Frederic Sowa, Felix Fietkau

On Tue, Sep 29, 2020 at 12:19 PM Jakub Kicinski <kuba@kernel.org> wrote:
>
> On Mon, 28 Sep 2020 19:43:36 +0200 Eric Dumazet wrote:
> > Wei, this is a very nice work.
> >
> > Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.
>
> The problem is for the application I'm testing with this implementation
> is significantly slower (in terms of RPS) than Felix's code:
>
>               |        L  A  T  E  N  C  Y       |  App   |     C P U     |
>        |  RPS |   AVG  |  P50  |   P99  |   P999 | Overld |  busy |  PSI  |
> thread | 1.1% | -15.6% | -0.3% | -42.5% |  -8.1% | -83.4% | -2.3% | 60.6% |
> work q | 4.3% | -13.1% |  0.1% | -44.4% |  -1.1% |   2.3% | -1.2% | 90.1% |
> TAPI   | 4.4% | -17.1% | -1.4% | -43.8% | -11.0% | -60.2% | -2.3% | 46.7% |
>
> thread is this code, "work q" is Felix's code, TAPI is my hacks.
>
> The numbers are comparing performance to normal NAPI.
>
> In all cases (but not the baseline) I configured timer-based polling
> (defer_hard_irqs), with around 100us timeout. Without deferring hard
> IRQs threaded NAPI is actually slower for this app. Also I'm not
> modifying niceness, this again causes application performance
> regression here.
>

If I remember correctly, Felix's workqueue code uses HIGHPRIO flag
which by default uses -20 as the nice value for the workqueue threads.
But the kthread implementation leaves nice level as 20 by default.
This could be 1 difference.
I am not sure what the benchmark is doing, but one thing to try is to
limit the CPUs that run the kthreads to a smaller # of CPUs. This
could bring up the kernel cpu usage to a higher %, e.g. > 80%, so the
scheduler is less likely to schedule user threads on these CPUs, thus
providing isolations between kthreads and the user threads, and
reducing the scheduling overhead. This could help if the throughput
drop is caused by higher scheduling latency for the user threads.
Another thing to try is to raise the scheduling class of the kthread
from SCHED_OTHER to SCHED_FIFO. This could help if the throughput drop
is caused by the kthreads experiencing higher scheduling latency.


> 1 NUMA node. 18 NAPI instances each is around 25% of a single CPU.
>
> I was initially hoping that TAPI would fit nicely as an extension
> of this code, but I don't think that will be the case.
>
> Are there any assumptions you're making about the configuration that
> I should try to replicate?

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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-29 20:16     ` Wei Wang
@ 2020-09-29 21:48       ` Jakub Kicinski
  2020-09-30  8:23         ` David Laight
  2020-09-30  8:58         ` Paolo Abeni
  0 siblings, 2 replies; 32+ messages in thread
From: Jakub Kicinski @ 2020-09-29 21:48 UTC (permalink / raw)
  To: Wei Wang
  Cc: Eric Dumazet, David S . Miller, netdev, Paolo Abeni,
	Hannes Frederic Sowa, Felix Fietkau

On Tue, 29 Sep 2020 13:16:59 -0700 Wei Wang wrote:
> On Tue, Sep 29, 2020 at 12:19 PM Jakub Kicinski <kuba@kernel.org> wrote:
> > On Mon, 28 Sep 2020 19:43:36 +0200 Eric Dumazet wrote:  
> > > Wei, this is a very nice work.
> > >
> > > Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.  
> >
> > The problem is for the application I'm testing with this implementation
> > is significantly slower (in terms of RPS) than Felix's code:
> >
> >               |        L  A  T  E  N  C  Y       |  App   |     C P U     |
> >        |  RPS |   AVG  |  P50  |   P99  |   P999 | Overld |  busy |  PSI  |
> > thread | 1.1% | -15.6% | -0.3% | -42.5% |  -8.1% | -83.4% | -2.3% | 60.6% |
> > work q | 4.3% | -13.1% |  0.1% | -44.4% |  -1.1% |   2.3% | -1.2% | 90.1% |
> > TAPI   | 4.4% | -17.1% | -1.4% | -43.8% | -11.0% | -60.2% | -2.3% | 46.7% |
> >
> > thread is this code, "work q" is Felix's code, TAPI is my hacks.
> >
> > The numbers are comparing performance to normal NAPI.
> >
> > In all cases (but not the baseline) I configured timer-based polling
> > (defer_hard_irqs), with around 100us timeout. Without deferring hard
> > IRQs threaded NAPI is actually slower for this app. Also I'm not
> > modifying niceness, this again causes application performance
> > regression here.
> >  
> 
> If I remember correctly, Felix's workqueue code uses HIGHPRIO flag
> which by default uses -20 as the nice value for the workqueue threads.
> But the kthread implementation leaves nice level as 20 by default.
> This could be 1 difference.

FWIW this is the data based on which I concluded the nice -20 actually
makes things worse here:

      threded: -1.50%
 threded p-20: -5.67%
     thr poll:  2.93%
thr poll p-20:  2.22%

Annoyingly relative performance change varies day to day and this test
was run a while back (over the weekend I was getting < 2% improvement
with this set).

> I am not sure what the benchmark is doing

Not a benchmark, real workload :)

> but one thing to try is to limit the CPUs that run the kthreads to a
> smaller # of CPUs. This could bring up the kernel cpu usage to a
> higher %, e.g. > 80%, so the scheduler is less likely to schedule
> user threads on these CPUs, thus providing isolations between
> kthreads and the user threads, and reducing the scheduling overhead.

Yeah... If I do pinning or isolation I can get to 15% RPS improvement
for this application.. no threaded NAPI needed. The point for me is to
not have to do such tuning per app x platform x workload of the day.

> This could help if the throughput drop is caused by higher scheduling
> latency for the user threads. Another thing to try is to raise the
> scheduling class of the kthread from SCHED_OTHER to SCHED_FIFO. This
> could help if the throughput drop is caused by the kthreads
> experiencing higher scheduling latency.

Isn't the fundamental problem that scheduler works at ms scale while
where we're talking about 100us at most? And AFAICT scheduler doesn't
have a knob to adjust migration cost per process? :(

I just reached out to the kernel experts @FB for their input.

Also let me re-run with a normal prio WQ.

> > 1 NUMA node. 18 NAPI instances each is around 25% of a single CPU.
> >
> > I was initially hoping that TAPI would fit nicely as an extension
> > of this code, but I don't think that will be the case.
> >
> > Are there any assumptions you're making about the configuration that
> > I should try to replicate?  

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

* RE: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-29 21:48       ` Jakub Kicinski
@ 2020-09-30  8:23         ` David Laight
  2020-09-30  8:58         ` Paolo Abeni
  1 sibling, 0 replies; 32+ messages in thread
From: David Laight @ 2020-09-30  8:23 UTC (permalink / raw)
  To: 'Jakub Kicinski', Wei Wang
  Cc: Eric Dumazet, David S . Miller, netdev, Paolo Abeni,
	Hannes Frederic Sowa, Felix Fietkau

From: Jakub Kicinski
> Sent: 29 September 2020 22:49
...
> Isn't the fundamental problem that scheduler works at ms scale while
> where we're talking about 100us at most? And AFAICT scheduler doesn't
> have a knob to adjust migration cost per process? :(

Have you tried setting the application processes to RT priorities?
The scheduler tries very hard (maybe too hard) to avoid migrating
RT processes.

	David

-
Registered Address Lakeside, Bramley Road, Mount Farm, Milton Keynes, MK1 1PT, UK
Registration No: 1397386 (Wales)


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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-29 21:48       ` Jakub Kicinski
  2020-09-30  8:23         ` David Laight
@ 2020-09-30  8:58         ` Paolo Abeni
  2020-09-30 15:58           ` Jakub Kicinski
  1 sibling, 1 reply; 32+ messages in thread
From: Paolo Abeni @ 2020-09-30  8:58 UTC (permalink / raw)
  To: Jakub Kicinski, Wei Wang
  Cc: Eric Dumazet, David S . Miller, netdev, Hannes Frederic Sowa,
	Felix Fietkau, Luigi Rizzo

On Tue, 2020-09-29 at 14:48 -0700, Jakub Kicinski wrote:
> On Tue, 29 Sep 2020 13:16:59 -0700 Wei Wang wrote:
> > On Tue, Sep 29, 2020 at 12:19 PM Jakub Kicinski <kuba@kernel.org> wrote:
> > > On Mon, 28 Sep 2020 19:43:36 +0200 Eric Dumazet wrote:  
> > > > Wei, this is a very nice work.
> > > > 
> > > > Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.  
> > > 
> > > The problem is for the application I'm testing with this implementation
> > > is significantly slower (in terms of RPS) than Felix's code:
> > > 
> > >               |        L  A  T  E  N  C  Y       |  App   |     C P U     |
> > >        |  RPS |   AVG  |  P50  |   P99  |   P999 | Overld |  busy |  PSI  |
> > > thread | 1.1% | -15.6% | -0.3% | -42.5% |  -8.1% | -83.4% | -2.3% | 60.6% |
> > > work q | 4.3% | -13.1% |  0.1% | -44.4% |  -1.1% |   2.3% | -1.2% | 90.1% |
> > > TAPI   | 4.4% | -17.1% | -1.4% | -43.8% | -11.0% | -60.2% | -2.3% | 46.7% |
> > > 
> > > thread is this code, "work q" is Felix's code, TAPI is my hacks.
> > > 
> > > The numbers are comparing performance to normal NAPI.
> > > 
> > > In all cases (but not the baseline) I configured timer-based polling
> > > (defer_hard_irqs), with around 100us timeout. Without deferring hard
> > > IRQs threaded NAPI is actually slower for this app. Also I'm not
> > > modifying niceness, this again causes application performance
> > > regression here.
> > >  
> > 
> > If I remember correctly, Felix's workqueue code uses HIGHPRIO flag
> > which by default uses -20 as the nice value for the workqueue threads.
> > But the kthread implementation leaves nice level as 20 by default.
> > This could be 1 difference.
> 
> FWIW this is the data based on which I concluded the nice -20 actually
> makes things worse here:
> 
>       threded: -1.50%
>  threded p-20: -5.67%
>      thr poll:  2.93%
> thr poll p-20:  2.22%
> 
> Annoyingly relative performance change varies day to day and this test
> was run a while back (over the weekend I was getting < 2% improvement
> with this set).

I'm assuming your application uses UDP as the transport protocol - raw
IP or packet socket should behave in the same way. I observed similar
behavior - that is unstable figures, and end-to-end tput decrease when
network stack get more cycles (or become faster) - when the bottle-neck 
was in user-space processing[1].

You can double check you are hitting the same scenario observing the
UDP protocol stats (you should see higher drops figures with threaded
and even more with threded p-20, compared to the other impls).

If you are hitting such scenario, you should be able to improve things
setting nice-20 to the user-space process, increasing the UDP socket
receive buffer size or enabling socket busy polling
(/proc/sys/net/core/busy_poll, I mean). 

Cheers,

Paolo

[1] Perhaps that is obvious to you, but I personally was confused the
first time I observed this fact. There is a nice paper from Luigi Rizzo
explaining why that happen:
http://www.iet.unipi.it/~a007834/papers/2016-ancs-cvt.pdf


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

* Re: [RFC PATCH net-next 0/6] implement kthread based napi poll
  2020-09-30  8:58         ` Paolo Abeni
@ 2020-09-30 15:58           ` Jakub Kicinski
  0 siblings, 0 replies; 32+ messages in thread
From: Jakub Kicinski @ 2020-09-30 15:58 UTC (permalink / raw)
  To: Paolo Abeni
  Cc: Wei Wang, Eric Dumazet, David S . Miller, netdev,
	Hannes Frederic Sowa, Felix Fietkau, Luigi Rizzo

On Wed, 30 Sep 2020 10:58:00 +0200 Paolo Abeni wrote:
> On Tue, 2020-09-29 at 14:48 -0700, Jakub Kicinski wrote:
> > On Tue, 29 Sep 2020 13:16:59 -0700 Wei Wang wrote:  
> > > On Tue, Sep 29, 2020 at 12:19 PM Jakub Kicinski <kuba@kernel.org> wrote:  
> > > > On Mon, 28 Sep 2020 19:43:36 +0200 Eric Dumazet wrote:    
> > > > > Wei, this is a very nice work.
> > > > > 
> > > > > Please re-send it without the RFC tag, so that we can hopefully merge it ASAP.    
> > > > 
> > > > The problem is for the application I'm testing with this implementation
> > > > is significantly slower (in terms of RPS) than Felix's code:
> > > > 
> > > >               |        L  A  T  E  N  C  Y       |  App   |     C P U     |
> > > >        |  RPS |   AVG  |  P50  |   P99  |   P999 | Overld |  busy |  PSI  |
> > > > thread | 1.1% | -15.6% | -0.3% | -42.5% |  -8.1% | -83.4% | -2.3% | 60.6% |
> > > > work q | 4.3% | -13.1% |  0.1% | -44.4% |  -1.1% |   2.3% | -1.2% | 90.1% |
> > > > TAPI   | 4.4% | -17.1% | -1.4% | -43.8% | -11.0% | -60.2% | -2.3% | 46.7% |
> > > > 
> > > > thread is this code, "work q" is Felix's code, TAPI is my hacks.
> > > > 
> > > > The numbers are comparing performance to normal NAPI.
> > > > 
> > > > In all cases (but not the baseline) I configured timer-based polling
> > > > (defer_hard_irqs), with around 100us timeout. Without deferring hard
> > > > IRQs threaded NAPI is actually slower for this app. Also I'm not
> > > > modifying niceness, this again causes application performance
> > > > regression here.
> > > >    
> > > 
> > > If I remember correctly, Felix's workqueue code uses HIGHPRIO flag
> > > which by default uses -20 as the nice value for the workqueue threads.
> > > But the kthread implementation leaves nice level as 20 by default.
> > > This could be 1 difference.  
> > 
> > FWIW this is the data based on which I concluded the nice -20 actually
> > makes things worse here:
> > 
> >       threded: -1.50%
> >  threded p-20: -5.67%
> >      thr poll:  2.93%
> > thr poll p-20:  2.22%
> > 
> > Annoyingly relative performance change varies day to day and this test
> > was run a while back (over the weekend I was getting < 2% improvement
> > with this set).  
> 
> I'm assuming your application uses UDP as the transport protocol - raw
> IP or packet socket should behave in the same way. I observed similar
> behavior - that is unstable figures, and end-to-end tput decrease when
> network stack get more cycles (or become faster) - when the bottle-neck 
> was in user-space processing[1].
> 
> You can double check you are hitting the same scenario observing the
> UDP protocol stats (you should see higher drops figures with threaded
> and even more with threded p-20, compared to the other impls).
> 
> If you are hitting such scenario, you should be able to improve things
> setting nice-20 to the user-space process, increasing the UDP socket
> receive buffer size or enabling socket busy polling
> (/proc/sys/net/core/busy_poll, I mean). 

It's not UDP. The application has some logic to tell the load balancer
to back off whenever it feels like it's not processing requests fast enough
(App Overld in the table 2 emails back). That statistic is higher with p-20.
Application latency suffers, too.

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

end of thread, other threads:[~2020-09-30 15:58 UTC | newest]

Thread overview: 32+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-09-14 17:24 [RFC PATCH net-next 0/6] implement kthread based napi poll Wei Wang
2020-09-14 17:24 ` [RFC PATCH net-next 1/6] net: implement threaded-able napi poll loop support Wei Wang
2020-09-25 19:45   ` Hannes Frederic Sowa
2020-09-25 23:50     ` Wei Wang
2020-09-26 14:22       ` Hannes Frederic Sowa
2020-09-28  8:45         ` Paolo Abeni
2020-09-28 18:13           ` Wei Wang
2020-09-14 17:24 ` [RFC PATCH net-next 2/6] net: add sysfs attribute to control napi threaded mode Wei Wang
2020-09-15  2:50   ` kernel test robot
2020-09-15  3:47   ` kernel test robot
2020-09-14 17:24 ` [RFC PATCH net-next 3/6] net: extract napi poll functionality to __napi_poll() Wei Wang
2020-09-14 17:24 ` [RFC PATCH net-next 4/6] net: modify kthread handler to use __napi_poll() Wei Wang
2020-09-14 17:24 ` [RFC PATCH net-next 5/6] net: process RPS/RFS work in kthread context Wei Wang
2020-09-18 22:44   ` Wei Wang
2020-09-21  8:11     ` Eric Dumazet
2020-09-14 17:24 ` [RFC PATCH net-next 6/6] net: improve napi threaded config Wei Wang
2020-09-25 13:48 ` [RFC PATCH net-next 0/6] implement kthread based napi poll Magnus Karlsson
2020-09-25 17:15   ` Wei Wang
2020-09-25 17:30     ` Eric Dumazet
2020-09-25 18:16     ` Stephen Hemminger
2020-09-25 18:23       ` Eric Dumazet
2020-09-25 19:00         ` Stephen Hemminger
2020-09-25 19:06   ` Jakub Kicinski
2020-09-28 14:07     ` Magnus Karlsson
2020-09-28 17:43 ` Eric Dumazet
2020-09-28 18:15   ` Wei Wang
2020-09-29 19:19   ` Jakub Kicinski
2020-09-29 20:16     ` Wei Wang
2020-09-29 21:48       ` Jakub Kicinski
2020-09-30  8:23         ` David Laight
2020-09-30  8:58         ` Paolo Abeni
2020-09-30 15:58           ` Jakub Kicinski

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.