All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH net-next 0/5] implement kthread based napi poll
@ 2020-09-30 19:21 Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 1/5] net: implement threaded-able napi poll loop support Wei Wang
                   ` (6 more replies)
  0 siblings, 7 replies; 27+ messages in thread
From: Wei Wang @ 2020-09-30 19:21 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni,
	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 have 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()
Wei Wang (1):
  net: improve napi threaded config

 include/linux/netdevice.h |   5 ++
 net/core/dev.c            | 139 +++++++++++++++++++++++++++++++++++---
 net/core/net-sysfs.c      | 100 +++++++++++++++++++++++++++
 3 files changed, 235 insertions(+), 9 deletions(-)

-- 
2.28.0.709.gb0816b6eb0-goog


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

* [PATCH net-next 1/5] net: implement threaded-able napi poll loop support
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
@ 2020-09-30 19:21 ` Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 2/5] net: add sysfs attribute to control napi threaded mode Wei Wang
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 27+ messages in thread
From: Wei Wang @ 2020-09-30 19:21 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni,
	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 28cfa53daf72..b3516e77371e 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 9d55bf5d1a65..259cd7f3434f 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>
@@ -1487,9 +1488,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();
@@ -1521,6 +1532,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)
@@ -1566,6 +1580,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;
@@ -1594,6 +1616,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.
@@ -1605,6 +1628,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);
 	}
@@ -4241,6 +4267,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);
 }
@@ -6654,6 +6685,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)
 {
@@ -6794,6 +6849,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.709.gb0816b6eb0-goog


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

* [PATCH net-next 2/5] net: add sysfs attribute to control napi threaded mode
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 1/5] net: implement threaded-able napi poll loop support Wei Wang
@ 2020-09-30 19:21 ` Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 3/5] net: extract napi poll functionality to __napi_poll() Wei Wang
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 27+ messages in thread
From: Wei Wang @ 2020-09-30 19:21 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni,
	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>
---
 net/core/net-sysfs.c | 103 +++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 103 insertions(+)

diff --git a/net/core/net-sysfs.c b/net/core/net-sysfs.c
index efec66fa78b7..fe81b344447d 100644
--- a/net/core/net-sysfs.c
+++ b/net/core/net-sysfs.c
@@ -538,6 +538,108 @@ static ssize_t phys_switch_id_show(struct device *dev,
 }
 static DEVICE_ATTR_RO(phys_switch_id);
 
+static 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 +672,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.709.gb0816b6eb0-goog


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

* [PATCH net-next 3/5] net: extract napi poll functionality to __napi_poll()
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 1/5] net: implement threaded-able napi poll loop support Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 2/5] net: add sysfs attribute to control napi threaded mode Wei Wang
@ 2020-09-30 19:21 ` Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 4/5] net: modify kthread handler to use __napi_poll() Wei Wang
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 27+ messages in thread
From: Wei Wang @ 2020-09-30 19:21 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni,
	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 259cd7f3434f..c82522262ca8 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -6783,15 +6783,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
@@ -6811,7 +6806,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
@@ -6820,7 +6815,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) {
@@ -6832,6 +6827,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.709.gb0816b6eb0-goog


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

* [PATCH net-next 4/5] net: modify kthread handler to use __napi_poll()
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
                   ` (2 preceding siblings ...)
  2020-09-30 19:21 ` [PATCH net-next 3/5] net: extract napi poll functionality to __napi_poll() Wei Wang
@ 2020-09-30 19:21 ` Wei Wang
  2020-09-30 19:21 ` [PATCH net-next 5/5] net: improve napi threaded config Wei Wang
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 27+ messages in thread
From: Wei Wang @ 2020-09-30 19:21 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni,
	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 c82522262ca8..b4f33e442b5e 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -6827,6 +6827,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;
@@ -6847,15 +6856,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:
@@ -6884,40 +6884,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.709.gb0816b6eb0-goog


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

* [PATCH net-next 5/5] net: improve napi threaded config
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
                   ` (3 preceding siblings ...)
  2020-09-30 19:21 ` [PATCH net-next 4/5] net: modify kthread handler to use __napi_poll() Wei Wang
@ 2020-09-30 19:21 ` Wei Wang
  2020-10-01 10:01   ` Felix Fietkau
  2020-09-30 20:08 ` [PATCH net-next 0/5] implement kthread based napi poll Jakub Kicinski
  2020-10-02 23:00 ` David Miller
  6 siblings, 1 reply; 27+ messages in thread
From: Wei Wang @ 2020-09-30 19:21 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni,
	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>
---
Changes since RFC:
changed the thread name to napi/<dev>-<napi-id>

 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 b4f33e442b5e..bf878d3a9d89 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -1490,17 +1490,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, "napi/%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();
@@ -1532,9 +1539,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)
@@ -1585,6 +1589,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;
 }
 
@@ -4267,7 +4272,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;
 	}
@@ -6687,25 +6692,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);
 
@@ -6750,6 +6755,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);
 }
@@ -6870,6 +6876,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 fe81b344447d..b54dbccf00be 100644
--- a/net/core/net-sysfs.c
+++ b/net/core/net-sysfs.c
@@ -609,11 +609,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;
@@ -626,7 +621,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.709.gb0816b6eb0-goog


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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
                   ` (4 preceding siblings ...)
  2020-09-30 19:21 ` [PATCH net-next 5/5] net: improve napi threaded config Wei Wang
@ 2020-09-30 20:08 ` Jakub Kicinski
  2020-10-01  7:52   ` Eric Dumazet
  2020-10-02 23:00 ` David Miller
  6 siblings, 1 reply; 27+ messages in thread
From: Jakub Kicinski @ 2020-09-30 20:08 UTC (permalink / raw)
  To: Wei Wang
  Cc: David S . Miller, netdev, Eric Dumazet, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Wed, 30 Sep 2020 12:21:35 -0700 Wei Wang wrote:
> 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. 

As I said in my reply to the RFC I see better performance with the
workqueue implementation, so I would hold off until we have more
conclusive results there, as this set adds fairly strong uAPI that 
we'll have to support for ever.


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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-09-30 20:08 ` [PATCH net-next 0/5] implement kthread based napi poll Jakub Kicinski
@ 2020-10-01  7:52   ` Eric Dumazet
  2020-10-01 20:26     ` Jakub Kicinski
  0 siblings, 1 reply; 27+ messages in thread
From: Eric Dumazet @ 2020-10-01  7:52 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: Wei Wang, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Wed, Sep 30, 2020 at 10:08 PM Jakub Kicinski <kuba@kernel.org> wrote:
>
> On Wed, 30 Sep 2020 12:21:35 -0700 Wei Wang wrote:
> > 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.
>
> As I said in my reply to the RFC I see better performance with the
> workqueue implementation, so I would hold off until we have more
> conclusive results there, as this set adds fairly strong uAPI that
> we'll have to support for ever.


We can make incremental changes, the kthread implementation looks much
nicer to us.

The unique work queue is a problem on server class platforms, with
NUMA placement.
We now have servers with NIC on different NUMA nodes.

We can not introduce a new model that will make all workload better
without any tuning.
If you really think you can do that, think again.

Even the old ' fix'  (commit 4cd13c21b207e80ddb1144c576500098f2d5f882
"softirq: Let ksoftirqd do its job" )
had severe issues for latency sensitive jobs.

We need to be able to opt-in to threads, and let process scheduler
take decisions.
If we believe the process scheduler takes bad decision, it should be
reported to scheduler experts.

I fully support this implementation, I do not want to wait for yet
another 'work queue' model or scheduler classes.

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-09-30 19:21 ` [PATCH net-next 5/5] net: improve napi threaded config Wei Wang
@ 2020-10-01 10:01   ` Felix Fietkau
  2020-10-01 17:01     ` Wei Wang
  0 siblings, 1 reply; 27+ messages in thread
From: Felix Fietkau @ 2020-10-01 10:01 UTC (permalink / raw)
  To: Wei Wang, David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni


On 2020-09-30 21:21, Wei Wang wrote:
> 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>
> ---
> Changes since RFC:
> changed the thread name to napi/<dev>-<napi-id>
> 
>  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 b4f33e442b5e..bf878d3a9d89 100644
> --- a/net/core/dev.c
> +++ b/net/core/dev.c
> @@ -1490,17 +1490,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, "napi/%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;
If I remember correctly, using kthread_create with no explicit first
wakeup means the task will sit there and contribute to system loadavg
until it is woken up the first time.
Shouldn't we use kthread_run here instead?

- Felix

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 10:01   ` Felix Fietkau
@ 2020-10-01 17:01     ` Wei Wang
  2020-10-01 17:11       ` Felix Fietkau
  0 siblings, 1 reply; 27+ messages in thread
From: Wei Wang @ 2020-10-01 17:01 UTC (permalink / raw)
  To: Felix Fietkau
  Cc: David S . Miller, Linux Kernel Network Developers, Eric Dumazet,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni

On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
>
>
> On 2020-09-30 21:21, Wei Wang wrote:
> > 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>
> > ---
> > Changes since RFC:
> > changed the thread name to napi/<dev>-<napi-id>
> >
> >  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 b4f33e442b5e..bf878d3a9d89 100644
> > --- a/net/core/dev.c
> > +++ b/net/core/dev.c
> > @@ -1490,17 +1490,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, "napi/%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;
> If I remember correctly, using kthread_create with no explicit first
> wakeup means the task will sit there and contribute to system loadavg
> until it is woken up the first time.
> Shouldn't we use kthread_run here instead?
>

Right. kthread_create() basically creates the thread and leaves it in
sleep mode. I think that is what we want. We rely on the next
___napi_schedule() call to wake up this thread when there is work to
do.

> - Felix

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 17:01     ` Wei Wang
@ 2020-10-01 17:11       ` Felix Fietkau
  2020-10-01 18:03         ` Eric Dumazet
  0 siblings, 1 reply; 27+ messages in thread
From: Felix Fietkau @ 2020-10-01 17:11 UTC (permalink / raw)
  To: Wei Wang
  Cc: David S . Miller, Linux Kernel Network Developers, Eric Dumazet,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni

On 2020-10-01 19:01, Wei Wang wrote:
> On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
>>
>>
>> On 2020-09-30 21:21, Wei Wang wrote:
>> > 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>
>> > ---
>> > Changes since RFC:
>> > changed the thread name to napi/<dev>-<napi-id>
>> >
>> >  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 b4f33e442b5e..bf878d3a9d89 100644
>> > --- a/net/core/dev.c
>> > +++ b/net/core/dev.c
>> > @@ -1490,17 +1490,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, "napi/%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;
>> If I remember correctly, using kthread_create with no explicit first
>> wakeup means the task will sit there and contribute to system loadavg
>> until it is woken up the first time.
>> Shouldn't we use kthread_run here instead?
>>
> 
> Right. kthread_create() basically creates the thread and leaves it in
> sleep mode. I think that is what we want. We rely on the next
> ___napi_schedule() call to wake up this thread when there is work to
> do.
But what if you have a device that's basically idle and napi isn't
scheduled until much later? It will get a confusing loadavg until then.
I'd prefer waking up the thread immediately and filtering going back to
sleep once in the thread function before running the loop if
NAPI_STATE_SCHED wasn't set.

- Felix

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 17:11       ` Felix Fietkau
@ 2020-10-01 18:03         ` Eric Dumazet
  2020-10-01 18:37           ` Felix Fietkau
  0 siblings, 1 reply; 27+ messages in thread
From: Eric Dumazet @ 2020-10-01 18:03 UTC (permalink / raw)
  To: Felix Fietkau
  Cc: Wei Wang, David S . Miller, Linux Kernel Network Developers,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni

On Thu, Oct 1, 2020 at 7:12 PM Felix Fietkau <nbd@nbd.name> wrote:
>
> On 2020-10-01 19:01, Wei Wang wrote:
> > On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
> >>
> >>
> >> On 2020-09-30 21:21, Wei Wang wrote:
> >> > 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>
> >> > ---
> >> > Changes since RFC:
> >> > changed the thread name to napi/<dev>-<napi-id>
> >> >
> >> >  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 b4f33e442b5e..bf878d3a9d89 100644
> >> > --- a/net/core/dev.c
> >> > +++ b/net/core/dev.c
> >> > @@ -1490,17 +1490,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, "napi/%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;
> >> If I remember correctly, using kthread_create with no explicit first
> >> wakeup means the task will sit there and contribute to system loadavg
> >> until it is woken up the first time.
> >> Shouldn't we use kthread_run here instead?
> >>
> >
> > Right. kthread_create() basically creates the thread and leaves it in
> > sleep mode. I think that is what we want. We rely on the next
> > ___napi_schedule() call to wake up this thread when there is work to
> > do.
> But what if you have a device that's basically idle and napi isn't
> scheduled until much later? It will get a confusing loadavg until then.
> I'd prefer waking up the thread immediately and filtering going back to
> sleep once in the thread function before running the loop if
> NAPI_STATE_SCHED wasn't set.
>

I was not aware of this kthread_create() impact on loadavg.
This seems like a bug to me. (although I do not care about loadavg)

Do you have pointers on some documentation ?

Probably not a big deal, but this seems quite odd to me.

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 18:03         ` Eric Dumazet
@ 2020-10-01 18:37           ` Felix Fietkau
  2020-10-01 19:24             ` Wei Wang
  0 siblings, 1 reply; 27+ messages in thread
From: Felix Fietkau @ 2020-10-01 18:37 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: Wei Wang, David S . Miller, Linux Kernel Network Developers,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni


On 2020-10-01 20:03, Eric Dumazet wrote:
> On Thu, Oct 1, 2020 at 7:12 PM Felix Fietkau <nbd@nbd.name> wrote:
>>
>> On 2020-10-01 19:01, Wei Wang wrote:
>> > On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
>> >>
>> >>
>> >> On 2020-09-30 21:21, Wei Wang wrote:
>> >> > 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>
>> >> > ---
>> >> > Changes since RFC:
>> >> > changed the thread name to napi/<dev>-<napi-id>
>> >> >
>> >> >  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 b4f33e442b5e..bf878d3a9d89 100644
>> >> > --- a/net/core/dev.c
>> >> > +++ b/net/core/dev.c
>> >> > @@ -1490,17 +1490,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, "napi/%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;
>> >> If I remember correctly, using kthread_create with no explicit first
>> >> wakeup means the task will sit there and contribute to system loadavg
>> >> until it is woken up the first time.
>> >> Shouldn't we use kthread_run here instead?
>> >>
>> >
>> > Right. kthread_create() basically creates the thread and leaves it in
>> > sleep mode. I think that is what we want. We rely on the next
>> > ___napi_schedule() call to wake up this thread when there is work to
>> > do.
>> But what if you have a device that's basically idle and napi isn't
>> scheduled until much later? It will get a confusing loadavg until then.
>> I'd prefer waking up the thread immediately and filtering going back to
>> sleep once in the thread function before running the loop if
>> NAPI_STATE_SCHED wasn't set.
>>
> 
> I was not aware of this kthread_create() impact on loadavg.
> This seems like a bug to me. (although I do not care about loadavg)
> 
> Do you have pointers on some documentation ?
I don't have any specific documentation pointers, but this is something
I observed on several occasions when playing with kthreads.

From what I can find in the loadavg code it seems that tasks in
TASK_UNINTERRUPTIBLE state are counted for loadavg alongside actually
runnable tasks. This seems intentional to me, but I don't know why it
was made like this.

A kthread does not start the thread function until it has been woken up
at least once, most likely to give the creating code a chance to perform
some initializations after successfully creating the thread, before the
thread function starts doing something. Instead, kthread() sets
TASK_UNINTERRUPTIBLE and calls schedule() once.

> Probably not a big deal, but this seems quite odd to me.
I've run into enough users that look at loadavg as a measure of system
load and would likely start reporting bugs if they observe such
behavior. I'd like to avoid that.

- Felix

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 18:37           ` Felix Fietkau
@ 2020-10-01 19:24             ` Wei Wang
  2020-10-01 20:48               ` Felix Fietkau
  0 siblings, 1 reply; 27+ messages in thread
From: Wei Wang @ 2020-10-01 19:24 UTC (permalink / raw)
  To: Felix Fietkau
  Cc: Eric Dumazet, David S . Miller, Linux Kernel Network Developers,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni

On Thu, Oct 1, 2020 at 11:38 AM Felix Fietkau <nbd@nbd.name> wrote:
>
>
> On 2020-10-01 20:03, Eric Dumazet wrote:
> > On Thu, Oct 1, 2020 at 7:12 PM Felix Fietkau <nbd@nbd.name> wrote:
> >>
> >> On 2020-10-01 19:01, Wei Wang wrote:
> >> > On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
> >> >>
> >> >>
> >> >> On 2020-09-30 21:21, Wei Wang wrote:
> >> >> > 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>
> >> >> > ---
> >> >> > Changes since RFC:
> >> >> > changed the thread name to napi/<dev>-<napi-id>
> >> >> >
> >> >> >  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 b4f33e442b5e..bf878d3a9d89 100644
> >> >> > --- a/net/core/dev.c
> >> >> > +++ b/net/core/dev.c
> >> >> > @@ -1490,17 +1490,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, "napi/%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;
> >> >> If I remember correctly, using kthread_create with no explicit first
> >> >> wakeup means the task will sit there and contribute to system loadavg
> >> >> until it is woken up the first time.
> >> >> Shouldn't we use kthread_run here instead?
> >> >>
> >> >
> >> > Right. kthread_create() basically creates the thread and leaves it in
> >> > sleep mode. I think that is what we want. We rely on the next
> >> > ___napi_schedule() call to wake up this thread when there is work to
> >> > do.
> >> But what if you have a device that's basically idle and napi isn't
> >> scheduled until much later? It will get a confusing loadavg until then.
> >> I'd prefer waking up the thread immediately and filtering going back to
> >> sleep once in the thread function before running the loop if
> >> NAPI_STATE_SCHED wasn't set.
> >>
> >
> > I was not aware of this kthread_create() impact on loadavg.
> > This seems like a bug to me. (although I do not care about loadavg)
> >
> > Do you have pointers on some documentation ?

I found this link:
http://www.brendangregg.com/blog/2017-08-08/linux-load-averages.html
It has a section called "Linux Uninterruptible Tasks" which explains
this behavior specifically. But I don't see a good conclusion on why.
Seems to be a convention.
IMHO, this is actually the problem/decision of the loadavg. It should
not impact how the kernel code is implemented. I think it makes more
sense to only wake up the thread when there is work to do.

> I don't have any specific documentation pointers, but this is something
> I observed on several occasions when playing with kthreads.
>
> From what I can find in the loadavg code it seems that tasks in
> TASK_UNINTERRUPTIBLE state are counted for loadavg alongside actually
> runnable tasks. This seems intentional to me, but I don't know why it
> was made like this.
>
> A kthread does not start the thread function until it has been woken up
> at least once, most likely to give the creating code a chance to perform
> some initializations after successfully creating the thread, before the
> thread function starts doing something. Instead, kthread() sets
> TASK_UNINTERRUPTIBLE and calls schedule() once.
>
> > Probably not a big deal, but this seems quite odd to me.
> I've run into enough users that look at loadavg as a measure of system
> load and would likely start reporting bugs if they observe such
> behavior. I'd like to avoid that.
>
> - Felix

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-01  7:52   ` Eric Dumazet
@ 2020-10-01 20:26     ` Jakub Kicinski
  2020-10-01 22:12       ` Wei Wang
  2020-10-02  7:56       ` Eric Dumazet
  0 siblings, 2 replies; 27+ messages in thread
From: Jakub Kicinski @ 2020-10-01 20:26 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: Wei Wang, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Thu, 1 Oct 2020 09:52:45 +0200 Eric Dumazet wrote:
> On Wed, Sep 30, 2020 at 10:08 PM Jakub Kicinski <kuba@kernel.org> wrote:
> > On Wed, 30 Sep 2020 12:21:35 -0700 Wei Wang wrote:  
> > > 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.  
> >
> > As I said in my reply to the RFC I see better performance with the
> > workqueue implementation, so I would hold off until we have more
> > conclusive results there, as this set adds fairly strong uAPI that
> > we'll have to support for ever.  
> 
> We can make incremental changes, the kthread implementation looks much
> nicer to us.

Having done two implementation of something more wq-like now 
I can say with some confidence that it's quite likely not a 
simple extension of this model. And since we'll likely need
to support switching at runtime there will be a fast-path
synchronization overhead.

> The unique work queue is a problem on server class platforms, with
> NUMA placement.
> We now have servers with NIC on different NUMA nodes.

Are you saying that the wq code is less NUMA friendly than unpinned
threads?

> We can not introduce a new model that will make all workload better
> without any tuning.
> If you really think you can do that, think again.

Has Wei tested the wq implementation with real workloads?

All the cover letter has is some basic netperf runs and a vague
sentence saying "real workload also improved".

I think it's possible to get something that will be a better default
for 90% of workloads. Our current model predates SMP by two decades.
It's pretty bad.

I'm talking about upstream defaults, obviously, maybe you're starting
from a different baseline configuration than the rest of the world..

> Even the old ' fix'  (commit 4cd13c21b207e80ddb1144c576500098f2d5f882
> "softirq: Let ksoftirqd do its job" )
> had severe issues for latency sensitive jobs.
> 
> We need to be able to opt-in to threads, and let process scheduler
> take decisions.
> If we believe the process scheduler takes bad decision, it should be
> reported to scheduler experts.

I wouldn't expect that the scheduler will learn all by itself how to
group processes that run identical code for cache efficiency, and how
to schedule at 10us scale. I hope I'm wrong.

> I fully support this implementation, I do not want to wait for yet
> another 'work queue' model or scheduler classes.

I can't sympathize. I don't understand why you're trying to rush this.
And you're not giving me enough info about your target config to be able
to understand your thinking.

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 19:24             ` Wei Wang
@ 2020-10-01 20:48               ` Felix Fietkau
  2020-10-01 22:42                 ` Wei Wang
  0 siblings, 1 reply; 27+ messages in thread
From: Felix Fietkau @ 2020-10-01 20:48 UTC (permalink / raw)
  To: Wei Wang
  Cc: Eric Dumazet, David S . Miller, Linux Kernel Network Developers,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni


On 2020-10-01 21:24, Wei Wang wrote:
> On Thu, Oct 1, 2020 at 11:38 AM Felix Fietkau <nbd@nbd.name> wrote:
>>
>>
>> On 2020-10-01 20:03, Eric Dumazet wrote:
>> > On Thu, Oct 1, 2020 at 7:12 PM Felix Fietkau <nbd@nbd.name> wrote:
>> >>
>> >> On 2020-10-01 19:01, Wei Wang wrote:
>> >> > On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
>> >> >>
>> >> >>
>> >> >> On 2020-09-30 21:21, Wei Wang wrote:
>> >> >> > 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>
>> >> >> > ---
>> >> >> > Changes since RFC:
>> >> >> > changed the thread name to napi/<dev>-<napi-id>
>> >> >> >
>> >> >> >  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 b4f33e442b5e..bf878d3a9d89 100644
>> >> >> > --- a/net/core/dev.c
>> >> >> > +++ b/net/core/dev.c
>> >> >> > @@ -1490,17 +1490,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, "napi/%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;
>> >> >> If I remember correctly, using kthread_create with no explicit first
>> >> >> wakeup means the task will sit there and contribute to system loadavg
>> >> >> until it is woken up the first time.
>> >> >> Shouldn't we use kthread_run here instead?
>> >> >>
>> >> >
>> >> > Right. kthread_create() basically creates the thread and leaves it in
>> >> > sleep mode. I think that is what we want. We rely on the next
>> >> > ___napi_schedule() call to wake up this thread when there is work to
>> >> > do.
>> >> But what if you have a device that's basically idle and napi isn't
>> >> scheduled until much later? It will get a confusing loadavg until then.
>> >> I'd prefer waking up the thread immediately and filtering going back to
>> >> sleep once in the thread function before running the loop if
>> >> NAPI_STATE_SCHED wasn't set.
>> >>
>> >
>> > I was not aware of this kthread_create() impact on loadavg.
>> > This seems like a bug to me. (although I do not care about loadavg)
>> >
>> > Do you have pointers on some documentation ?
> 
> I found this link:
> http://www.brendangregg.com/blog/2017-08-08/linux-load-averages.html
> It has a section called "Linux Uninterruptible Tasks" which explains
> this behavior specifically. But I don't see a good conclusion on why.
> Seems to be a convention.
> IMHO, this is actually the problem/decision of the loadavg. It should
> not impact how the kernel code is implemented. I think it makes more
> sense to only wake up the thread when there is work to do.
There were other users of kthread where the same issue was fixed.
With a quick search, I found these commits:
e890591413819eeb604207ad3261ba617b2ec0bb
3f776e8a25a9d281125490562e1cc5bd7c14cf7c

Please note that one of these describes that a kthread that was created
but not woken was triggering a blocked task warning - so it's not just
the loadavg that matters here.

All the other users of kthread that I looked at also do an initial
wakeup of the thread. Not doing it seems like wrong use of the API to me.

- Felix

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-01 20:26     ` Jakub Kicinski
@ 2020-10-01 22:12       ` Wei Wang
  2020-10-01 23:46         ` Jakub Kicinski
  2020-10-02  7:56       ` Eric Dumazet
  1 sibling, 1 reply; 27+ messages in thread
From: Wei Wang @ 2020-10-01 22:12 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: Eric Dumazet, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Thu, Oct 1, 2020 at 1:26 PM Jakub Kicinski <kuba@kernel.org> wrote:
>
> On Thu, 1 Oct 2020 09:52:45 +0200 Eric Dumazet wrote:
> > On Wed, Sep 30, 2020 at 10:08 PM Jakub Kicinski <kuba@kernel.org> wrote:
> > > On Wed, 30 Sep 2020 12:21:35 -0700 Wei Wang wrote:
> > > > 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.
> > >
> > > As I said in my reply to the RFC I see better performance with the
> > > workqueue implementation, so I would hold off until we have more
> > > conclusive results there, as this set adds fairly strong uAPI that
> > > we'll have to support for ever.
> >
> > We can make incremental changes, the kthread implementation looks much
> > nicer to us.
>
> Having done two implementation of something more wq-like now
> I can say with some confidence that it's quite likely not a
> simple extension of this model. And since we'll likely need
> to support switching at runtime there will be a fast-path
> synchronization overhead.
>
> > The unique work queue is a problem on server class platforms, with
> > NUMA placement.
> > We now have servers with NIC on different NUMA nodes.
>
> Are you saying that the wq code is less NUMA friendly than unpinned
> threads?
>
> > We can not introduce a new model that will make all workload better
> > without any tuning.
> > If you really think you can do that, think again.
>
> Has Wei tested the wq implementation with real workloads?
>
> All the cover letter has is some basic netperf runs and a vague
> sentence saying "real workload also improved".
>

Yes. I did a round of testing with workqueue as well. The "real
workload" I mentioned is a google internal application benchmark which
involves networking  as well as disk ops.
There are 2 types of tests there.
1 is sustained tests, where the ops/s is being pushed to very high,
and keeps the overall cpu usage to > 80%, with various sizes of
payload.
In this type of test case, I see a better result with the kthread
model compared to workqueue in the latency metrics, and similar CPU
savings, with some tuning of the kthreads. (e.g., we limit the
kthreads to a pool of CPUs to run on, to avoid mixture with
application threads. I did the same for workqueue as well to be fair.)
The other is trace based tests, where the load is based on the actual
trace taken from the real servers. This kind of test has less load and
ops/s overall. (~25% total cpu usage on the host)
In this test case, I observe a similar amount of latency savings with
both kthread and workqueue, but workqueue seems to have better cpu
saving here, possibly due to less # of threads woken up to process the
load.

And one reason we would like to push forward with 1 kthread per NAPI,
is we are also trying to do busy polling with the kthread. And it
seems a good model to have 1 kthread dedicated to 1 NAPI to begin
with.

> I think it's possible to get something that will be a better default
> for 90% of workloads. Our current model predates SMP by two decades.
> It's pretty bad.
>
> I'm talking about upstream defaults, obviously, maybe you're starting
> from a different baseline configuration than the rest of the world..
>
> > Even the old ' fix'  (commit 4cd13c21b207e80ddb1144c576500098f2d5f882
> > "softirq: Let ksoftirqd do its job" )
> > had severe issues for latency sensitive jobs.
> >
> > We need to be able to opt-in to threads, and let process scheduler
> > take decisions.
> > If we believe the process scheduler takes bad decision, it should be
> > reported to scheduler experts.
>
> I wouldn't expect that the scheduler will learn all by itself how to
> group processes that run identical code for cache efficiency, and how
> to schedule at 10us scale. I hope I'm wrong.
>
> > I fully support this implementation, I do not want to wait for yet
> > another 'work queue' model or scheduler classes.
>
> I can't sympathize. I don't understand why you're trying to rush this.
> And you're not giving me enough info about your target config to be able
> to understand your thinking.

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

* Re: [PATCH net-next 5/5] net: improve napi threaded config
  2020-10-01 20:48               ` Felix Fietkau
@ 2020-10-01 22:42                 ` Wei Wang
  0 siblings, 0 replies; 27+ messages in thread
From: Wei Wang @ 2020-10-01 22:42 UTC (permalink / raw)
  To: Felix Fietkau
  Cc: Eric Dumazet, David S . Miller, Linux Kernel Network Developers,
	Jakub Kicinski, Hannes Frederic Sowa, Paolo Abeni

On Thu, Oct 1, 2020 at 1:48 PM Felix Fietkau <nbd@nbd.name> wrote:
>
>
> On 2020-10-01 21:24, Wei Wang wrote:
> > On Thu, Oct 1, 2020 at 11:38 AM Felix Fietkau <nbd@nbd.name> wrote:
> >>
> >>
> >> On 2020-10-01 20:03, Eric Dumazet wrote:
> >> > On Thu, Oct 1, 2020 at 7:12 PM Felix Fietkau <nbd@nbd.name> wrote:
> >> >>
> >> >> On 2020-10-01 19:01, Wei Wang wrote:
> >> >> > On Thu, Oct 1, 2020 at 3:01 AM Felix Fietkau <nbd@nbd.name> wrote:
> >> >> >>
> >> >> >>
> >> >> >> On 2020-09-30 21:21, Wei Wang wrote:
> >> >> >> > 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>
> >> >> >> > ---
> >> >> >> > Changes since RFC:
> >> >> >> > changed the thread name to napi/<dev>-<napi-id>
> >> >> >> >
> >> >> >> >  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 b4f33e442b5e..bf878d3a9d89 100644
> >> >> >> > --- a/net/core/dev.c
> >> >> >> > +++ b/net/core/dev.c
> >> >> >> > @@ -1490,17 +1490,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, "napi/%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;
> >> >> >> If I remember correctly, using kthread_create with no explicit first
> >> >> >> wakeup means the task will sit there and contribute to system loadavg
> >> >> >> until it is woken up the first time.
> >> >> >> Shouldn't we use kthread_run here instead?
> >> >> >>
> >> >> >
> >> >> > Right. kthread_create() basically creates the thread and leaves it in
> >> >> > sleep mode. I think that is what we want. We rely on the next
> >> >> > ___napi_schedule() call to wake up this thread when there is work to
> >> >> > do.
> >> >> But what if you have a device that's basically idle and napi isn't
> >> >> scheduled until much later? It will get a confusing loadavg until then.
> >> >> I'd prefer waking up the thread immediately and filtering going back to
> >> >> sleep once in the thread function before running the loop if
> >> >> NAPI_STATE_SCHED wasn't set.
> >> >>
> >> >
> >> > I was not aware of this kthread_create() impact on loadavg.
> >> > This seems like a bug to me. (although I do not care about loadavg)
> >> >
> >> > Do you have pointers on some documentation ?
> >
> > I found this link:
> > http://www.brendangregg.com/blog/2017-08-08/linux-load-averages.html
> > It has a section called "Linux Uninterruptible Tasks" which explains
> > this behavior specifically. But I don't see a good conclusion on why.
> > Seems to be a convention.
> > IMHO, this is actually the problem/decision of the loadavg. It should
> > not impact how the kernel code is implemented. I think it makes more
> > sense to only wake up the thread when there is work to do.
> There were other users of kthread where the same issue was fixed.
> With a quick search, I found these commits:
> e890591413819eeb604207ad3261ba617b2ec0bb
> 3f776e8a25a9d281125490562e1cc5bd7c14cf7c
>
> Please note that one of these describes that a kthread that was created
> but not woken was triggering a blocked task warning - so it's not just
> the loadavg that matters here.
>
> All the other users of kthread that I looked at also do an initial
> wakeup of the thread. Not doing it seems like wrong use of the API to me.
>

Thanks Felix for digging up the above commits. Very helpful. I will
change it to kthread_run() in v2.

> - Felix

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-01 22:12       ` Wei Wang
@ 2020-10-01 23:46         ` Jakub Kicinski
  2020-10-02  1:44           ` Wei Wang
  0 siblings, 1 reply; 27+ messages in thread
From: Jakub Kicinski @ 2020-10-01 23:46 UTC (permalink / raw)
  To: Wei Wang
  Cc: Eric Dumazet, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Thu, 1 Oct 2020 15:12:20 -0700 Wei Wang wrote:
> Yes. I did a round of testing with workqueue as well. The "real
> workload" I mentioned is a google internal application benchmark which
> involves networking  as well as disk ops.
> There are 2 types of tests there.
> 1 is sustained tests, where the ops/s is being pushed to very high,
> and keeps the overall cpu usage to > 80%, with various sizes of
> payload.
> In this type of test case, I see a better result with the kthread
> model compared to workqueue in the latency metrics, and similar CPU
> savings, with some tuning of the kthreads. (e.g., we limit the
> kthreads to a pool of CPUs to run on, to avoid mixture with
> application threads. I did the same for workqueue as well to be fair.)

Can you share relative performance delta of this banchmark?

Could you explain why threads are slower than ksoftirqd if you pin the
application away? From your cover letter it sounded like you want the
scheduler to see the NAPI load, but then you say you pinned the
application away from the NAPI cores for the test, so I'm confused.

> The other is trace based tests, where the load is based on the actual
> trace taken from the real servers. This kind of test has less load and
> ops/s overall. (~25% total cpu usage on the host)
> In this test case, I observe a similar amount of latency savings with
> both kthread and workqueue, but workqueue seems to have better cpu
> saving here, possibly due to less # of threads woken up to process the
> load.
> 
> And one reason we would like to push forward with 1 kthread per NAPI,
> is we are also trying to do busy polling with the kthread. And it
> seems a good model to have 1 kthread dedicated to 1 NAPI to begin
> with.

And you'd pin those busy polling threads to a specific, single CPU, too?
1 cpu : 1 thread : 1 NAPI?

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-01 23:46         ` Jakub Kicinski
@ 2020-10-02  1:44           ` Wei Wang
  2020-10-02 22:53             ` Jakub Kicinski
  0 siblings, 1 reply; 27+ messages in thread
From: Wei Wang @ 2020-10-02  1:44 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: Eric Dumazet, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Thu, Oct 1, 2020 at 4:46 PM Jakub Kicinski <kuba@kernel.org> wrote:
>
> On Thu, 1 Oct 2020 15:12:20 -0700 Wei Wang wrote:
> > Yes. I did a round of testing with workqueue as well. The "real
> > workload" I mentioned is a google internal application benchmark which
> > involves networking  as well as disk ops.
> > There are 2 types of tests there.
> > 1 is sustained tests, where the ops/s is being pushed to very high,
> > and keeps the overall cpu usage to > 80%, with various sizes of
> > payload.
> > In this type of test case, I see a better result with the kthread
> > model compared to workqueue in the latency metrics, and similar CPU
> > savings, with some tuning of the kthreads. (e.g., we limit the
> > kthreads to a pool of CPUs to run on, to avoid mixture with
> > application threads. I did the same for workqueue as well to be fair.)
>
> Can you share relative performance delta of this banchmark?
>
> Could you explain why threads are slower than ksoftirqd if you pin the
> application away? From your cover letter it sounded like you want the
> scheduler to see the NAPI load, but then you say you pinned the
> application away from the NAPI cores for the test, so I'm confused.
>

No. We did not explicitly pin the application threads away.
Application threads are free to run anywhere. What we do is we
restrict the NAPI kthreads to only those CPUs handling rx interrupts.
(For us, 8 cpus out of 56.) So the load on those CPUs are very high
when running the test. And the scheduler is smart enough to avoid
using those CPUs for the application threads automatically.
Here is the results of 1 representative test result:
                     cpu/op   50%tile     95%tile       99%tile
base            71.47        417us      1.01ms          2.9ms
kthread         67.84       396us      976us            2.4ms
workqueue   69.68       386us      791us             1.9ms

Actually, I remembered it wrong. It does seem workqueue is doing
better on latencies. But cpu/op wise, kthread seems to be a bit
better.

> > The other is trace based tests, where the load is based on the actual
> > trace taken from the real servers. This kind of test has less load and
> > ops/s overall. (~25% total cpu usage on the host)
> > In this test case, I observe a similar amount of latency savings with
> > both kthread and workqueue, but workqueue seems to have better cpu
> > saving here, possibly due to less # of threads woken up to process the
> > load.
> >
> > And one reason we would like to push forward with 1 kthread per NAPI,
> > is we are also trying to do busy polling with the kthread. And it
> > seems a good model to have 1 kthread dedicated to 1 NAPI to begin
> > with.
>
> And you'd pin those busy polling threads to a specific, single CPU, too?
> 1 cpu : 1 thread : 1 NAPI?
Yes. That is my thought.

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-01 20:26     ` Jakub Kicinski
  2020-10-01 22:12       ` Wei Wang
@ 2020-10-02  7:56       ` Eric Dumazet
  2020-10-02 22:49         ` Jakub Kicinski
  1 sibling, 1 reply; 27+ messages in thread
From: Eric Dumazet @ 2020-10-02  7:56 UTC (permalink / raw)
  To: Jakub Kicinski
  Cc: Wei Wang, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Thu, Oct 1, 2020 at 10:26 PM Jakub Kicinski <kuba@kernel.org> wrote:
>
> On Thu, 1 Oct 2020 09:52:45 +0200 Eric Dumazet wrote:

> > The unique work queue is a problem on server class platforms, with
> > NUMA placement.
> > We now have servers with NIC on different NUMA nodes.
>
> Are you saying that the wq code is less NUMA friendly than unpinned
> threads?

Yes this is what I am saying.

Using a single and shared wq wont allow you to make sure :
- work for NIC0 attached on NUMA node#0 will be using CPUS belonging to node#0
- work for NIC1 attached on NUMA node#1 will be using CPUS belonging to node#1


The only way you can tune things with a single wq is tweaking a single cpumask,
that we can change with /sys/devices/virtual/workqueue/{wqname}/cpumask
The same for the nice value with  /sys/devices/virtual/workqueue/{wqname}/nice.

In contrast, having kthreads let you tune things independently, if needed.

Even with a single NIC, you can still need isolation between queues.
We have queues dedicated to a certain kind of traffic/application.

The work queue approach would need to be able to create/delete
independent workqueues.
But we tested the workqueue with a single NIC and our results gave to
kthreads a win over the work queue.

Really, wq concept might be a nice abstraction when each work can be
running for arbitrary durations,
and arbitrary numbers of cpus, but with the NAPI model of up to 64
packets at a time, and a fixed number of queues,
we should not add the work queue overhead.

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-02  7:56       ` Eric Dumazet
@ 2020-10-02 22:49         ` Jakub Kicinski
  0 siblings, 0 replies; 27+ messages in thread
From: Jakub Kicinski @ 2020-10-02 22:49 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: Wei Wang, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Fri, 2 Oct 2020 09:56:31 +0200 Eric Dumazet wrote:
> On Thu, Oct 1, 2020 at 10:26 PM Jakub Kicinski <kuba@kernel.org> wrote:
> >
> > On Thu, 1 Oct 2020 09:52:45 +0200 Eric Dumazet wrote:  
> 
> > > The unique work queue is a problem on server class platforms, with
> > > NUMA placement.
> > > We now have servers with NIC on different NUMA nodes.  
> >
> > Are you saying that the wq code is less NUMA friendly than unpinned
> > threads?  
> 
> Yes this is what I am saying.
> 
> Using a single and shared wq wont allow you to make sure :
> - work for NIC0 attached on NUMA node#0 will be using CPUS belonging to node#0
> - work for NIC1 attached on NUMA node#1 will be using CPUS belonging to node#1
> 
> 
> The only way you can tune things with a single wq is tweaking a single cpumask,
> that we can change with /sys/devices/virtual/workqueue/{wqname}/cpumask
> The same for the nice value with  /sys/devices/virtual/workqueue/{wqname}/nice.
> 
> In contrast, having kthreads let you tune things independently, if needed.
> 
> Even with a single NIC, you can still need isolation between queues.
> We have queues dedicated to a certain kind of traffic/application.
> 
> The work queue approach would need to be able to create/delete
> independent workqueues.
> But we tested the workqueue with a single NIC and our results gave to
> kthreads a win over the work queue.

Not according to the results Wei posted last night..

> Really, wq concept might be a nice abstraction when each work can be
> running for arbitrary durations,
> and arbitrary numbers of cpus, but with the NAPI model of up to 64
> packets at a time, and a fixed number of queues,

In my experiments the worker threads get stalled sooner or later. 
And unless there is some work stealing going on latency spikes follow.

I would also not discount the variability in processing time. For a
budget of 64 the processing can take 0-500us per round, not counting
outliers.

> we should not add the work queue overhead.

Does this mean you're going to be against the (more fleshed out)
work queue implementation?

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-02  1:44           ` Wei Wang
@ 2020-10-02 22:53             ` Jakub Kicinski
  0 siblings, 0 replies; 27+ messages in thread
From: Jakub Kicinski @ 2020-10-02 22:53 UTC (permalink / raw)
  To: Wei Wang
  Cc: Eric Dumazet, David S . Miller, netdev, Hannes Frederic Sowa,
	Paolo Abeni, Felix Fietkau

On Thu, 1 Oct 2020 18:44:40 -0700 Wei Wang wrote:
> > Can you share relative performance delta of this banchmark?
> >
> > Could you explain why threads are slower than ksoftirqd if you pin the
> > application away? From your cover letter it sounded like you want the
> > scheduler to see the NAPI load, but then you say you pinned the
> > application away from the NAPI cores for the test, so I'm confused.
> 
> No. We did not explicitly pin the application threads away.
> Application threads are free to run anywhere. What we do is we
> restrict the NAPI kthreads to only those CPUs handling rx interrupts.

Whatever. You pin the NAPI threads and hand-tune their number so the
load of the NAPI CPUs is always higher. If the workload changes the
system will get very unhappy.

> (For us, 8 cpus out of 56.) So the load on those CPUs are very high
> when running the test. And the scheduler is smart enough to avoid
> using those CPUs for the application threads automatically.
> Here is the results of 1 representative test result:
>                      cpu/op   50%tile     95%tile       99%tile
> base            71.47        417us      1.01ms          2.9ms
> kthread         67.84       396us      976us            2.4ms
> workqueue   69.68       386us      791us             1.9ms

Did you renice ksoftirqd in "base"?

> Actually, I remembered it wrong. It does seem workqueue is doing
> better on latencies. But cpu/op wise, kthread seems to be a bit
> better.

Q.E.D.

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
                   ` (5 preceding siblings ...)
  2020-09-30 20:08 ` [PATCH net-next 0/5] implement kthread based napi poll Jakub Kicinski
@ 2020-10-02 23:00 ` David Miller
  2020-10-02 23:15   ` Alexei Starovoitov
  6 siblings, 1 reply; 27+ messages in thread
From: David Miller @ 2020-10-02 23:00 UTC (permalink / raw)
  To: weiwan; +Cc: netdev, edumazet, kuba, hannes, pabeni, nbd

From: Wei Wang <weiwan@google.com>
Date: Wed, 30 Sep 2020 12:21:35 -0700

 ...
> 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. 
...

I think we still need to discuss this some more.

Jakub has some ideas and I honestly think the whole workqueue
approach hasn't been fully considered yet.

If this wan't urgent years ago (when it was NACK'd btw), it isn't
urgent for 5.10 so I don't know why we are pushing so hard for
this patch series to go in as-is right now.

Please be patient and let's have a full discussion on this.

Thank you.

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-02 23:00 ` David Miller
@ 2020-10-02 23:15   ` Alexei Starovoitov
  2020-10-03  3:54     ` Eric Dumazet
  0 siblings, 1 reply; 27+ messages in thread
From: Alexei Starovoitov @ 2020-10-02 23:15 UTC (permalink / raw)
  To: David Miller
  Cc: Wei Wang, Network Development, Eric Dumazet, Jakub Kicinski,
	Hannes Frederic Sowa, Paolo Abeni, nbd

On Fri, Oct 2, 2020 at 4:02 PM David Miller <davem@davemloft.net> wrote:
>
> From: Wei Wang <weiwan@google.com>
> Date: Wed, 30 Sep 2020 12:21:35 -0700
>
>  ...
> > 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.
> ...
>
> I think we still need to discuss this some more.
>
> Jakub has some ideas and I honestly think the whole workqueue
> approach hasn't been fully considered yet.

I want to point out that it's not kthread vs wq. I think the mechanism
has to be pluggable. The kernel needs to support both kthread and wq.
Or maybe even the 3rd option. Whatever it might be.
Via sysctl or something.
I suspect for some production workloads wq will perform better.
For the others it will be kthread.
Clearly kthread is more tunable, but not everyone would have
knowledge and desire to do the tunning.
We can argue what should be the default, but that's secondary.

> If this wan't urgent years ago (when it was NACK'd btw), it isn't
> urgent for 5.10 so I don't know why we are pushing so hard for
> this patch series to go in as-is right now.
>
> Please be patient and let's have a full discussion on this.

+1. This is the biggest change to the kernel networking in years.
Let's make it right.

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-02 23:15   ` Alexei Starovoitov
@ 2020-10-03  3:54     ` Eric Dumazet
  2020-10-03  4:17       ` Alexei Starovoitov
  0 siblings, 1 reply; 27+ messages in thread
From: Eric Dumazet @ 2020-10-03  3:54 UTC (permalink / raw)
  To: Alexei Starovoitov
  Cc: David Miller, Wei Wang, Network Development, Jakub Kicinski,
	Hannes Frederic Sowa, Paolo Abeni, Felix Fietkau

On Sat, Oct 3, 2020 at 1:15 AM Alexei Starovoitov
<alexei.starovoitov@gmail.com> wrote:
>
> On Fri, Oct 2, 2020 at 4:02 PM David Miller <davem@davemloft.net> wrote:
> >
> > From: Wei Wang <weiwan@google.com>
> > Date: Wed, 30 Sep 2020 12:21:35 -0700
> >
> >  ...
> > > 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.
> > ...
> >
> > I think we still need to discuss this some more.
> >
> > Jakub has some ideas and I honestly think the whole workqueue
> > approach hasn't been fully considered yet.
>
> I want to point out that it's not kthread vs wq. I think the mechanism
> has to be pluggable. The kernel needs to support both kthread and wq.
> Or maybe even the 3rd option. Whatever it might be.
> Via sysctl or something.
> I suspect for some production workloads wq will perform better.
> For the others it will be kthread.
> Clearly kthread is more tunable, but not everyone would have
> knowledge and desire to do the tunning.

The exact same arguments can be used against RPS and RFS

RPS went first, and was later augmented with RFS (with very different goals)

They both are opt-in

> We can argue what should be the default, but that's secondary.
>
> > If this wan't urgent years ago (when it was NACK'd btw), it isn't
> > urgent for 5.10 so I don't know why we are pushing so hard for
> > this patch series to go in as-is right now.
> >
> > Please be patient and let's have a full discussion on this.
>
> +1. This is the biggest change to the kernel networking in years.
> Let's make it right.

Sure. I do not think it is urgent.

This has been revived by Felix some weeks ago, and I think we were the
ones spending time on the proposed patch set.
Not giving feedback to Felix would have been "something not right".

We reviewed and tested Felix patches, and came up with something more flexible.

Sure, a WQ is already giving nice results on appliances, because there
you do not need strong isolation.
Would a kthread approach also work well on appliances ? Probably...

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

* Re: [PATCH net-next 0/5] implement kthread based napi poll
  2020-10-03  3:54     ` Eric Dumazet
@ 2020-10-03  4:17       ` Alexei Starovoitov
  0 siblings, 0 replies; 27+ messages in thread
From: Alexei Starovoitov @ 2020-10-03  4:17 UTC (permalink / raw)
  To: Eric Dumazet
  Cc: David Miller, Wei Wang, Network Development, Jakub Kicinski,
	Hannes Frederic Sowa, Paolo Abeni, Felix Fietkau

On Sat, Oct 03, 2020 at 05:54:38AM +0200, Eric Dumazet wrote:
> 
> Sure, a WQ is already giving nice results on appliances, because there
> you do not need strong isolation.
> Would a kthread approach also work well on appliances ? Probably...

Right. I think we're on the same page.
The only reason I'm bringing up multiple co-existing approaches now is to make
sure they are designed in from the start instead of as afterthought. Two
implementations already exist, but it doesn't look like that they can co-exist
in the kernel. Like this NAPI_STATE_THREADED bit in this patch set. Should we
burn that bit for kthread approach and another bit for workqueue based? 
I don't know. As the user of the feature I would be happy with any mechanism as
long as I can flip between them in runtime :) Just like RPS and RFS.

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

end of thread, other threads:[~2020-10-03  4:17 UTC | newest]

Thread overview: 27+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-09-30 19:21 [PATCH net-next 0/5] implement kthread based napi poll Wei Wang
2020-09-30 19:21 ` [PATCH net-next 1/5] net: implement threaded-able napi poll loop support Wei Wang
2020-09-30 19:21 ` [PATCH net-next 2/5] net: add sysfs attribute to control napi threaded mode Wei Wang
2020-09-30 19:21 ` [PATCH net-next 3/5] net: extract napi poll functionality to __napi_poll() Wei Wang
2020-09-30 19:21 ` [PATCH net-next 4/5] net: modify kthread handler to use __napi_poll() Wei Wang
2020-09-30 19:21 ` [PATCH net-next 5/5] net: improve napi threaded config Wei Wang
2020-10-01 10:01   ` Felix Fietkau
2020-10-01 17:01     ` Wei Wang
2020-10-01 17:11       ` Felix Fietkau
2020-10-01 18:03         ` Eric Dumazet
2020-10-01 18:37           ` Felix Fietkau
2020-10-01 19:24             ` Wei Wang
2020-10-01 20:48               ` Felix Fietkau
2020-10-01 22:42                 ` Wei Wang
2020-09-30 20:08 ` [PATCH net-next 0/5] implement kthread based napi poll Jakub Kicinski
2020-10-01  7:52   ` Eric Dumazet
2020-10-01 20:26     ` Jakub Kicinski
2020-10-01 22:12       ` Wei Wang
2020-10-01 23:46         ` Jakub Kicinski
2020-10-02  1:44           ` Wei Wang
2020-10-02 22:53             ` Jakub Kicinski
2020-10-02  7:56       ` Eric Dumazet
2020-10-02 22:49         ` Jakub Kicinski
2020-10-02 23:00 ` David Miller
2020-10-02 23:15   ` Alexei Starovoitov
2020-10-03  3:54     ` Eric Dumazet
2020-10-03  4:17       ` Alexei Starovoitov

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.