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

The idea of moving the napi poll process out of softirq context to a
kernel thread based context is not new.
Paolo Abeni and Hannes Frederic Sowa 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. 

Changes since v1:
Replaced kthread_create() with kthread_run() in patch 5 as suggested by
Felix Fietkau.

Changes since RFC:
Renamed the kthreads to be napi/<dev>-<napi_id> in patch 5 as suggested
by Hannes Frederic Sowa.

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            | 143 +++++++++++++++++++++++++++++++++++---
 net/core/net-sysfs.c      | 100 ++++++++++++++++++++++++++
 3 files changed, 239 insertions(+), 9 deletions(-)

-- 
2.28.0.806.g8561365e88-goog


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

* [PATCH net-next v2 1/5] net: implement threaded-able napi poll loop support
  2020-10-02 22:25 [PATCH net-next v2 0/5] implement kthread based napi poll Wei Wang
@ 2020-10-02 22:25 ` Wei Wang
  2020-10-02 22:25 ` [PATCH net-next v2 2/5] net: add sysfs attribute to control napi threaded mode Wei Wang
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Wei Wang @ 2020-10-02 22:25 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Paolo Abeni <pabeni@redhat.com>

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

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

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

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

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index 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.806.g8561365e88-goog


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

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

From: Paolo Abeni <pabeni@redhat.com>

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

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

Signed-off-by: Paolo Abeni <pabeni@redhat.com>
Signed-off-by: Hannes Frederic Sowa <hannes@stressinduktion.org>
Signed-off-by: Wei Wang <weiwan@google.com>
---
 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 94fff0700bdd..df8dd25e5e4b 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.806.g8561365e88-goog


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

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

From: Felix Fietkau <nbd@nbd.name>

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

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

diff --git a/net/core/dev.c b/net/core/dev.c
index 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.806.g8561365e88-goog


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

* [PATCH net-next v2 4/5] net: modify kthread handler to use __napi_poll()
  2020-10-02 22:25 [PATCH net-next v2 0/5] implement kthread based napi poll Wei Wang
                   ` (2 preceding siblings ...)
  2020-10-02 22:25 ` [PATCH net-next v2 3/5] net: extract napi poll functionality to __napi_poll() Wei Wang
@ 2020-10-02 22:25 ` Wei Wang
  2020-10-02 22:25 ` [PATCH net-next v2 5/5] net: improve napi threaded config Wei Wang
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Wei Wang @ 2020-10-02 22:25 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

From: Jakub Kicinski <kuba@kernel.org> 

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

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

diff --git a/net/core/dev.c b/net/core/dev.c
index 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.806.g8561365e88-goog


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

* [PATCH net-next v2 5/5] net: improve napi threaded config
  2020-10-02 22:25 [PATCH net-next v2 0/5] implement kthread based napi poll Wei Wang
                   ` (3 preceding siblings ...)
  2020-10-02 22:25 ` [PATCH net-next v2 4/5] net: modify kthread handler to use __napi_poll() Wei Wang
@ 2020-10-02 22:25 ` Wei Wang
  2020-10-03  9:57 ` [PATCH net-next v2 0/5] implement kthread based napi poll David Laight
  2020-10-03 10:56 ` David Laight
  6 siblings, 0 replies; 8+ messages in thread
From: Wei Wang @ 2020-10-02 22:25 UTC (permalink / raw)
  To: David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau, Wei Wang

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

Signed-off-by: Wei Wang <weiwan@google.com>
---
Changes since v1:
replaced kthread_create() with kthread_run()

Changes since RFC:
changed the thread name to napi/<dev>-<napi-id>

 net/core/dev.c       | 53 ++++++++++++++++++++++++++------------------
 net/core/net-sysfs.c |  9 +++-----
 2 files changed, 35 insertions(+), 27 deletions(-)

diff --git a/net/core/dev.c b/net/core/dev.c
index b4f33e442b5e..e89a7f869c73 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -1490,17 +1490,28 @@ 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;
+
+	/* Create and wake up the kthread once to put it in
+	 * TASK_INTERRUPTIBLE mode to avoid the blocked task
+	 * warning and work with loadavg.
+	 */
+	n->thread = kthread_run(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_run 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 +1543,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 +1593,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 +4276,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 +6696,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 +6759,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 +6880,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 df8dd25e5e4b..1e24c1e81ad8 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.806.g8561365e88-goog


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

* RE: [PATCH net-next v2 0/5] implement kthread based napi poll
  2020-10-02 22:25 [PATCH net-next v2 0/5] implement kthread based napi poll Wei Wang
                   ` (4 preceding siblings ...)
  2020-10-02 22:25 ` [PATCH net-next v2 5/5] net: improve napi threaded config Wei Wang
@ 2020-10-03  9:57 ` David Laight
  2020-10-03 10:56 ` David Laight
  6 siblings, 0 replies; 8+ messages in thread
From: David Laight @ 2020-10-03  9:57 UTC (permalink / raw)
  To: 'Wei Wang', David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau

From: Wei Wang
> Sent: 02 October 2020 23:25
> 
> 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.

What default scheduler priority are you planning to use?

The current 'softint' is (effectively) slightly higher priority
than the highest RT priority.

I think you need to use a 'middle' priority RT process so that
applications can decide whether they need to be higher/lower
priority than the network code.

But then you hit the problem that the scheduler gives RT
processes a very 'sticky' cpu affinity.
IIRC they don't ever get 'stolen' by an idle cpu, so only
migrate when the scheduler for the cpu they last ran on
decides to run something of a higher priority.
This is problematic if a low priority process in looping
in kernel space somewhere (without a cond_resched()).
(I've been running ftrace...)

Given that the napi cpu cycles have to happen sometime,
the biggest problem I found with the current softint
implementation is that a hardware interrupt can happen
while an application is holding a (user space) mutex.
This will block other application threads from acquiring
the mutex until not only the hardware interrupt
completes, but also all the associated softint (typically
napi and rcu) processing has completed.
This can take a while!
Moving the 'softint' processing to a separate thread
will allow the interrupted process to release the mutex
and all the application threads continue.

I guess the downside of using a thread is that the
data needed is likely to be in the wrong cache.

	David

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

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

* RE: [PATCH net-next v2 0/5] implement kthread based napi poll
  2020-10-02 22:25 [PATCH net-next v2 0/5] implement kthread based napi poll Wei Wang
                   ` (5 preceding siblings ...)
  2020-10-03  9:57 ` [PATCH net-next v2 0/5] implement kthread based napi poll David Laight
@ 2020-10-03 10:56 ` David Laight
  6 siblings, 0 replies; 8+ messages in thread
From: David Laight @ 2020-10-03 10:56 UTC (permalink / raw)
  To: 'Wei Wang', David S . Miller, netdev
  Cc: Eric Dumazet, Jakub Kicinski, Paolo Abeni, Hannes Frederic Sowa,
	Felix Fietkau

From: Wei Wang
> Sent: 02 October 2020 23:25
> 
> 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.

I didn't spot anything that makes this continue to work?

static inline bool netdev_xmit_more(void)
{
        return __this_cpu_read(softnet_data.xmit.more);
}

I assume it normally relies on the softint code running with
pre-emption disabled.

(It also needs a level of indirection.
xmit.more is only set if more packets are queued when the tx
call is done.
I've seen a workload that manages to repeatedly add an extra
packet while the tx setup is in progress.)

	David

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

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

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

Thread overview: 8+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-10-02 22:25 [PATCH net-next v2 0/5] implement kthread based napi poll Wei Wang
2020-10-02 22:25 ` [PATCH net-next v2 1/5] net: implement threaded-able napi poll loop support Wei Wang
2020-10-02 22:25 ` [PATCH net-next v2 2/5] net: add sysfs attribute to control napi threaded mode Wei Wang
2020-10-02 22:25 ` [PATCH net-next v2 3/5] net: extract napi poll functionality to __napi_poll() Wei Wang
2020-10-02 22:25 ` [PATCH net-next v2 4/5] net: modify kthread handler to use __napi_poll() Wei Wang
2020-10-02 22:25 ` [PATCH net-next v2 5/5] net: improve napi threaded config Wei Wang
2020-10-03  9:57 ` [PATCH net-next v2 0/5] implement kthread based napi poll David Laight
2020-10-03 10:56 ` David Laight

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.