linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs
@ 2022-10-04 15:05 Valentin Schneider
  2022-10-04 15:05 ` [PATCH v4 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
                   ` (5 more replies)
  0 siblings, 6 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-10-04 15:05 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Hi folks,

I haven't sent an update for this in a while, but the issue has risen again in
some other environment so I'm getting more reasons to push this out.

Revisions
=========

RFCv3 -> v4
+++++++++++

o Rebase onto v6.0
o Split into more patches for reviewability
o Take dying workers out of the pool->workers as suggested by Lai

RFCv2 -> RFCv3
++++++++++++++

o Rebase onto v5.19
o Add new patch (1/3) around accessing wq_unbound_cpumask

o Prevent WORKER_DIE workers for kfree()'ing themselves before the idle reaper
  gets to handle them (Tejun)

  Bit of an aside on that: I've been struggling to convince myself this can
  happen due to spurious wakeups and would like some help here.

  Idle workers are TASK_UNINTERRUPTIBLE, so they can't be woken up by
  signals. That state is set *under* pool->lock, and all wakeups (before this
  patch) are also done while holding pool->lock.
  
  wake_up_worker() is done under pool->lock AND only wakes a worker on the
  pool->idle_list. Thus the to-be-woken worker *cannot* have WORKER_DIE, though
  it could gain it *after* being woken but *before* it runs, e.g.:
                          
  LOCK pool->lock
  wake_up_worker(pool)
      wake_up_process(p)
  UNLOCK pool->lock
                          idle_reaper_fn()
                            LOCK pool->lock
                            destroy_worker(worker, list);
			    UNLOCK pool->lock
			                            worker_thread()
						      goto woke_up;
                                                      LOCK pool->lock
						      READ worker->flags & WORKER_DIE
                                                          UNLOCK pool->lock
                                                          ...
						          kfree(worker);
                            reap_worker(worker);
			        // Uh-oh
			  
  ... But IMO that's not a spurious wakeup, that's a concurrency issue. I don't
  see any spurious/unexpected worker wakeup happening once a worker is off the
  pool->idle_list.
  

RFCv1 -> RFCv2
++++++++++++++

o Change the pool->timer into a delayed_work to have a sleepable context for
  unbinding kworkers

Cheers,
Valentin

Lai Jiangshan (1):
  workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex

Valentin Schneider (3):
  workqueue: Factorize unbind/rebind_workers() logic
  workqueue: Convert the idle_timer to a delayed_work
  workqueue: Unbind workers before sending them to exit()

 kernel/workqueue.c | 195 +++++++++++++++++++++++++++++++--------------
 1 file changed, 136 insertions(+), 59 deletions(-)

--
2.31.1


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

* [PATCH v4 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex
  2022-10-04 15:05 [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
@ 2022-10-04 15:05 ` Valentin Schneider
  2022-10-04 15:05 ` [PATCH v4 2/4] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-10-04 15:05 UTC (permalink / raw)
  To: linux-kernel
  Cc: Lai Jiangshan, Tejun Heo, Lai Jiangshan, Peter Zijlstra,
	Frederic Weisbecker, Juri Lelli, Phil Auld, Marcelo Tosatti

From: Lai Jiangshan <jiangshan.ljs@antgroup.com>

When unbind_workers() reads wq_unbound_cpumask to set the affinity of
freshly-unbound kworkers, it only holds wq_pool_attach_mutex. This isn't
sufficient as wq_unbound_cpumask is only protected by wq_pool_mutex.

Make wq_unbound_cpumask protected with wq_pool_attach_mutex and also
remove the need of temporary saved_cpumask.

Fixes: 10a5a651e3af ("workqueue: Restrict kworker in the offline CPU pool running on housekeeping CPUs")
Reported-by: Valentin Schneider <vschneid@redhat.com>
Signed-off-by: Lai Jiangshan <jiangshan.ljs@antgroup.com>
---
 kernel/workqueue.c | 41 ++++++++++++++++-------------------------
 1 file changed, 16 insertions(+), 25 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 7cd5f5e7e0a1..8e21c352c155 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -326,7 +326,7 @@ static struct rcuwait manager_wait = __RCUWAIT_INITIALIZER(manager_wait);
 static LIST_HEAD(workqueues);		/* PR: list of all workqueues */
 static bool workqueue_freezing;		/* PL: have wqs started freezing? */
 
-/* PL: allowable cpus for unbound wqs and work items */
+/* PL&A: allowable cpus for unbound wqs and work items */
 static cpumask_var_t wq_unbound_cpumask;
 
 /* CPU where unbound work was last round robin scheduled from this CPU */
@@ -3952,7 +3952,8 @@ static void apply_wqattrs_cleanup(struct apply_wqattrs_ctx *ctx)
 /* allocate the attrs and pwqs for later installation */
 static struct apply_wqattrs_ctx *
 apply_wqattrs_prepare(struct workqueue_struct *wq,
-		      const struct workqueue_attrs *attrs)
+		      const struct workqueue_attrs *attrs,
+		      const cpumask_var_t unbound_cpumask)
 {
 	struct apply_wqattrs_ctx *ctx;
 	struct workqueue_attrs *new_attrs, *tmp_attrs;
@@ -3968,14 +3969,15 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
 		goto out_free;
 
 	/*
-	 * Calculate the attrs of the default pwq.
+	 * Calculate the attrs of the default pwq with unbound_cpumask
+	 * which is wq_unbound_cpumask or to set to wq_unbound_cpumask.
 	 * If the user configured cpumask doesn't overlap with the
 	 * wq_unbound_cpumask, we fallback to the wq_unbound_cpumask.
 	 */
 	copy_workqueue_attrs(new_attrs, attrs);
-	cpumask_and(new_attrs->cpumask, new_attrs->cpumask, wq_unbound_cpumask);
+	cpumask_and(new_attrs->cpumask, new_attrs->cpumask, unbound_cpumask);
 	if (unlikely(cpumask_empty(new_attrs->cpumask)))
-		cpumask_copy(new_attrs->cpumask, wq_unbound_cpumask);
+		cpumask_copy(new_attrs->cpumask, unbound_cpumask);
 
 	/*
 	 * We may create multiple pwqs with differing cpumasks.  Make a
@@ -4072,7 +4074,7 @@ static int apply_workqueue_attrs_locked(struct workqueue_struct *wq,
 		wq->flags &= ~__WQ_ORDERED;
 	}
 
-	ctx = apply_wqattrs_prepare(wq, attrs);
+	ctx = apply_wqattrs_prepare(wq, attrs, wq_unbound_cpumask);
 	if (!ctx)
 		return -ENOMEM;
 
@@ -5334,7 +5336,7 @@ void thaw_workqueues(void)
 }
 #endif /* CONFIG_FREEZER */
 
-static int workqueue_apply_unbound_cpumask(void)
+static int workqueue_apply_unbound_cpumask(const cpumask_var_t unbound_cpumask)
 {
 	LIST_HEAD(ctxs);
 	int ret = 0;
@@ -5350,7 +5352,7 @@ static int workqueue_apply_unbound_cpumask(void)
 		if (wq->flags & __WQ_ORDERED)
 			continue;
 
-		ctx = apply_wqattrs_prepare(wq, wq->unbound_attrs);
+		ctx = apply_wqattrs_prepare(wq, wq->unbound_attrs, unbound_cpumask);
 		if (!ctx) {
 			ret = -ENOMEM;
 			break;
@@ -5365,6 +5367,11 @@ static int workqueue_apply_unbound_cpumask(void)
 		apply_wqattrs_cleanup(ctx);
 	}
 
+	if (!ret) {
+		mutex_lock(&wq_pool_attach_mutex);
+		cpumask_copy(wq_unbound_cpumask, unbound_cpumask);
+		mutex_unlock(&wq_pool_attach_mutex);
+	}
 	return ret;
 }
 
@@ -5383,7 +5390,6 @@ static int workqueue_apply_unbound_cpumask(void)
 int workqueue_set_unbound_cpumask(cpumask_var_t cpumask)
 {
 	int ret = -EINVAL;
-	cpumask_var_t saved_cpumask;
 
 	/*
 	 * Not excluding isolated cpus on purpose.
@@ -5397,23 +5403,8 @@ int workqueue_set_unbound_cpumask(cpumask_var_t cpumask)
 			goto out_unlock;
 		}
 
-		if (!zalloc_cpumask_var(&saved_cpumask, GFP_KERNEL)) {
-			ret = -ENOMEM;
-			goto out_unlock;
-		}
-
-		/* save the old wq_unbound_cpumask. */
-		cpumask_copy(saved_cpumask, wq_unbound_cpumask);
-
-		/* update wq_unbound_cpumask at first and apply it to wqs. */
-		cpumask_copy(wq_unbound_cpumask, cpumask);
-		ret = workqueue_apply_unbound_cpumask();
-
-		/* restore the wq_unbound_cpumask when failed. */
-		if (ret < 0)
-			cpumask_copy(wq_unbound_cpumask, saved_cpumask);
+		ret = workqueue_apply_unbound_cpumask(cpumask);
 
-		free_cpumask_var(saved_cpumask);
 out_unlock:
 		apply_wqattrs_unlock();
 	}
-- 
2.31.1


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

* [PATCH v4 2/4] workqueue: Factorize unbind/rebind_workers() logic
  2022-10-04 15:05 [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  2022-10-04 15:05 ` [PATCH v4 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
@ 2022-10-04 15:05 ` Valentin Schneider
  2022-10-04 15:05 ` [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work Valentin Schneider
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-10-04 15:05 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Later patches will reuse this code, move it into reusable functions.

Signed-off-by: Valentin Schneider <vschneid@redhat.com>
---
 kernel/workqueue.c | 33 +++++++++++++++++++++------------
 1 file changed, 21 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 8e21c352c155..8185a42848c5 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1972,6 +1972,23 @@ static struct worker *create_worker(struct worker_pool *pool)
 	return NULL;
 }
 
+static void unbind_worker(struct worker *worker)
+{
+	lockdep_assert_held(&wq_pool_attach_mutex);
+
+	kthread_set_per_cpu(worker->task, -1);
+	if (cpumask_intersects(wq_unbound_cpumask, cpu_active_mask))
+		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, wq_unbound_cpumask) < 0);
+	else
+		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, cpu_possible_mask) < 0);
+}
+
+static void rebind_worker(struct worker *worker, struct worker_pool *pool)
+{
+	kthread_set_per_cpu(worker->task, pool->cpu);
+	WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask) < 0);
+}
+
 /**
  * destroy_worker - destroy a workqueue worker
  * @worker: worker to be destroyed
@@ -5008,13 +5025,8 @@ static void unbind_workers(int cpu)
 
 		raw_spin_unlock_irq(&pool->lock);
 
-		for_each_pool_worker(worker, pool) {
-			kthread_set_per_cpu(worker->task, -1);
-			if (cpumask_intersects(wq_unbound_cpumask, cpu_active_mask))
-				WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, wq_unbound_cpumask) < 0);
-			else
-				WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, cpu_possible_mask) < 0);
-		}
+		for_each_pool_worker(worker, pool)
+			unbind_worker(worker);
 
 		mutex_unlock(&wq_pool_attach_mutex);
 	}
@@ -5039,11 +5051,8 @@ static void rebind_workers(struct worker_pool *pool)
 	 * of all workers first and then clear UNBOUND.  As we're called
 	 * from CPU_ONLINE, the following shouldn't fail.
 	 */
-	for_each_pool_worker(worker, pool) {
-		kthread_set_per_cpu(worker->task, pool->cpu);
-		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task,
-						  pool->attrs->cpumask) < 0);
-	}
+	for_each_pool_worker(worker, pool)
+		rebind_worker(worker, pool);
 
 	raw_spin_lock_irq(&pool->lock);
 
-- 
2.31.1


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

* [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work
  2022-10-04 15:05 [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  2022-10-04 15:05 ` [PATCH v4 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
  2022-10-04 15:05 ` [PATCH v4 2/4] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
@ 2022-10-04 15:05 ` Valentin Schneider
  2022-10-31 18:49   ` Tejun Heo
  2022-10-04 15:05 ` [PATCH v4 4/4] workqueue: Unbind workers before sending them to exit() Valentin Schneider
                   ` (2 subsequent siblings)
  5 siblings, 1 reply; 11+ messages in thread
From: Valentin Schneider @ 2022-10-04 15:05 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

A later patch will require a sleepable context in the idle worker timeout
function. Converting worker_pool.idle_timer to a delayed_work gives us just
that.

One caveat is that we need to be careful about re-queuing the dwork from
its callback function. Lai expressed concerns about overtly violating
documented locking rules, but extra locking is required around delaying the
dwork, else a worker thread adding itself to the idle_list might push the
dwork further back (IDLE_WORKER_TIMEOUT) than the work callback would (next
idle worker expiry).

No change in functionality intended.

Signed-off-by: Valentin Schneider <vschneid@redhat.com>
---
 kernel/workqueue.c | 41 +++++++++++++++++++++++++++++------------
 1 file changed, 29 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 8185a42848c5..436b1dbdf9ff 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -167,9 +167,9 @@ struct worker_pool {
 	int			nr_workers;	/* L: total number of workers */
 	int			nr_idle;	/* L: currently idle workers */
 
-	struct list_head	idle_list;	/* L: list of idle workers */
-	struct timer_list	idle_timer;	/* L: worker idle timeout */
-	struct timer_list	mayday_timer;	/* L: SOS timer for workers */
+	struct list_head	idle_list;	  /* L: list of idle workers */
+	struct delayed_work     idle_reaper_work; /* L: worker idle timeout */
+	struct timer_list	mayday_timer;	  /* L: SOS timer for workers */
 
 	/* a workers is either on busy_hash or idle_list, or the manager */
 	DECLARE_HASHTABLE(busy_hash, BUSY_WORKER_HASH_ORDER);
@@ -1806,8 +1806,10 @@ static void worker_enter_idle(struct worker *worker)
 	/* idle_list is LIFO */
 	list_add(&worker->entry, &pool->idle_list);
 
-	if (too_many_workers(pool) && !timer_pending(&pool->idle_timer))
-		mod_timer(&pool->idle_timer, jiffies + IDLE_WORKER_TIMEOUT);
+	if (too_many_workers(pool) && !delayed_work_pending(&pool->idle_reaper_work))
+		mod_delayed_work(system_unbound_wq,
+				 &pool->idle_reaper_work,
+				 IDLE_WORKER_TIMEOUT);
 
 	/* Sanity check nr_running. */
 	WARN_ON_ONCE(pool->nr_workers == pool->nr_idle && pool->nr_running);
@@ -2019,22 +2021,37 @@ static void destroy_worker(struct worker *worker)
 	wake_up_process(worker->task);
 }
 
-static void idle_worker_timeout(struct timer_list *t)
+/*
+ * idle_reaper_fn - reap workers that have been idle for too long.
+ *
+ * The delayed_work is only ever modified under raw_spin_lock_irq(pool->lock).
+ */
+static void idle_reaper_fn(struct work_struct *work)
 {
-	struct worker_pool *pool = from_timer(pool, t, idle_timer);
+	struct delayed_work *dwork = to_delayed_work(work);
+	struct worker_pool *pool = container_of(dwork, struct worker_pool, idle_reaper_work);
 
 	raw_spin_lock_irq(&pool->lock);
 
 	while (too_many_workers(pool)) {
 		struct worker *worker;
 		unsigned long expires;
+		unsigned long now = jiffies;
 
-		/* idle_list is kept in LIFO order, check the last one */
+		/* idle_list is kept in LIFO order, check the oldest entry */
 		worker = list_entry(pool->idle_list.prev, struct worker, entry);
 		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
 
-		if (time_before(jiffies, expires)) {
-			mod_timer(&pool->idle_timer, expires);
+		/*
+		 * Careful: queueing a work item from here can and will cause a
+		 * self-deadlock when dealing with an unbound pool. However,
+		 * here the delay *cannot* be zero, so the queuing will
+		 * happen in the timer callback.
+		 */
+		if (time_before(now, expires)) {
+			mod_delayed_work(system_unbound_wq,
+					 &pool->idle_reaper_work,
+					 expires - now);
 			break;
 		}
 
@@ -3478,7 +3495,7 @@ static int init_worker_pool(struct worker_pool *pool)
 	INIT_LIST_HEAD(&pool->idle_list);
 	hash_init(pool->busy_hash);
 
-	timer_setup(&pool->idle_timer, idle_worker_timeout, TIMER_DEFERRABLE);
+	INIT_DEFERRABLE_WORK(&pool->idle_reaper_work, idle_reaper_fn);
 
 	timer_setup(&pool->mayday_timer, pool_mayday_timeout, 0);
 
@@ -3625,7 +3642,7 @@ static void put_unbound_pool(struct worker_pool *pool)
 		wait_for_completion(pool->detach_completion);
 
 	/* shut down the timers */
-	del_timer_sync(&pool->idle_timer);
+	cancel_delayed_work_sync(&pool->idle_reaper_work);
 	del_timer_sync(&pool->mayday_timer);
 
 	/* RCU protected to allow dereferences from get_work_pool() */
-- 
2.31.1


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

* [PATCH v4 4/4] workqueue: Unbind workers before sending them to exit()
  2022-10-04 15:05 [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
                   ` (2 preceding siblings ...)
  2022-10-04 15:05 ` [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work Valentin Schneider
@ 2022-10-04 15:05 ` Valentin Schneider
       [not found] ` <20221005010832.1934-1-hdanton@sina.com>
  2022-10-25  9:42 ` [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  5 siblings, 0 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-10-04 15:05 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

It has been reported that isolated CPUs can suffer from interference due to
per-CPU kworkers waking up just to die.

A surge of workqueue activity during initial setup of a latency-sensitive
application (refresh_vm_stats() being one of the culprits) can cause extra
per-CPU kworkers to be spawned. Then, said latency-sensitive task can be
running merrily on an isolated CPU only to be interrupted sometime later by
a kworker marked for death (cf. IDLE_WORKER_TIMEOUT, 5 minutes after last
kworker activity).

Prevent this by affining kworkers to the wq_unbound_cpumask (which doesn't
contain isolated CPUs, cf. HK_TYPE_WQ) before waking them up after marking
them with WORKER_DIE.

Changing the affinity does require a sleepable context, leverage the newly
introduced pool->idle_reaper_work to get that.
Remove dying workers from pool->workers and keep track of them in a
separate list. This intentionally prevents for_each_loop_worker() from
iterating over workers that are marked for death.

Signed-off-by: Valentin Schneider <vschneid@redhat.com>
---
 kernel/workqueue.c | 80 ++++++++++++++++++++++++++++++++++++++++------
 1 file changed, 70 insertions(+), 10 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 436b1dbdf9ff..714db7df7105 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -177,6 +177,7 @@ struct worker_pool {
 
 	struct worker		*manager;	/* L: purely informational */
 	struct list_head	workers;	/* A: attached workers */
+	struct list_head        dying_workers;  /* A: workers about to die */
 	struct completion	*detach_completion; /* all workers detached */
 
 	struct ida		worker_ida;	/* worker IDs for task name */
@@ -1902,7 +1903,7 @@ static void worker_detach_from_pool(struct worker *worker)
 	list_del(&worker->node);
 	worker->pool = NULL;
 
-	if (list_empty(&pool->workers))
+	if (list_empty(&pool->workers) && list_empty(&pool->dying_workers))
 		detach_completion = pool->detach_completion;
 	mutex_unlock(&wq_pool_attach_mutex);
 
@@ -1991,9 +1992,31 @@ static void rebind_worker(struct worker *worker, struct worker_pool *pool)
 	WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask) < 0);
 }
 
+static void reap_workers(struct list_head *reaplist)
+{
+	struct worker *worker, *tmp;
+
+	list_for_each_entry_safe(worker, tmp, reaplist, entry) {
+		list_del_init(&worker->entry);
+		unbind_worker(worker);
+		/*
+		 * If the worker was somehow already running, then it had to be
+		 * in pool->idle_list when destroy_worker() happened or we
+		 * wouldn't have gotten here.
+		 *
+		 * Thus, the worker must either have observed the WORKER_DIE
+		 * flag, or have set its state to TASK_IDLE. Either way, the
+		 * below will be observed by the worker and is safe to do
+		 * outside of pool->lock.
+		 */
+		wake_up_process(worker->task);
+	}
+}
+
 /**
  * destroy_worker - destroy a workqueue worker
  * @worker: worker to be destroyed
+ * @list: transfer worker away from its pool->idle_list and into list
  *
  * Destroy @worker and adjust @pool stats accordingly.  The worker should
  * be idle.
@@ -2001,11 +2024,12 @@ static void rebind_worker(struct worker *worker, struct worker_pool *pool)
  * CONTEXT:
  * raw_spin_lock_irq(pool->lock).
  */
-static void destroy_worker(struct worker *worker)
+static void destroy_worker(struct worker *worker, struct list_head *list)
 {
 	struct worker_pool *pool = worker->pool;
 
 	lockdep_assert_held(&pool->lock);
+	lockdep_assert_held(&wq_pool_attach_mutex);
 
 	/* sanity check frenzy */
 	if (WARN_ON(worker->current_work) ||
@@ -2016,21 +2040,50 @@ static void destroy_worker(struct worker *worker)
 	pool->nr_workers--;
 	pool->nr_idle--;
 
-	list_del_init(&worker->entry);
 	worker->flags |= WORKER_DIE;
-	wake_up_process(worker->task);
+
+	list_move(&worker->entry, list);
+	list_move(&worker->node, &pool->dying_workers);
 }
 
 /*
  * idle_reaper_fn - reap workers that have been idle for too long.
  *
+ * Unbinding marked-for-destruction workers requires a sleepable context, as
+ * changing a task's affinity is not an atomic operation, and we don't want
+ * to disturb isolated CPUs IDLE_WORKER_TIMEOUT in the future just for a kworker
+ * to do_exit().
+ *
+ * Percpu kworkers should meet the conditions for the affinity change to not
+ * block (not migration-disabled and not running), but there is no *hard*
+ * guarantee that they are not running when we get here.
+ *
  * The delayed_work is only ever modified under raw_spin_lock_irq(pool->lock).
  */
 static void idle_reaper_fn(struct work_struct *work)
 {
 	struct delayed_work *dwork = to_delayed_work(work);
 	struct worker_pool *pool = container_of(dwork, struct worker_pool, idle_reaper_work);
+	struct list_head reaplist;
 
+	INIT_LIST_HEAD(&reaplist);
+
+	/*
+	 * Unlikely as it may be, a to-be-reaped worker could run after
+	 * idle_reaper_fn()::destroy_worker() has happened but before
+	 * idle_reaper_fn()::reap_workers() (consider a worker that stays
+	 * preempted after setting itself in the idle list, or before removing
+	 * itself from it).
+	 *
+	 * WORKER_DIE would be set in worker->flags, so it would be able to
+	 * kfree(worker) and head out to do_exit(), which wouldn't be nice to
+	 * the idle reaper.
+	 *
+	 * Grabbing wq_pool_attach_mutex here ensures an already-running worker
+	 * won't go beyond worker_detach_from_pool() in its self-destruct path
+	 * (WORKER_DIE is set with wq_pool_attach_mutex set).
+	 */
+	mutex_lock(&wq_pool_attach_mutex);
 	raw_spin_lock_irq(&pool->lock);
 
 	while (too_many_workers(pool)) {
@@ -2055,10 +2108,11 @@ static void idle_reaper_fn(struct work_struct *work)
 			break;
 		}
 
-		destroy_worker(worker);
+		destroy_worker(worker, &reaplist);
 	}
-
 	raw_spin_unlock_irq(&pool->lock);
+	reap_workers(&reaplist);
+	mutex_unlock(&wq_pool_attach_mutex);
 }
 
 static void send_mayday(struct work_struct *work)
@@ -2422,12 +2476,12 @@ static int worker_thread(void *__worker)
 	/* am I supposed to die? */
 	if (unlikely(worker->flags & WORKER_DIE)) {
 		raw_spin_unlock_irq(&pool->lock);
-		WARN_ON_ONCE(!list_empty(&worker->entry));
 		set_pf_worker(false);
 
 		set_task_comm(worker->task, "kworker/dying");
 		ida_free(&pool->worker_ida, worker->id);
 		worker_detach_from_pool(worker);
+		WARN_ON_ONCE(!list_empty(&worker->entry));
 		kfree(worker);
 		return 0;
 	}
@@ -3500,6 +3554,7 @@ static int init_worker_pool(struct worker_pool *pool)
 	timer_setup(&pool->mayday_timer, pool_mayday_timeout, 0);
 
 	INIT_LIST_HEAD(&pool->workers);
+	INIT_LIST_HEAD(&pool->dying_workers);
 
 	ida_init(&pool->worker_ida);
 	INIT_HLIST_NODE(&pool->hash_node);
@@ -3600,8 +3655,11 @@ static bool wq_manager_inactive(struct worker_pool *pool)
 static void put_unbound_pool(struct worker_pool *pool)
 {
 	DECLARE_COMPLETION_ONSTACK(detach_completion);
+	struct list_head reaplist;
 	struct worker *worker;
 
+	INIT_LIST_HEAD(&reaplist);
+
 	lockdep_assert_held(&wq_pool_mutex);
 
 	if (--pool->refcnt)
@@ -3624,17 +3682,19 @@ static void put_unbound_pool(struct worker_pool *pool)
 	 * Because of how wq_manager_inactive() works, we will hold the
 	 * spinlock after a successful wait.
 	 */
+	mutex_lock(&wq_pool_attach_mutex);
 	rcuwait_wait_event(&manager_wait, wq_manager_inactive(pool),
 			   TASK_UNINTERRUPTIBLE);
 	pool->flags |= POOL_MANAGER_ACTIVE;
 
 	while ((worker = first_idle_worker(pool)))
-		destroy_worker(worker);
+		destroy_worker(worker, &reaplist);
 	WARN_ON(pool->nr_workers || pool->nr_idle);
 	raw_spin_unlock_irq(&pool->lock);
 
-	mutex_lock(&wq_pool_attach_mutex);
-	if (!list_empty(&pool->workers))
+	reap_workers(&reaplist);
+
+	if (!list_empty(&pool->workers) || !list_empty(&pool->dying_workers))
 		pool->detach_completion = &detach_completion;
 	mutex_unlock(&wq_pool_attach_mutex);
 
-- 
2.31.1


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

* Re: [PATCH v4 4/4] workqueue: Unbind workers before sending them to exit()
       [not found] ` <20221005010832.1934-1-hdanton@sina.com>
@ 2022-10-05 11:13   ` Valentin Schneider
       [not found]   ` <20221005145022.1695-1-hdanton@sina.com>
  1 sibling, 0 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-10-05 11:13 UTC (permalink / raw)
  To: Hillf Danton
  Cc: linux-mm, linux-kernel, Lai Jiangshan, Peter Zijlstra,
	Frederic Weisbecker, Marcelo Tosatti

On 05/10/22 09:08, Hillf Danton wrote:
> On 4 Oct 2022 16:05:21 +0100 Valentin Schneider <vschneid@redhat.com>
>> It has been reported that isolated CPUs can suffer from interference due to
>> per-CPU kworkers waking up just to die.
>>
>> A surge of workqueue activity during initial setup of a latency-sensitive
>> application (refresh_vm_stats() being one of the culprits) can cause extra
>> per-CPU kworkers to be spawned. Then, said latency-sensitive task can be
>> running merrily on an isolated CPU only to be interrupted sometime later by
>> a kworker marked for death (cf. IDLE_WORKER_TIMEOUT, 5 minutes after last
>> kworker activity).
>>
> Is tick stopped on the isolated CPU? If tick can hit it then it can accept
> more than exiting kworker.

From what I've seen in the scenarios where that happens, yes. The
pool->idle_timer gets queued from an isolated CPU and ends up on a
housekeeping CPU (cf. get_target_base()).


> Another option is exclude isolated CPUs from
> active CPUs because workqueue has other works to do than isolating CPUs.
>

With nohz_full on the cmdline, wq_unbound_cpumask already excludes isolated
CPU, but that doesn't apply to per-CPU kworkers. Or did you mean some other
mechanism?

>> Prevent this by affining kworkers to the wq_unbound_cpumask (which doesn't
>> contain isolated CPUs, cf. HK_TYPE_WQ) before waking them up after marking
>> them with WORKER_DIE.
>>


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

* Re: [PATCH v4 4/4] workqueue: Unbind workers before sending them to exit()
       [not found]   ` <20221005145022.1695-1-hdanton@sina.com>
@ 2022-10-05 16:14     ` Valentin Schneider
  0 siblings, 0 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-10-05 16:14 UTC (permalink / raw)
  To: Hillf Danton
  Cc: linux-mm, linux-kernel, Lai Jiangshan, Peter Zijlstra,
	Frederic Weisbecker, Marcelo Tosatti

On 05/10/22 22:50, Hillf Danton wrote:
> On 05 Oct 2022 12:13:17 +0100 Valentin Schneider <vschneid@redhat.com>
>
> Bound kworkers can be destroyed by the idle timer on a housekeeping CPU.
>
> Diff is only for thoughts.
>
> +++ b/kernel/workqueue.c
> @@ -1985,6 +1985,7 @@ fail:
>  static void destroy_worker(struct worker *worker)
>  {
>       struct worker_pool *pool = worker->pool;
> +	int cpu = smp_processor_id();
>
>       lockdep_assert_held(&pool->lock);
>
> @@ -1999,6 +2000,12 @@ static void destroy_worker(struct worker
>
>       list_del_init(&worker->entry);
>       worker->flags |= WORKER_DIE;
> +
> +	if (!(pool->flags & POOL_DISASSOCIATED) && pool->cpu != cpu) {
> +		/* send worker to die on a housekeeping cpu */
> +		cpumask_clear(&worker->task->cpus_mask);
> +		cpumask_set_cpu(cpu, &worker->task->cpus_mask);
> +	}
>       wake_up_process(worker->task);
>  }
>

The proper interface to play with that cpumask is set_cpus_allowed_ptr(),
which requires a sleepable context, hence the whole series.


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

* Re: [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs
  2022-10-04 15:05 [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
                   ` (4 preceding siblings ...)
       [not found] ` <20221005010832.1934-1-hdanton@sina.com>
@ 2022-10-25  9:42 ` Valentin Schneider
  2022-10-31 18:45   ` Tejun Heo
  5 siblings, 1 reply; 11+ messages in thread
From: Valentin Schneider @ 2022-10-25  9:42 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On 04/10/22 16:05, Valentin Schneider wrote:
> Hi folks,
>
> I haven't sent an update for this in a while, but the issue has risen again in
> some other environment so I'm getting more reasons to push this out.

Gentle ping.


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

* Re: [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs
  2022-10-25  9:42 ` [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
@ 2022-10-31 18:45   ` Tejun Heo
  0 siblings, 0 replies; 11+ messages in thread
From: Tejun Heo @ 2022-10-31 18:45 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On Tue, Oct 25, 2022 at 10:42:46AM +0100, Valentin Schneider wrote:
> On 04/10/22 16:05, Valentin Schneider wrote:
> > Hi folks,
> >
> > I haven't sent an update for this in a while, but the issue has risen again in
> > some other environment so I'm getting more reasons to push this out.
> 
> Gentle ping.

The approach looks fine to me. I only have minor comments. I'll reply to
each patch.

Lai, you've been involved in this effort in earlier patch series, how does
this look to you now?

Thanks.

-- 
tejun

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

* Re: [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work
  2022-10-04 15:05 ` [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work Valentin Schneider
@ 2022-10-31 18:49   ` Tejun Heo
  2022-11-02 17:13     ` Valentin Schneider
  0 siblings, 1 reply; 11+ messages in thread
From: Tejun Heo @ 2022-10-31 18:49 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Hello,

On Tue, Oct 04, 2022 at 04:05:20PM +0100, Valentin Schneider wrote:
> +static void idle_reaper_fn(struct work_struct *work)
>  {
> -	struct worker_pool *pool = from_timer(pool, t, idle_timer);
> +	struct delayed_work *dwork = to_delayed_work(work);
> +	struct worker_pool *pool = container_of(dwork, struct worker_pool, idle_reaper_work);
>  
>  	raw_spin_lock_irq(&pool->lock);
>  
>  	while (too_many_workers(pool)) {
>  		struct worker *worker;
>  		unsigned long expires;
> +		unsigned long now = jiffies;
>  
> -		/* idle_list is kept in LIFO order, check the last one */
> +		/* idle_list is kept in LIFO order, check the oldest entry */
>  		worker = list_entry(pool->idle_list.prev, struct worker, entry);
>  		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
>  
> -		if (time_before(jiffies, expires)) {
> -			mod_timer(&pool->idle_timer, expires);

So, one thing which bothers me is that the idle timer is supposed to go off
spuriously periodically. The idea being that letting it expire spuriously
should usually be cheaper than trying to update it constantly as workers
wake up and sleep. Converting the timer to a delayed work makes spurious
wakeups significantly more expensive tho as it's now a full scheduling
event.

Can we separate the timer and work item out so that we can kick off the work
item iff there actually are tasks to kill?

Thanks.

-- 
tejun

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

* Re: [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work
  2022-10-31 18:49   ` Tejun Heo
@ 2022-11-02 17:13     ` Valentin Schneider
  0 siblings, 0 replies; 11+ messages in thread
From: Valentin Schneider @ 2022-11-02 17:13 UTC (permalink / raw)
  To: Tejun Heo
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On 31/10/22 08:49, Tejun Heo wrote:
> Hello,
>
> On Tue, Oct 04, 2022 at 04:05:20PM +0100, Valentin Schneider wrote:
>> +static void idle_reaper_fn(struct work_struct *work)
>>  {
>> -	struct worker_pool *pool = from_timer(pool, t, idle_timer);
>> +	struct delayed_work *dwork = to_delayed_work(work);
>> +	struct worker_pool *pool = container_of(dwork, struct worker_pool, idle_reaper_work);
>>
>>      raw_spin_lock_irq(&pool->lock);
>>
>>      while (too_many_workers(pool)) {
>>              struct worker *worker;
>>              unsigned long expires;
>> +		unsigned long now = jiffies;
>>
>> -		/* idle_list is kept in LIFO order, check the last one */
>> +		/* idle_list is kept in LIFO order, check the oldest entry */
>>              worker = list_entry(pool->idle_list.prev, struct worker, entry);
>>              expires = worker->last_active + IDLE_WORKER_TIMEOUT;
>>
>> -		if (time_before(jiffies, expires)) {
>> -			mod_timer(&pool->idle_timer, expires);
>
> So, one thing which bothers me is that the idle timer is supposed to go off
> spuriously periodically. The idea being that letting it expire spuriously
> should usually be cheaper than trying to update it constantly as workers
> wake up and sleep. Converting the timer to a delayed work makes spurious
> wakeups significantly more expensive tho as it's now a full scheduling
> event.
>

Right.

> Can we separate the timer and work item out so that we can kick off the work
> item iff there actually are tasks to kill?
>

One thing I can try to have a DIY delayed_work where the timer callback
doesn't just queue the work but first checks for too_many_workers(). This
will mostly likely result in a different behaviour as worker deletion will
then involve two pool->lock regions, but this will still catch long-idling
workers.

> Thanks.
>
> --
> tejun


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

end of thread, other threads:[~2022-11-02 17:14 UTC | newest]

Thread overview: 11+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-10-04 15:05 [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
2022-10-04 15:05 ` [PATCH v4 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
2022-10-04 15:05 ` [PATCH v4 2/4] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
2022-10-04 15:05 ` [PATCH v4 3/4] workqueue: Convert the idle_timer to a delayed_work Valentin Schneider
2022-10-31 18:49   ` Tejun Heo
2022-11-02 17:13     ` Valentin Schneider
2022-10-04 15:05 ` [PATCH v4 4/4] workqueue: Unbind workers before sending them to exit() Valentin Schneider
     [not found] ` <20221005010832.1934-1-hdanton@sina.com>
2022-10-05 11:13   ` Valentin Schneider
     [not found]   ` <20221005145022.1695-1-hdanton@sina.com>
2022-10-05 16:14     ` Valentin Schneider
2022-10-25  9:42 ` [PATCH v4 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
2022-10-31 18:45   ` Tejun Heo

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).