linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v7 0/4] workqueue: destroy_worker() vs isolated CPUs
@ 2023-01-09 13:33 Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
                   ` (3 more replies)
  0 siblings, 4 replies; 9+ messages in thread
From: Valentin Schneider @ 2023-01-09 13:33 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Hi folks,

New year, new version!

No major changes here, mainly some tidying up from Tejun's comments and a bugfix
spotted by Lai.

Revisions
=========

v6 -> v7
++++++++

o Rebased onto v6.2-rc3

o Dropped work pending check in worker_enter_idle() (Tejun)
o Overall comment cleanup (Tejun)

o put_unbound_pool() locking issue (Lai)
  Unfortunately the mutex cannot be acquired from within wq_manager_inactive()
  as rcuwait_wait_event() sets the task state to TASK_UNINTERRUPTIBLE before
  invoking it, so grabbing the mutex could clobber the task state.

  I've gone with dropping the pool->lock and reacquiring the two locks in the
  right order after we've become the manager, see comments.

o Applied Lai's RB to patches that just had cosmetic changes

v5 -> v6
++++++++

o Rebase onto v6.1-rc7
o Get rid of worker_pool.idle_cull_list; only do minimal amount of work in the
  timer callback (Tejun)
o Dropped the too_many_workers() -> nr_workers_to_cull() change

v4 -> v5
++++++++

o Rebase onto v6.1-rc6

o Overall renaming from "reaping" to "cull"
  I somehow convinced myself this was more appropriate
  
o Split the dwork into timer callback + work item (Tejun)

  I didn't want to have redudant operations happen in the timer callback and in
  the work item, so I made the timer callback detect which workers are "ripe"
  enough and then toss them to a worker for removal.

  This however means we release the pool->lock before getting to actually doing
  anything to those idle workers, which means they can wake up in the meantime.
  The new worker_pool.idle_cull_list is there for that reason.

  The alternative was to have the timer callback detect if any worker was ripe
  enough, kick the work item if so, and have the work item do the same thing
  again, which I didn't like.

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 timer + work_struct
  workqueue: Unbind kworkers before sending them to exit()

 kernel/workqueue.c | 205 ++++++++++++++++++++++++++++++++++-----------
 1 file changed, 154 insertions(+), 51 deletions(-)

-- 
2.31.1


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

* [PATCH v7 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex
  2023-01-09 13:33 [PATCH v7 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
@ 2023-01-09 13:33 ` Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 2/4] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 9+ messages in thread
From: Valentin Schneider @ 2023-01-09 13:33 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 07895deca2711..76ea87b0251ce 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] 9+ messages in thread

* [PATCH v7 2/4] workqueue: Factorize unbind/rebind_workers() logic
  2023-01-09 13:33 [PATCH v7 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
@ 2023-01-09 13:33 ` Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 3/4] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit() Valentin Schneider
  3 siblings, 0 replies; 9+ messages in thread
From: Valentin Schneider @ 2023-01-09 13:33 UTC (permalink / raw)
  To: linux-kernel
  Cc: Lai Jiangshan, Tejun Heo, 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>
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>
---
 kernel/workqueue.c | 33 +++++++++++++++++++++------------
 1 file changed, 21 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 76ea87b0251ce..5e6897f0ef105 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] 9+ messages in thread

* [PATCH v7 3/4] workqueue: Convert the idle_timer to a timer + work_struct
  2023-01-09 13:33 [PATCH v7 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 2/4] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
@ 2023-01-09 13:33 ` Valentin Schneider
  2023-01-09 13:33 ` [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit() Valentin Schneider
  3 siblings, 0 replies; 9+ messages in thread
From: Valentin Schneider @ 2023-01-09 13:33 UTC (permalink / raw)
  To: linux-kernel
  Cc: Lai Jiangshan, Tejun Heo, 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, however this would imply turning all idle_timer expiries into
scheduler events (waking up a worker to handle the dwork).

Instead, implement a "custom dwork" where the timer callback does some
extra checks before queuing the associated work.

No change in functionality intended.

Signed-off-by: Valentin Schneider <vschneid@redhat.com>
Reviewed-by: Lai Jiangshan <jiangshanlai@gmail.com>
---
 kernel/workqueue.c | 52 ++++++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 50 insertions(+), 2 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 5e6897f0ef105..11bb657059bcd 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -169,7 +169,9 @@ struct worker_pool {
 
 	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 work_struct      idle_cull_work; /* L: worker idle cleanup */
+
+	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);
@@ -2019,17 +2021,61 @@ static void destroy_worker(struct worker *worker)
 	wake_up_process(worker->task);
 }
 
+/**
+ * idle_worker_timeout - check if some idle workers can now be deleted.
+ * @t: The pool's idle_timer that just expired
+ *
+ * The timer is armed in worker_enter_idle(). Note that it isn't disarmed in
+ * worker_leave_idle(), as a worker flicking between idle and active while its
+ * pool is at the too_many_workers() tipping point would cause too much timer
+ * housekeeping overhead. Since IDLE_WORKER_TIMEOUT is long enough, we just let
+ * it expire and re-evaluate things from there.
+ */
 static void idle_worker_timeout(struct timer_list *t)
 {
 	struct worker_pool *pool = from_timer(pool, t, idle_timer);
+	bool do_cull = false;
+
+	if (work_pending(&pool->idle_cull_work))
+		return;
 
 	raw_spin_lock_irq(&pool->lock);
 
-	while (too_many_workers(pool)) {
+	if (too_many_workers(pool)) {
 		struct worker *worker;
 		unsigned long expires;
 
 		/* idle_list is kept in LIFO order, check the last one */
+		worker = list_entry(pool->idle_list.prev, struct worker, entry);
+		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
+		do_cull = !time_before(jiffies, expires);
+
+		if (!do_cull)
+			mod_timer(&pool->idle_timer, expires);
+	}
+	raw_spin_unlock_irq(&pool->lock);
+
+	if (do_cull)
+		queue_work(system_unbound_wq, &pool->idle_cull_work);
+}
+
+/**
+ * idle_cull_fn - cull workers that have been idle for too long.
+ * @work: the pool's work for handling these idle workers
+ *
+ * This goes through a pool's idle workers and gets rid of those that have been
+ * idle for at least IDLE_WORKER_TIMEOUT seconds.
+ */
+static void idle_cull_fn(struct work_struct *work)
+{
+	struct worker_pool *pool = container_of(work, struct worker_pool, idle_cull_work);
+
+	raw_spin_lock_irq(&pool->lock);
+
+	while (too_many_workers(pool)) {
+		struct worker *worker;
+		unsigned long expires;
+
 		worker = list_entry(pool->idle_list.prev, struct worker, entry);
 		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
 
@@ -3479,6 +3525,7 @@ static int init_worker_pool(struct worker_pool *pool)
 	hash_init(pool->busy_hash);
 
 	timer_setup(&pool->idle_timer, idle_worker_timeout, TIMER_DEFERRABLE);
+	INIT_WORK(&pool->idle_cull_work, idle_cull_fn);
 
 	timer_setup(&pool->mayday_timer, pool_mayday_timeout, 0);
 
@@ -3626,6 +3673,7 @@ static void put_unbound_pool(struct worker_pool *pool)
 
 	/* shut down the timers */
 	del_timer_sync(&pool->idle_timer);
+	cancel_work_sync(&pool->idle_cull_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] 9+ messages in thread

* [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit()
  2023-01-09 13:33 [PATCH v7 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
                   ` (2 preceding siblings ...)
  2023-01-09 13:33 ` [PATCH v7 3/4] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
@ 2023-01-09 13:33 ` Valentin Schneider
  2023-01-10 20:28   ` Tejun Heo
  3 siblings, 1 reply; 9+ messages in thread
From: Valentin Schneider @ 2023-01-09 13:33 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_cull_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.

Rename destroy_worker() to set_working_dying() to better reflect its
effects and relationship with wake_dying_workers().

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 11bb657059bcd..45f2187836c67 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -179,6 +179,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,21 +1992,44 @@ 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 wake_dying_workers(struct list_head *cull_list)
+{
+	struct worker *worker, *tmp;
+
+	list_for_each_entry_safe(worker, tmp, cull_list, 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 set_worker_dying() 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
+ * set_worker_dying - Tag a worker for destruction
  * @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.
+ * Tag @worker for destruction and adjust @pool stats accordingly.  The worker
+ * should be idle.
  *
  * CONTEXT:
  * raw_spin_lock_irq(pool->lock).
  */
-static void destroy_worker(struct worker *worker)
+static void set_worker_dying(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,9 +2040,10 @@ 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);
 }
 
 /**
@@ -2065,11 +2090,24 @@ static void idle_worker_timeout(struct timer_list *t)
  *
  * This goes through a pool's idle workers and gets rid of those that have been
  * idle for at least IDLE_WORKER_TIMEOUT seconds.
+ *
+ * We don't want to disturb isolated CPUs because of a pcpu kworker being
+ * culled, so this also resets worker affinity. This requires a sleepable
+ * context, hence the split between timer callback and work item.
  */
 static void idle_cull_fn(struct work_struct *work)
 {
 	struct worker_pool *pool = container_of(work, struct worker_pool, idle_cull_work);
+	struct list_head cull_list;
 
+	INIT_LIST_HEAD(&cull_list);
+	/*
+	 * Grabbing wq_pool_attach_mutex here ensures an already-running worker
+	 * cannot proceed beyong worker_detach_from_pool() in its self-destruct
+	 * path. This is required as a previously-preempted worker could run after
+	 * set_worker_dying() has happened but before wake_dying_workers() did.
+	 */
+	mutex_lock(&wq_pool_attach_mutex);
 	raw_spin_lock_irq(&pool->lock);
 
 	while (too_many_workers(pool)) {
@@ -2084,10 +2122,12 @@ static void idle_cull_fn(struct work_struct *work)
 			break;
 		}
 
-		destroy_worker(worker);
+		set_worker_dying(worker, &cull_list);
 	}
 
 	raw_spin_unlock_irq(&pool->lock);
+	wake_dying_workers(&cull_list);
+	mutex_unlock(&wq_pool_attach_mutex);
 }
 
 static void send_mayday(struct work_struct *work)
@@ -2451,12 +2491,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;
 	}
@@ -3530,6 +3570,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);
@@ -3630,8 +3671,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 cull_list;
 	struct worker *worker;
 
+	INIT_LIST_HEAD(&cull_list);
+
 	lockdep_assert_held(&wq_pool_mutex);
 
 	if (--pool->refcnt)
@@ -3658,13 +3702,24 @@ static void put_unbound_pool(struct worker_pool *pool)
 			   TASK_UNINTERRUPTIBLE);
 	pool->flags |= POOL_MANAGER_ACTIVE;
 
+	/*
+	 * We need to hold wq_pool_attach_mutex() while destroying the workers,
+	 * but we can't grab it in rcuwait_wait_event() as it can clobber
+	 * current's task state. We can drop pool->lock here as we've set
+	 * POOL_MANAGER_ACTIVE, no one else can steal our manager position.
+	 */
+	raw_spin_unlock_irq(&pool->lock);
+	mutex_lock(&wq_pool_attach_mutex);
+	raw_spin_lock_irq(&pool->lock);
+
 	while ((worker = first_idle_worker(pool)))
-		destroy_worker(worker);
+		set_worker_dying(worker, &cull_list);
 	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))
+	wake_dying_workers(&cull_list);
+
+	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] 9+ messages in thread

* Re: [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit()
  2023-01-09 13:33 ` [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit() Valentin Schneider
@ 2023-01-10 20:28   ` Tejun Heo
  2023-01-11 12:49     ` Valentin Schneider
  0 siblings, 1 reply; 9+ messages in thread
From: Tejun Heo @ 2023-01-10 20:28 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Hello,

The series generally looks good to me. Just one thing.

On Mon, Jan 09, 2023 at 01:33:16PM +0000, Valentin Schneider wrote:
> @@ -3658,13 +3702,24 @@ static void put_unbound_pool(struct worker_pool *pool)
>  			   TASK_UNINTERRUPTIBLE);
>  	pool->flags |= POOL_MANAGER_ACTIVE;
>  
> +	/*
> +	 * We need to hold wq_pool_attach_mutex() while destroying the workers,
> +	 * but we can't grab it in rcuwait_wait_event() as it can clobber
> +	 * current's task state. We can drop pool->lock here as we've set
> +	 * POOL_MANAGER_ACTIVE, no one else can steal our manager position.
> +	 */
> +	raw_spin_unlock_irq(&pool->lock);
> +	mutex_lock(&wq_pool_attach_mutex);
> +	raw_spin_lock_irq(&pool->lock);

The original pattern was a bit weird to begin with and this makes it quite
worse. Let's do something more straight forward like:

        while (true) {
                rcuwait_wait_event(&manager_wait,
                                   !(pool->flags & POOL_MANAGER_ACTIVE),
                                   TASK_UNINTERRUPTIBLE);
                mutex_lock(&wq_pool_attach_mutex);
                raw_spin_lock_irq(&pool->lock);
                if (!(pool->flags & POOL_MANAGER_ACTIVE)) {
                        pool->flags |= POOL_MANAGER_ACTIVE;
                        break;
                }
                raw_spin_unlock_irq(&pool->lock);
                mutex_unlock(&wq_pool_attach_mutex);
        }

Thanks.

-- 
tejun

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

* Re: [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit()
  2023-01-10 20:28   ` Tejun Heo
@ 2023-01-11 12:49     ` Valentin Schneider
  2023-01-11 16:55       ` Tejun Heo
  0 siblings, 1 reply; 9+ messages in thread
From: Valentin Schneider @ 2023-01-11 12:49 UTC (permalink / raw)
  To: Tejun Heo
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On 10/01/23 10:28, Tejun Heo wrote:
> Hello,
>
> The series generally looks good to me. Just one thing.
>
> On Mon, Jan 09, 2023 at 01:33:16PM +0000, Valentin Schneider wrote:
>> @@ -3658,13 +3702,24 @@ static void put_unbound_pool(struct worker_pool *pool)
>>  			   TASK_UNINTERRUPTIBLE);
>>  	pool->flags |= POOL_MANAGER_ACTIVE;
>>  
>> +	/*
>> +	 * We need to hold wq_pool_attach_mutex() while destroying the workers,
>> +	 * but we can't grab it in rcuwait_wait_event() as it can clobber
>> +	 * current's task state. We can drop pool->lock here as we've set
>> +	 * POOL_MANAGER_ACTIVE, no one else can steal our manager position.
>> +	 */
>> +	raw_spin_unlock_irq(&pool->lock);
>> +	mutex_lock(&wq_pool_attach_mutex);
>> +	raw_spin_lock_irq(&pool->lock);
>
> The original pattern was a bit weird to begin with and this makes it quite
> worse.

That it does!

> Let's do something more straight forward like:
>
>         while (true) {
>                 rcuwait_wait_event(&manager_wait,
>                                    !(pool->flags & POOL_MANAGER_ACTIVE),
>                                    TASK_UNINTERRUPTIBLE);
>                 mutex_lock(&wq_pool_attach_mutex);
>                 raw_spin_lock_irq(&pool->lock);
>                 if (!(pool->flags & POOL_MANAGER_ACTIVE)) {
>                         pool->flags |= POOL_MANAGER_ACTIVE;
>                         break;
>                 }
>                 raw_spin_unlock_irq(&pool->lock);
>                 mutex_unlock(&wq_pool_attach_mutex);
>         }
>

That should do the trick, I'll go test it out.


While we're here, for my own education I was trying to figure out in what
scenarios we can hit this manager-already-active condition. When sending
out v6 I had convinced myself it could happen during failed
initialization of a new unbound pool, but having another look at it now I'm
not so sure anymore.

The only scenario I can think of now is around maybe_create_worker()'s
release of pool->lock, as that implies another worker can drain the
pool->worklist and thus let pool->refcnt reach 0 while another worker is
being the pool manager. Am I looking at the right thing?

Thanks


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

* Re: [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit()
  2023-01-11 12:49     ` Valentin Schneider
@ 2023-01-11 16:55       ` Tejun Heo
  2023-01-11 17:18         ` Valentin Schneider
  0 siblings, 1 reply; 9+ messages in thread
From: Tejun Heo @ 2023-01-11 16:55 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On Wed, Jan 11, 2023 at 12:49:49PM +0000, Valentin Schneider wrote:
> While we're here, for my own education I was trying to figure out in what
> scenarios we can hit this manager-already-active condition. When sending
> out v6 I had convinced myself it could happen during failed
> initialization of a new unbound pool, but having another look at it now I'm
> not so sure anymore.
> 
> The only scenario I can think of now is around maybe_create_worker()'s
> release of pool->lock, as that implies another worker can drain the
> pool->worklist and thus let pool->refcnt reach 0 while another worker is
> being the pool manager. Am I looking at the right thing?

To be frank, I'm not sure and can't remember why the code is like that off
the top of my head. It could well be that I was just habitually thinking
that MANAGER can be contended while in practice the scenario can never
happen in this particular case. I'll need to look harder at it but maybe we
can leave that to another day?

Thanks.

-- 
tejun

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

* Re: [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit()
  2023-01-11 16:55       ` Tejun Heo
@ 2023-01-11 17:18         ` Valentin Schneider
  0 siblings, 0 replies; 9+ messages in thread
From: Valentin Schneider @ 2023-01-11 17:18 UTC (permalink / raw)
  To: Tejun Heo
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On 11/01/23 06:55, Tejun Heo wrote:
> On Wed, Jan 11, 2023 at 12:49:49PM +0000, Valentin Schneider wrote:
>> While we're here, for my own education I was trying to figure out in what
>> scenarios we can hit this manager-already-active condition. When sending
>> out v6 I had convinced myself it could happen during failed
>> initialization of a new unbound pool, but having another look at it now I'm
>> not so sure anymore.
>> 
>> The only scenario I can think of now is around maybe_create_worker()'s
>> release of pool->lock, as that implies another worker can drain the
>> pool->worklist and thus let pool->refcnt reach 0 while another worker is
>> being the pool manager. Am I looking at the right thing?
>
> To be frank, I'm not sure and can't remember why the code is like that off
> the top of my head. It could well be that I was just habitually thinking
> that MANAGER can be contended while in practice the scenario can never
> happen in this particular case. I'll need to look harder at it but maybe we
> can leave that to another day?
>

For sure :-)

> Thanks.
>
> -- 
> tejun


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

end of thread, other threads:[~2023-01-11 17:19 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-01-09 13:33 [PATCH v7 0/4] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
2023-01-09 13:33 ` [PATCH v7 1/4] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
2023-01-09 13:33 ` [PATCH v7 2/4] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
2023-01-09 13:33 ` [PATCH v7 3/4] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
2023-01-09 13:33 ` [PATCH v7 4/4] workqueue: Unbind kworkers before sending them to exit() Valentin Schneider
2023-01-10 20:28   ` Tejun Heo
2023-01-11 12:49     ` Valentin Schneider
2023-01-11 16:55       ` Tejun Heo
2023-01-11 17:18         ` Valentin Schneider

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