Subject: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

The RPS code and "deferred skb free" both send IPI/ function call
to a remote CPU in which a softirq is raised. This leads to a warning on
PREEMPT_RT because raising softiqrs from function call led to undesired
behaviour in the past. I had duct tape in RT for the "deferred skb free"
and Wander Lairson Costa reported the RPS case.

Changes:
- RFC…v1 https://lore.kernel.org/all/[email protected]

- Patch #2 has been removed. Removing the warning is still an option.

- There are two patches in the series:
- Patch #1 always creates backlog threads
- Patch #2 creates the backlog threads if requested at boot time,
mandatory on PREEMPT_RT.
So it is either or and I wanted to show how both look like.

- The kernel test robot reported a performance regression with
loopback (stress-ng --udp X --udp-ops Y) against the RFC version.
The regression is now avoided by using local-NAPI if backlog
processing is requested on the local CPU.

Sebastian


Subject: [PATCH net-next 1/2] net: Use SMP threads for backlog NAPI.

Backlog NAPI is a per-CPU NAPI struct only (with no device behind it)
used by drivers which don't do NAPI them self, RPS and parts of the
stack which need to avoid recursive deadlocks while processing a packet.

The non-NAPI driver use the CPU local backlog NAPI. If RPS is enabled
then a flow for the skb is computed and based on the flow the skb can be
enqueued on a remote CPU. Scheduling/ raising the softirq (for backlog's
NAPI) on the remote CPU isn't trivial because the softirq is only
scheduled on the local CPU and performed after the hardirq is done.
In order to schedule a softirq on the remote CPU, an IPI is sent to the
remote CPU which schedules the backlog-NAPI on the then local CPU.

On PREEMPT_RT interrupts are force-threaded. The soft interrupts are
raised within the interrupt thread and processed after the interrupt
handler completed still within the context of the interrupt thread. The
softirq is handled in the context where it originated.

With force-threaded interrupts enabled, ksoftirqd is woken up if a
softirq is raised from hardirq context. This is the case if it is raised
from an IPI. Additionally there is a warning on PREEMPT_RT if the
softirq is raised from the idle thread.
This was done for two reasons:
- With threaded interrupts the processing should happen in thread
context (where it originated) and ksoftirqd is the only thread for
this context if raised from hardirq. Using the currently running task
instead would "punish" a random task.
- Once ksoftirqd is active it consumes all further softirqs until it
stops running. This changed recently and is no longer the case.

Instead of keeping the backlog NAPI in ksoftirqd (in force-threaded/
PREEMPT_RT setups) I am proposing NAPI-threads for backlog.
The "proper" setup with threaded-NAPI is not doable because the threads
are not pinned to an individual CPU and can be modified by the user.
Additionally a dummy network device would have to be assigned. Also
CPU-hotplug has to be considered if additional CPUs show up.
All this can be probably done/ solved but the smpboot-threads already
provide this infrastructure.

Sending UDP packets over loopback expects that the packet is processed
within the call. Delaying it by handing it over to the thread hurts
performance. It is not beneficial to the outcome if the context switch
happens immediately after enqueue or after a while to process a few
packets in a batch.
There is no need to always use the thread if the backlog NAPI is
requested on the local CPU. This restores the loopback throuput. The
performance drops mostly to the same value after enabling RPS on the
loopback comparing the IPI and the tread result.

Create NAPI-threads for backlog. The thread runs the inner loop from
napi_threaded_poll(), the wait part is different. It checks for
NAPI_STATE_SCHED (the backlog NAPI can not be disabled).
Since there are now per-CPU threads for backlog the remote IPI for
signaling is not needed and can be removed. The NAPI for backlog can
always be scheduled as it ends in waking the corresponding thread.
Add a shortcut to ____napi_schedule() to use local poll_list (instead of
the thread) if backlog is requested for the local CPU.
Since "deferred skb free" use a similar IPI mechanism for signaling, it
is also using the backlog threads.

This makes NAPI threads mandatory for backlog and it can not be
disabled. The other visibile part with RPS (or backlog usage in general)
is that it becomes now visible in `top' if requested for a remote CPU
while earlier it would remain unaccounted.

Signed-off-by: Sebastian Andrzej Siewior <[email protected]>
---
include/linux/netdevice.h | 9 --
net/core/dev.c | 236 ++++++++++++++------------------------
net/core/net-procfs.c | 2 +-
net/core/skbuff.c | 4 +-
4 files changed, 90 insertions(+), 161 deletions(-)

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index 7e520c14eb8c6..aa6f0457cb5a5 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -3200,9 +3200,6 @@ struct softnet_data {
/* stats */
unsigned int processed;
unsigned int time_squeeze;
-#ifdef CONFIG_RPS
- struct softnet_data *rps_ipi_list;
-#endif

bool in_net_rx_action;
bool in_napi_threaded_poll;
@@ -3231,12 +3228,8 @@ struct softnet_data {
unsigned int input_queue_head ____cacheline_aligned_in_smp;

/* Elements below can be accessed between CPUs for RPS/RFS */
- call_single_data_t csd ____cacheline_aligned_in_smp;
- struct softnet_data *rps_ipi_next;
- unsigned int cpu;
unsigned int input_queue_tail;
#endif
- unsigned int received_rps;
unsigned int dropped;
struct sk_buff_head input_pkt_queue;
struct napi_struct backlog;
@@ -3244,9 +3237,7 @@ struct softnet_data {
/* Another possibly contended cache line */
spinlock_t defer_lock ____cacheline_aligned_in_smp;
int defer_count;
- int defer_ipi_scheduled;
struct sk_buff *defer_list;
- call_single_data_t defer_csd;
};

static inline void input_queue_head_incr(struct softnet_data *sd)
diff --git a/net/core/dev.c b/net/core/dev.c
index 606a366cc2095..7732fd91d2fd5 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -153,6 +153,7 @@
#include <linux/prandom.h>
#include <linux/once_lite.h>
#include <net/netdev_rx_queue.h>
+#include <linux/smpboot.h>

#include "dev.h"
#include "net-sysfs.h"
@@ -4409,6 +4410,7 @@ EXPORT_SYMBOL(__dev_direct_xmit);
/*************************************************************************
* Receiver routines
*************************************************************************/
+static DEFINE_PER_CPU(struct task_struct *, backlog_napi);

int netdev_max_backlog __read_mostly = 1000;
EXPORT_SYMBOL(netdev_max_backlog);
@@ -4441,6 +4443,9 @@ static inline void ____napi_schedule(struct softnet_data *sd,
*/
thread = READ_ONCE(napi->thread);
if (thread) {
+ if (thread == raw_cpu_read(backlog_napi))
+ goto use_local_napi;
+
/* Avoid doing set_bit() if the thread is in
* INTERRUPTIBLE state, cause napi_thread_wait()
* makes sure to proceed with napi polling
@@ -4453,6 +4458,7 @@ static inline void ____napi_schedule(struct softnet_data *sd,
}
}

+use_local_napi:
list_add_tail(&napi->poll_list, &sd->poll_list);
WRITE_ONCE(napi->list_owner, smp_processor_id());
/* If not called from net_rx_action()
@@ -4661,57 +4667,8 @@ bool rps_may_expire_flow(struct net_device *dev, u16 rxq_index,
EXPORT_SYMBOL(rps_may_expire_flow);

#endif /* CONFIG_RFS_ACCEL */
-
-/* Called from hardirq (IPI) context */
-static void rps_trigger_softirq(void *data)
-{
- struct softnet_data *sd = data;
-
- ____napi_schedule(sd, &sd->backlog);
- sd->received_rps++;
-}
-
#endif /* CONFIG_RPS */

-/* Called from hardirq (IPI) context */
-static void trigger_rx_softirq(void *data)
-{
- struct softnet_data *sd = data;
-
- __raise_softirq_irqoff(NET_RX_SOFTIRQ);
- smp_store_release(&sd->defer_ipi_scheduled, 0);
-}
-
-/*
- * After we queued a packet into sd->input_pkt_queue,
- * we need to make sure this queue is serviced soon.
- *
- * - If this is another cpu queue, link it to our rps_ipi_list,
- * and make sure we will process rps_ipi_list from net_rx_action().
- *
- * - If this is our own queue, NAPI schedule our backlog.
- * Note that this also raises NET_RX_SOFTIRQ.
- */
-static void napi_schedule_rps(struct softnet_data *sd)
-{
- struct softnet_data *mysd = this_cpu_ptr(&softnet_data);
-
-#ifdef CONFIG_RPS
- if (sd != mysd) {
- sd->rps_ipi_next = mysd->rps_ipi_list;
- mysd->rps_ipi_list = sd;
-
- /* If not called from net_rx_action() or napi_threaded_poll()
- * we have to raise NET_RX_SOFTIRQ.
- */
- if (!mysd->in_net_rx_action && !mysd->in_napi_threaded_poll)
- __raise_softirq_irqoff(NET_RX_SOFTIRQ);
- return;
- }
-#endif /* CONFIG_RPS */
- __napi_schedule_irqoff(&mysd->backlog);
-}
-
#ifdef CONFIG_NET_FLOW_LIMIT
int netdev_flow_limit_table_len __read_mostly = (1 << 12);
#endif
@@ -4784,7 +4741,7 @@ static int enqueue_to_backlog(struct sk_buff *skb, int cpu,
* We can use non atomic operation since we own the queue lock
*/
if (!__test_and_set_bit(NAPI_STATE_SCHED, &sd->backlog.state))
- napi_schedule_rps(sd);
+ __napi_schedule_irqoff(&sd->backlog);
goto enqueue;
}
reason = SKB_DROP_REASON_CPU_BACKLOG;
@@ -5899,63 +5856,12 @@ static void flush_all_backlogs(void)
cpus_read_unlock();
}

-static void net_rps_send_ipi(struct softnet_data *remsd)
-{
-#ifdef CONFIG_RPS
- while (remsd) {
- struct softnet_data *next = remsd->rps_ipi_next;
-
- if (cpu_online(remsd->cpu))
- smp_call_function_single_async(remsd->cpu, &remsd->csd);
- remsd = next;
- }
-#endif
-}
-
-/*
- * net_rps_action_and_irq_enable sends any pending IPI's for rps.
- * Note: called with local irq disabled, but exits with local irq enabled.
- */
-static void net_rps_action_and_irq_enable(struct softnet_data *sd)
-{
-#ifdef CONFIG_RPS
- struct softnet_data *remsd = sd->rps_ipi_list;
-
- if (remsd) {
- sd->rps_ipi_list = NULL;
-
- local_irq_enable();
-
- /* Send pending IPI's to kick RPS processing on remote cpus. */
- net_rps_send_ipi(remsd);
- } else
-#endif
- local_irq_enable();
-}
-
-static bool sd_has_rps_ipi_waiting(struct softnet_data *sd)
-{
-#ifdef CONFIG_RPS
- return sd->rps_ipi_list != NULL;
-#else
- return false;
-#endif
-}
-
static int process_backlog(struct napi_struct *napi, int quota)
{
struct softnet_data *sd = container_of(napi, struct softnet_data, backlog);
bool again = true;
int work = 0;

- /* Check if we have pending ipi, its better to send them now,
- * not waiting net_rx_action() end.
- */
- if (sd_has_rps_ipi_waiting(sd)) {
- local_irq_disable();
- net_rps_action_and_irq_enable(sd);
- }
-
napi->weight = READ_ONCE(dev_rx_weight);
while (again) {
struct sk_buff *skb;
@@ -5980,7 +5886,7 @@ static int process_backlog(struct napi_struct *napi, int quota)
* We can use a plain write instead of clear_bit(),
* and we dont need an smp_mb() memory barrier.
*/
- napi->state = 0;
+ napi->state = NAPIF_STATE_THREADED;
again = false;
} else {
skb_queue_splice_tail_init(&sd->input_pkt_queue,
@@ -6650,40 +6556,42 @@ static void skb_defer_free_flush(struct softnet_data *sd)
}
}

+static void napi_threaded_poll_loop(struct napi_struct *napi)
+{
+ struct softnet_data *sd;
+
+ for (;;) {
+ bool repoll = false;
+ void *have;
+
+ local_bh_disable();
+ sd = this_cpu_ptr(&softnet_data);
+ sd->in_napi_threaded_poll = true;
+
+ have = netpoll_poll_lock(napi);
+ __napi_poll(napi, &repoll);
+ netpoll_poll_unlock(have);
+
+ sd->in_napi_threaded_poll = false;
+ barrier();
+
+ skb_defer_free_flush(sd);
+ local_bh_enable();
+
+ if (!repoll)
+ break;
+
+ cond_resched();
+ }
+}
+
static int napi_threaded_poll(void *data)
{
struct napi_struct *napi = data;
- struct softnet_data *sd;
- void *have;

- while (!napi_thread_wait(napi)) {
- for (;;) {
- bool repoll = false;
+ while (!napi_thread_wait(napi))
+ napi_threaded_poll_loop(napi);

- local_bh_disable();
- sd = this_cpu_ptr(&softnet_data);
- sd->in_napi_threaded_poll = true;
-
- have = netpoll_poll_lock(napi);
- __napi_poll(napi, &repoll);
- netpoll_poll_unlock(have);
-
- sd->in_napi_threaded_poll = false;
- barrier();
-
- if (sd_has_rps_ipi_waiting(sd)) {
- local_irq_disable();
- net_rps_action_and_irq_enable(sd);
- }
- skb_defer_free_flush(sd);
- local_bh_enable();
-
- if (!repoll)
- break;
-
- cond_resched();
- }
- }
return 0;
}

@@ -6717,8 +6625,6 @@ static __latent_entropy void net_rx_action(struct softirq_action *h)
*/
if (!list_empty(&sd->poll_list))
goto start;
- if (!sd_has_rps_ipi_waiting(sd))
- goto end;
}
break;
}
@@ -6747,8 +6653,7 @@ static __latent_entropy void net_rx_action(struct softirq_action *h)
else
sd->in_net_rx_action = false;

- net_rps_action_and_irq_enable(sd);
-end:;
+ local_irq_enable();
}

struct netdev_adjacent {
@@ -11187,7 +11092,7 @@ static int dev_cpu_dead(unsigned int oldcpu)
struct sk_buff **list_skb;
struct sk_buff *skb;
unsigned int cpu;
- struct softnet_data *sd, *oldsd, *remsd = NULL;
+ struct softnet_data *sd, *oldsd;

local_irq_disable();
cpu = smp_processor_id();
@@ -11220,7 +11125,7 @@ static int dev_cpu_dead(unsigned int oldcpu)

list_del_init(&napi->poll_list);
if (napi->poll == process_backlog)
- napi->state = 0;
+ napi->state = NAPIF_STATE_THREADED;
else
____napi_schedule(sd, napi);
}
@@ -11228,13 +11133,6 @@ static int dev_cpu_dead(unsigned int oldcpu)
raise_softirq_irqoff(NET_TX_SOFTIRQ);
local_irq_enable();

-#ifdef CONFIG_RPS
- remsd = oldsd->rps_ipi_list;
- oldsd->rps_ipi_list = NULL;
-#endif
- /* send out pending IPI's on offline CPU */
- net_rps_send_ipi(remsd);
-
/* Process offline CPU's input_pkt_queue */
while ((skb = __skb_dequeue(&oldsd->process_queue))) {
netif_rx(skb);
@@ -11487,6 +11385,49 @@ static struct pernet_operations __net_initdata default_device_ops = {
*
*/

+static int backlog_napi_should_run(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+ struct napi_struct *napi = &sd->backlog;
+
+ if (READ_ONCE(sd->defer_list))
+ return 1;
+
+ return test_bit(NAPI_STATE_SCHED, &napi->state);
+}
+
+static void run_backlog_napi(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+
+ napi_threaded_poll_loop(&sd->backlog);
+}
+
+static void backlog_napi_setup(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+ struct napi_struct *napi = &sd->backlog;
+
+ napi->thread = this_cpu_read(backlog_napi);
+ set_bit(NAPI_STATE_THREADED, &napi->state);
+}
+
+static void backlog_napi_park(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+
+ skb_defer_free_flush(sd);
+}
+
+static struct smp_hotplug_thread backlog_threads = {
+ .store = &backlog_napi,
+ .thread_should_run = backlog_napi_should_run,
+ .thread_fn = run_backlog_napi,
+ .thread_comm = "backlog_napi/%u",
+ .setup = backlog_napi_setup,
+ .park = backlog_napi_park,
+};
+
/*
* This is called single threaded during boot, so no need
* to take the rtnl semaphore.
@@ -11527,17 +11468,14 @@ static int __init net_dev_init(void)
#endif
INIT_LIST_HEAD(&sd->poll_list);
sd->output_queue_tailp = &sd->output_queue;
-#ifdef CONFIG_RPS
- INIT_CSD(&sd->csd, rps_trigger_softirq, sd);
- sd->cpu = i;
-#endif
- INIT_CSD(&sd->defer_csd, trigger_rx_softirq, sd);
spin_lock_init(&sd->defer_lock);

init_gro_hash(&sd->backlog);
sd->backlog.poll = process_backlog;
sd->backlog.weight = weight_p;
+ INIT_LIST_HEAD(&sd->backlog.poll_list);
}
+ smpboot_register_percpu_thread(&backlog_threads);

dev_boot_phase = 0;

diff --git a/net/core/net-procfs.c b/net/core/net-procfs.c
index 09f7ed1a04e8a..086283cc8d47b 100644
--- a/net/core/net-procfs.c
+++ b/net/core/net-procfs.c
@@ -180,7 +180,7 @@ static int softnet_seq_show(struct seq_file *seq, void *v)
sd->processed, sd->dropped, sd->time_squeeze, 0,
0, 0, 0, 0, /* was fastroute */
0, /* was cpu_collision */
- sd->received_rps, flow_limit_count,
+ 0 /* was received_rps */, flow_limit_count,
input_qlen + process_qlen, (int)seq->index,
input_qlen, process_qlen);
return 0;
diff --git a/net/core/skbuff.c b/net/core/skbuff.c
index 2198979470ecf..166d0f300d170 100644
--- a/net/core/skbuff.c
+++ b/net/core/skbuff.c
@@ -6844,8 +6844,8 @@ nodefer: __kfree_skb(skb);
/* Make sure to trigger NET_RX_SOFTIRQ on the remote CPU
* if we are unlucky enough (this seems very unlikely).
*/
- if (unlikely(kick) && !cmpxchg(&sd->defer_ipi_scheduled, 0, 1))
- smp_call_function_single_async(cpu, &sd->defer_csd);
+ if (unlikely(kick))
+ __napi_schedule(&sd->backlog);
}

static void skb_splice_csum_page(struct sk_buff *skb, struct page *page,
--
2.42.0

2023-10-04 22:46:57

by Jakub Kicinski

[permalink] [raw]
Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On Fri, 29 Sep 2023 18:20:18 +0200 Sebastian Andrzej Siewior wrote:
> - Patch #2 has been removed. Removing the warning is still an option.
>
> - There are two patches in the series:
> - Patch #1 always creates backlog threads
> - Patch #2 creates the backlog threads if requested at boot time,
> mandatory on PREEMPT_RT.
> So it is either or and I wanted to show how both look like.
>
> - The kernel test robot reported a performance regression with
> loopback (stress-ng --udp X --udp-ops Y) against the RFC version.
> The regression is now avoided by using local-NAPI if backlog
> processing is requested on the local CPU.

Not what we asked for, and it doesn't apply.

Subject: [PATCH net-next 2/2] net: Allow to use SMP threads for backlog NAPI.

Backlog NAPI is a per-CPU NAPI struct only (with no device behind it)
used by drivers which don't do NAPI them self, RPS and parts of the
stack which need to avoid recursive deadlocks while processing a packet.

The non-NAPI driver use the CPU local backlog NAPI. If RPS is enabled
then a flow for the skb is computed and based on the flow the skb can be
enqueued on a remote CPU. Scheduling/ raising the softirq (for backlog's
NAPI) on the remote CPU isn't trivial because the softirq is only
scheduled on the local CPU and performed after the hardirq is done.
In order to schedule a softirq on the remote CPU, an IPI is sent to the
remote CPU which schedules the backlog-NAPI on the then local CPU.

On PREEMPT_RT interrupts are force-threaded. The soft interrupts are
raised within the interrupt thread and processed after the interrupt
handler completed still within the context of the interrupt thread. The
softirq is handled in the context where it originated.

With force-threaded interrupts enabled, ksoftirqd is woken up if a
softirq is raised from hardirq context. This is the case if it is raised
from an IPI. Additionally there is a warning on PREEMPT_RT if the
softirq is raised from the idle thread.
This was done for two reasons:
- With threaded interrupts the processing should happen in thread
context (where it originated) and ksoftirqd is the only thread for
this context if raised from hardirq. Using the currently running task
instead would "punish" a random task.
- Once ksoftirqd is active it consumes all further softirqs until it
stops running. This changed recently and is no longer the case.

Instead of keeping the backlog NAPI in ksoftirqd (in force-threaded/
PREEMPT_RT setups) I am proposing NAPI-threads for backlog.
The "proper" setup with threaded-NAPI is not doable because the threads
are not pinned to an individual CPU and can be modified by the user.
Additionally a dummy network device would have to be assigned. Also
CPU-hotplug has to be considered if additional CPUs show up.
All this can be probably done/ solved but the smpboot-threads already
provide this infrastructure.

Sending UDP packets over loopback expects that the packet is processed
within the call. Delaying it by handing it over to the thread hurts
performance. It is not beneficial to the outcome if the context switch
happens immediately after enqueue or after a while to process a few
packets in a batch.
There is no need to always use the thread if the backlog NAPI is
requested on the local CPU. This restores the loopback throuput. The
performance drops mostly to the same value after enabling RPS on the
loopback comparing the IPI and the tread result.

Create NAPI-threads for backlog if request during boot. The thread runs
the inner loop from napi_threaded_poll(), the wait part is different. It
checks for NAPI_STATE_SCHED (the backlog NAPI can not be disabled).
Since there are now per-CPU threads for backlog the remote IPI for
signaling is not needed and can be removed. The NAPI for backlog can
always be scheduled as it ends in waking the corresponding thread.
Add a shortcut to ____napi_schedule() to use local poll_list (instead of
the thread) if backlog is requested for the local CPU.
Since "deferred skb free" use a similar IPI mechanism for signaling, it
is also using the backlog threads.

This makes NAPI threads for backlog optional, it has to be enabled via
the boot argument "thread_backlog_napi". It is mandatory for PREEMPT_RT
to avoid the wakeup of ksoftirqd from the IPI.

Signed-off-by: Sebastian Andrzej Siewior <[email protected]>
---
include/linux/netdevice.h | 1 +
net/core/dev.c | 173 ++++++++++++++++++++++++++++++--------
net/core/skbuff.c | 4 +-
3 files changed, 139 insertions(+), 39 deletions(-)

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index 7e520c14eb8c6..bcaa967b8ddf9 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -3288,6 +3288,7 @@ static inline void dev_xmit_recursion_dec(void)
__this_cpu_dec(softnet_data.xmit.recursion);
}

+void net_trigger_defer_list_processing(unsigned int cpu);
void __netif_schedule(struct Qdisc *q);
void netif_schedule_queue(struct netdev_queue *txq);

diff --git a/net/core/dev.c b/net/core/dev.c
index 606a366cc2095..ec319e5dcb086 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -153,6 +153,7 @@
#include <linux/prandom.h>
#include <linux/once_lite.h>
#include <net/netdev_rx_queue.h>
+#include <linux/smpboot.h>

#include "dev.h"
#include "net-sysfs.h"
@@ -4409,6 +4410,46 @@ EXPORT_SYMBOL(__dev_direct_xmit);
/*************************************************************************
* Receiver routines
*************************************************************************/
+static DEFINE_PER_CPU(struct task_struct *, backlog_napi);
+
+#ifndef CONFIG_PREEMPT_RT
+
+static DEFINE_STATIC_KEY_FALSE(use_backlog_threads_key);
+
+static int __init setup_backlog_napi_threads(char *arg)
+{
+ static_branch_enable(&use_backlog_threads_key);
+ return 0;
+}
+early_param("thread_backlog_napi", setup_backlog_napi_threads);
+
+static bool use_backlog_threads(void)
+{
+ return static_branch_unlikely(&use_backlog_threads_key);
+}
+
+#else
+
+static bool use_backlog_threads(void)
+{
+ return true;
+}
+
+#endif
+
+void net_trigger_defer_list_processing(unsigned int cpu)
+{
+ struct softnet_data *sd;
+
+ sd = &per_cpu(softnet_data, cpu);
+ if (use_backlog_threads()) {
+ __napi_schedule(&sd->backlog);
+ return;
+ }
+
+ if (!cmpxchg(&sd->defer_ipi_scheduled, 0, 1))
+ smp_call_function_single_async(cpu, &sd->defer_csd);
+}

int netdev_max_backlog __read_mostly = 1000;
EXPORT_SYMBOL(netdev_max_backlog);
@@ -4441,6 +4482,9 @@ static inline void ____napi_schedule(struct softnet_data *sd,
*/
thread = READ_ONCE(napi->thread);
if (thread) {
+ if (use_backlog_threads() && thread == raw_cpu_read(backlog_napi))
+ goto use_local_napi;
+
/* Avoid doing set_bit() if the thread is in
* INTERRUPTIBLE state, cause napi_thread_wait()
* makes sure to proceed with napi polling
@@ -4453,6 +4497,7 @@ static inline void ____napi_schedule(struct softnet_data *sd,
}
}

+use_local_napi:
list_add_tail(&napi->poll_list, &sd->poll_list);
WRITE_ONCE(napi->list_owner, smp_processor_id());
/* If not called from net_rx_action()
@@ -4696,6 +4741,10 @@ static void napi_schedule_rps(struct softnet_data *sd)
{
struct softnet_data *mysd = this_cpu_ptr(&softnet_data);

+ if (use_backlog_threads()) {
+ __napi_schedule_irqoff(&sd->backlog);
+ return;
+ }
#ifdef CONFIG_RPS
if (sd != mysd) {
sd->rps_ipi_next = mysd->rps_ipi_list;
@@ -5921,7 +5970,7 @@ static void net_rps_action_and_irq_enable(struct softnet_data *sd)
#ifdef CONFIG_RPS
struct softnet_data *remsd = sd->rps_ipi_list;

- if (remsd) {
+ if (!use_backlog_threads() && remsd) {
sd->rps_ipi_list = NULL;

local_irq_enable();
@@ -5936,7 +5985,7 @@ static void net_rps_action_and_irq_enable(struct softnet_data *sd)
static bool sd_has_rps_ipi_waiting(struct softnet_data *sd)
{
#ifdef CONFIG_RPS
- return sd->rps_ipi_list != NULL;
+ return !use_backlog_threads() && sd->rps_ipi_list != NULL;
#else
return false;
#endif
@@ -5980,7 +6029,7 @@ static int process_backlog(struct napi_struct *napi, int quota)
* We can use a plain write instead of clear_bit(),
* and we dont need an smp_mb() memory barrier.
*/
- napi->state = 0;
+ napi->state &= NAPIF_STATE_THREADED;
again = false;
} else {
skb_queue_splice_tail_init(&sd->input_pkt_queue,
@@ -6650,40 +6699,42 @@ static void skb_defer_free_flush(struct softnet_data *sd)
}
}

+static void napi_threaded_poll_loop(struct napi_struct *napi)
+{
+ struct softnet_data *sd;
+
+ for (;;) {
+ bool repoll = false;
+ void *have;
+
+ local_bh_disable();
+ sd = this_cpu_ptr(&softnet_data);
+ sd->in_napi_threaded_poll = true;
+
+ have = netpoll_poll_lock(napi);
+ __napi_poll(napi, &repoll);
+ netpoll_poll_unlock(have);
+
+ sd->in_napi_threaded_poll = false;
+ barrier();
+
+ skb_defer_free_flush(sd);
+ local_bh_enable();
+
+ if (!repoll)
+ break;
+
+ cond_resched();
+ }
+}
+
static int napi_threaded_poll(void *data)
{
struct napi_struct *napi = data;
- struct softnet_data *sd;
- void *have;

- while (!napi_thread_wait(napi)) {
- for (;;) {
- bool repoll = false;
+ while (!napi_thread_wait(napi))
+ napi_threaded_poll_loop(napi);

- local_bh_disable();
- sd = this_cpu_ptr(&softnet_data);
- sd->in_napi_threaded_poll = true;
-
- have = netpoll_poll_lock(napi);
- __napi_poll(napi, &repoll);
- netpoll_poll_unlock(have);
-
- sd->in_napi_threaded_poll = false;
- barrier();
-
- if (sd_has_rps_ipi_waiting(sd)) {
- local_irq_disable();
- net_rps_action_and_irq_enable(sd);
- }
- skb_defer_free_flush(sd);
- local_bh_enable();
-
- if (!repoll)
- break;
-
- cond_resched();
- }
- }
return 0;
}

@@ -11220,7 +11271,7 @@ static int dev_cpu_dead(unsigned int oldcpu)

list_del_init(&napi->poll_list);
if (napi->poll == process_backlog)
- napi->state = 0;
+ napi->state &= NAPIF_STATE_THREADED;
else
____napi_schedule(sd, napi);
}
@@ -11228,12 +11279,14 @@ static int dev_cpu_dead(unsigned int oldcpu)
raise_softirq_irqoff(NET_TX_SOFTIRQ);
local_irq_enable();

+ if (!use_backlog_threads()) {
#ifdef CONFIG_RPS
- remsd = oldsd->rps_ipi_list;
- oldsd->rps_ipi_list = NULL;
+ remsd = oldsd->rps_ipi_list;
+ oldsd->rps_ipi_list = NULL;
#endif
- /* send out pending IPI's on offline CPU */
- net_rps_send_ipi(remsd);
+ /* send out pending IPI's on offline CPU */
+ net_rps_send_ipi(remsd);
+ }

/* Process offline CPU's input_pkt_queue */
while ((skb = __skb_dequeue(&oldsd->process_queue))) {
@@ -11487,6 +11540,49 @@ static struct pernet_operations __net_initdata default_device_ops = {
*
*/

+static int backlog_napi_should_run(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+ struct napi_struct *napi = &sd->backlog;
+
+ if (READ_ONCE(sd->defer_list))
+ return 1;
+
+ return test_bit(NAPI_STATE_SCHED, &napi->state);
+}
+
+static void run_backlog_napi(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+
+ napi_threaded_poll_loop(&sd->backlog);
+}
+
+static void backlog_napi_setup(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+ struct napi_struct *napi = &sd->backlog;
+
+ napi->thread = this_cpu_read(backlog_napi);
+ set_bit(NAPI_STATE_THREADED, &napi->state);
+}
+
+static void backlog_napi_park(unsigned int cpu)
+{
+ struct softnet_data *sd = per_cpu_ptr(&softnet_data, cpu);
+
+ skb_defer_free_flush(sd);
+}
+
+static struct smp_hotplug_thread backlog_threads = {
+ .store = &backlog_napi,
+ .thread_should_run = backlog_napi_should_run,
+ .thread_fn = run_backlog_napi,
+ .thread_comm = "backlog_napi/%u",
+ .setup = backlog_napi_setup,
+ .park = backlog_napi_park,
+};
+
/*
* This is called single threaded during boot, so no need
* to take the rtnl semaphore.
@@ -11537,7 +11633,10 @@ static int __init net_dev_init(void)
init_gro_hash(&sd->backlog);
sd->backlog.poll = process_backlog;
sd->backlog.weight = weight_p;
+ INIT_LIST_HEAD(&sd->backlog.poll_list);
}
+ if (use_backlog_threads())
+ smpboot_register_percpu_thread(&backlog_threads);

dev_boot_phase = 0;

diff --git a/net/core/skbuff.c b/net/core/skbuff.c
index 2198979470ecf..30cdb008dd65c 100644
--- a/net/core/skbuff.c
+++ b/net/core/skbuff.c
@@ -6844,8 +6844,8 @@ nodefer: __kfree_skb(skb);
/* Make sure to trigger NET_RX_SOFTIRQ on the remote CPU
* if we are unlucky enough (this seems very unlikely).
*/
- if (unlikely(kick) && !cmpxchg(&sd->defer_ipi_scheduled, 0, 1))
- smp_call_function_single_async(cpu, &sd->defer_csd);
+ if (unlikely(kick))
+ net_trigger_defer_list_processing(cpu);
}

static void skb_splice_csum_page(struct sk_buff *skb, struct page *page,
--
2.42.0

Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On 2023-10-04 15:46:09 [-0700], Jakub Kicinski wrote:
> On Fri, 29 Sep 2023 18:20:18 +0200 Sebastian Andrzej Siewior wrote:
> > - Patch #2 has been removed. Removing the warning is still an option.
> >
> > - There are two patches in the series:
> > - Patch #1 always creates backlog threads
> > - Patch #2 creates the backlog threads if requested at boot time,
> > mandatory on PREEMPT_RT.
> > So it is either or and I wanted to show how both look like.
> >
> > - The kernel test robot reported a performance regression with
> > loopback (stress-ng --udp X --udp-ops Y) against the RFC version.
> > The regression is now avoided by using local-NAPI if backlog
> > processing is requested on the local CPU.
>
> Not what we asked for, and it doesn't apply.

Apologies if I misunderstood. You said to make it optional which I did
with the static key in the second patch of this series. The first patch
is indeed not what we talked about I just to show what it would look
like now that there is no "delay" for backlog-NAPI on the local CPU.

If the optional part is okay then I can repost only that patch against
current net-next.

Sebastian

2023-10-10 01:09:56

by Jakub Kicinski

[permalink] [raw]
Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On Sat, 7 Oct 2023 17:59:57 +0200 Sebastian Andrzej Siewior wrote:
> Apologies if I misunderstood. You said to make it optional which I did
> with the static key in the second patch of this series. The first patch
> is indeed not what we talked about I just to show what it would look
> like now that there is no "delay" for backlog-NAPI on the local CPU.
>
> If the optional part is okay then I can repost only that patch against
> current net-next.

Do we have reason to believe nobody uses RPS?

Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

Sorry, getting back that late, I was traveling the last two weeks…

On 2023-10-09 18:09:37 [-0700], Jakub Kicinski wrote:
> On Sat, 7 Oct 2023 17:59:57 +0200 Sebastian Andrzej Siewior wrote:
> > Apologies if I misunderstood. You said to make it optional which I did
> > with the static key in the second patch of this series. The first patch
> > is indeed not what we talked about I just to show what it would look
> > like now that there is no "delay" for backlog-NAPI on the local CPU.
> >
> > If the optional part is okay then I can repost only that patch against
> > current net-next.
>
> Do we have reason to believe nobody uses RPS?

Not sure what you relate to. I would assume that RPS is used in general
on actual devices and not on loopback where backlog is used. But it is
just an assumption.
The performance drop, which I observed with RPS and stress-ng --udp, is
within the same range with threads and IPIs (based on memory). I can
re-run the test and provide actual numbers if you want.

Sebastian

2023-10-16 14:18:08

by Jakub Kicinski

[permalink] [raw]
Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On Mon, 16 Oct 2023 11:53:21 +0200 Sebastian Andrzej Siewior wrote:
> > Do we have reason to believe nobody uses RPS?
>
> Not sure what you relate to. I would assume that RPS is used in general
> on actual devices and not on loopback where backlog is used. But it is
> just an assumption.
> The performance drop, which I observed with RPS and stress-ng --udp, is
> within the same range with threads and IPIs (based on memory). I can
> re-run the test and provide actual numbers if you want.

I was asking about RPS because with your current series RPS processing
is forced into threads. IDK how well you can simulate the kind of
workload which requires RPS. I've seen it used mostly on proxyies
and gateways. For proxies Meta's experiments with threaded NAPI show
regressions across the board. So "force-threading" RPS will most likely
also cause regressions.

Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On 2023-10-16 07:17:56 [-0700], Jakub Kicinski wrote:
> On Mon, 16 Oct 2023 11:53:21 +0200 Sebastian Andrzej Siewior wrote:
> > > Do we have reason to believe nobody uses RPS?
> >
> > Not sure what you relate to. I would assume that RPS is used in general
> > on actual devices and not on loopback where backlog is used. But it is
> > just an assumption.
> > The performance drop, which I observed with RPS and stress-ng --udp, is
> > within the same range with threads and IPIs (based on memory). I can
> > re-run the test and provide actual numbers if you want.
>
> I was asking about RPS because with your current series RPS processing
> is forced into threads. IDK how well you can simulate the kind of
> workload which requires RPS. I've seen it used mostly on proxyies
> and gateways. For proxies Meta's experiments with threaded NAPI show
> regressions across the board. So "force-threading" RPS will most likely
> also cause regressions.

Understood.

Wandere/ Juri: Do you have any benchmark/ workload where you would see
whether RPS with IPI (now) vs RPS (this patch) shows any regression?

Sebastian

Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On 2023-10-16 16:53:39 [+0200], To Jakub Kicinski wrote:
> On 2023-10-16 07:17:56 [-0700], Jakub Kicinski wrote:
> > On Mon, 16 Oct 2023 11:53:21 +0200 Sebastian Andrzej Siewior wrote:
> > > > Do we have reason to believe nobody uses RPS?
> > >
> > > Not sure what you relate to. I would assume that RPS is used in general
> > > on actual devices and not on loopback where backlog is used. But it is
> > > just an assumption.
> > > The performance drop, which I observed with RPS and stress-ng --udp, is
> > > within the same range with threads and IPIs (based on memory). I can
> > > re-run the test and provide actual numbers if you want.
> >
> > I was asking about RPS because with your current series RPS processing
> > is forced into threads. IDK how well you can simulate the kind of
> > workload which requires RPS. I've seen it used mostly on proxyies
> > and gateways. For proxies Meta's experiments with threaded NAPI show
> > regressions across the board. So "force-threading" RPS will most likely
> > also cause regressions.
>
> Understood.
>
> Wandere/ Juri: Do you have any benchmark/ workload where you would see
> whether RPS with IPI (now) vs RPS (this patch) shows any regression?

So I poked offlist other RH people and I've been told that they hardly
ever test RPS since the NICs these days have RSS in hardware.

Sebastian

2023-10-31 11:37:26

by Wander Lairson Costa

[permalink] [raw]
Subject: Re: [PATCH net-next 0/2] net: Use SMP threads for backlog NAPI (or optional).

On Tue, Oct 31, 2023 at 7:14 AM Sebastian Andrzej Siewior
<[email protected]> wrote:
>
> On 2023-10-16 16:53:39 [+0200], To Jakub Kicinski wrote:
> > On 2023-10-16 07:17:56 [-0700], Jakub Kicinski wrote:
> > > On Mon, 16 Oct 2023 11:53:21 +0200 Sebastian Andrzej Siewior wrote:
> > > > > Do we have reason to believe nobody uses RPS?
> > > >
> > > > Not sure what you relate to. I would assume that RPS is used in general
> > > > on actual devices and not on loopback where backlog is used. But it is
> > > > just an assumption.
> > > > The performance drop, which I observed with RPS and stress-ng --udp, is
> > > > within the same range with threads and IPIs (based on memory). I can
> > > > re-run the test and provide actual numbers if you want.
> > >
> > > I was asking about RPS because with your current series RPS processing
> > > is forced into threads. IDK how well you can simulate the kind of
> > > workload which requires RPS. I've seen it used mostly on proxyies
> > > and gateways. For proxies Meta's experiments with threaded NAPI show
> > > regressions across the board. So "force-threading" RPS will most likely
> > > also cause regressions.
> >
> > Understood.
> >
> > Wandere/ Juri: Do you have any benchmark/ workload where you would see
> > whether RPS with IPI (now) vs RPS (this patch) shows any regression?
>
> So I poked offlist other RH people and I've been told that they hardly
> ever test RPS since the NICs these days have RSS in hardware.

Sorry, Juri is in PTO and I am just back from sick leave and still
catching up. I've been contacting some QE people, but so far it is
like you said, no stress test for RPS. If I have some news, I let you
know.


>
> Sebastian
>