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

Hi folks,

That's v5 (hopefully) addressing Tejun's comments, cf.
  https://lore.kernel.org/lkml/20221004150521.822266-1-vschneid@redhat.com/

Revisions
=========

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 (4):
  workqueue: Factorize unbind/rebind_workers() logic
  workqueue: Make too_many_workers() return the worker excess
  workqueue: Convert the idle_timer to a timer + work_struct
  workqueue: Unbind kworkers before sending them to exit()

 kernel/workqueue.c | 224 +++++++++++++++++++++++++++++++++------------
 1 file changed, 168 insertions(+), 56 deletions(-)

--
2.31.1


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

* [PATCH v5 1/5] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex
  2022-11-22 19:29 [PATCH v5 0/5] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
@ 2022-11-22 19:29 ` Valentin Schneider
  2022-11-22 19:29 ` [PATCH v5 2/5] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 10+ messages in thread
From: Valentin Schneider @ 2022-11-22 19:29 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 7cd5f5e7e0a1b..8e21c352c1558 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] 10+ messages in thread

* [PATCH v5 2/5] workqueue: Factorize unbind/rebind_workers() logic
  2022-11-22 19:29 [PATCH v5 0/5] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  2022-11-22 19:29 ` [PATCH v5 1/5] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
@ 2022-11-22 19:29 ` Valentin Schneider
  2022-11-22 19:29 ` [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess Valentin Schneider
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 10+ messages in thread
From: Valentin Schneider @ 2022-11-22 19:29 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 8e21c352c1558..8185a42848c50 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] 10+ messages in thread

* [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess
  2022-11-22 19:29 [PATCH v5 0/5] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
  2022-11-22 19:29 ` [PATCH v5 1/5] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
  2022-11-22 19:29 ` [PATCH v5 2/5] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
@ 2022-11-22 19:29 ` Valentin Schneider
  2022-11-22 20:17   ` Tejun Heo
  2022-11-22 19:29 ` [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
  2022-11-22 19:29 ` [PATCH v5 5/5] workqueue: Unbind kworkers before sending them to exit() Valentin Schneider
  4 siblings, 1 reply; 10+ messages in thread
From: Valentin Schneider @ 2022-11-22 19:29 UTC (permalink / raw)
  To: linux-kernel
  Cc: Tejun Heo, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Later patches will need the logic implemented within too_many_workers() to
get the amount of workers to delete. Rather than duplicate the logic,
rework too_many_workers() to return the count of workers to delete - its
return value can be used as a boolean value, so no change in behaviour
intended.

The function currently returns true when
  (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy
thus, the desired number of idle workers is expressed by
  (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO == nr_busy - 1
IOW
   nr_idle == ((nr_busy - 1) / MAX_IDLE_WORKERS_RATIO) + 2

MAX_IDLE_WORKERS_RATIO being a compile-time power of 2, we can leave that
as a division.

While at it, rename too_many_workers() to worker_cull_count().

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 8185a42848c50..4fc8085f3fe17 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -816,14 +816,20 @@ static bool need_to_create_worker(struct worker_pool *pool)
 	return need_more_worker(pool) && !may_start_working(pool);
 }
 
-/* Do we have too many workers and should some go away? */
-static bool too_many_workers(struct worker_pool *pool)
+/* How many idle workers should we get rid of, if any? */
+static unsigned int worker_cull_count(struct worker_pool *pool)
 {
 	bool managing = pool->flags & POOL_MANAGER_ACTIVE;
 	int nr_idle = pool->nr_idle + managing; /* manager is considered idle */
 	int nr_busy = pool->nr_workers - nr_idle;
 
-	return nr_idle > 2 && (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy;
+	lockdep_assert_held(&pool->lock);
+
+	/*
+	 * We keep at least 2 spare idle workers, but overall aim to keep at
+	 * most (1 / MAX_IDLE_WORKERS_RATIO) workers idle.
+	 */
+	return max(0, nr_idle - 2 - ((nr_busy - 1) / MAX_IDLE_WORKERS_RATIO));
 }
 
 /*
@@ -1806,7 +1812,7 @@ 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))
+	if (worker_cull_count(pool) && !timer_pending(&pool->idle_timer))
 		mod_timer(&pool->idle_timer, jiffies + IDLE_WORKER_TIMEOUT);
 
 	/* Sanity check nr_running. */
@@ -2025,7 +2031,7 @@ static void idle_worker_timeout(struct timer_list *t)
 
 	raw_spin_lock_irq(&pool->lock);
 
-	while (too_many_workers(pool)) {
+	while (worker_cull_count(pool)) {
 		struct worker *worker;
 		unsigned long expires;
 
-- 
2.31.1


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

* [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct
  2022-11-22 19:29 [PATCH v5 0/5] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
                   ` (2 preceding siblings ...)
  2022-11-22 19:29 ` [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess Valentin Schneider
@ 2022-11-22 19:29 ` Valentin Schneider
  2022-11-22 20:23   ` Tejun Heo
  2022-11-22 19:29 ` [PATCH v5 5/5] workqueue: Unbind kworkers before sending them to exit() Valentin Schneider
  4 siblings, 1 reply; 10+ messages in thread
From: Valentin Schneider @ 2022-11-22 19:29 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, 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.

The new worker_pool.idle_cull_list is made ____cacheline_aligned to prevent
it from sitting over two cachelines.

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

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 4fc8085f3fe17..b744288c58a4b 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -169,7 +169,12 @@ 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 list_head	idle_cull_list  /* L: list of idle workers to cull */
+	____cacheline_aligned;
+	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);
@@ -1812,7 +1817,9 @@ static void worker_enter_idle(struct worker *worker)
 	/* idle_list is LIFO */
 	list_add(&worker->entry, &pool->idle_list);
 
-	if (worker_cull_count(pool) && !timer_pending(&pool->idle_timer))
+	if (worker_cull_count(pool) &&
+	    !timer_pending(&pool->idle_timer) &&
+	    !work_pending(&pool->idle_cull_work))
 		mod_timer(&pool->idle_timer, jiffies + IDLE_WORKER_TIMEOUT);
 
 	/* Sanity check nr_running. */
@@ -2025,13 +2032,27 @@ static void destroy_worker(struct worker *worker)
 	wake_up_process(worker->task);
 }
 
+/*
+ * idle_worker_timeout - check if some idle workers can now be deleted.
+ *
+ * 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 worker_cull_count() 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);
+	unsigned int max_cull_cnt, cull_cnt;
+
+	if (work_pending(&pool->idle_cull_work))
+		return;
 
 	raw_spin_lock_irq(&pool->lock);
 
-	while (worker_cull_count(pool)) {
+	max_cull_cnt = worker_cull_count(pool);
+	for (cull_cnt = 0; cull_cnt < max_cull_cnt; cull_cnt++) {
 		struct worker *worker;
 		unsigned long expires;
 
@@ -2039,12 +2060,48 @@ static void idle_worker_timeout(struct timer_list *t)
 		worker = list_entry(pool->idle_list.prev, struct worker, entry);
 		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
 
+		/* All remaining entries will be younger than this */
 		if (time_before(jiffies, expires)) {
-			mod_timer(&pool->idle_timer, expires);
+			if (!cull_cnt)
+				mod_timer(&pool->idle_timer, expires);
 			break;
 		}
 
+		/*
+		 * Mark the idle worker ripe for culling.
+		 * If a preempted idle worker gets to run before the idle cull
+		 * handles it, it will just pop itself out of that list and
+		 * continue as normal.
+		 */
+		list_move(&worker->entry, &pool->idle_cull_list);
+	}
+	raw_spin_unlock_irq(&pool->lock);
+
+	if (cull_cnt)
+		queue_work(system_unbound_wq, &pool->idle_cull_work);
+}
+
+/*
+ * idle_cull_fn - cull workers that have been idle for too long.
+ */
+static void idle_cull_fn(struct work_struct *work)
+{
+	struct worker_pool *pool = container_of(work, struct worker_pool, idle_cull_work);
+	struct worker *worker, *tmp;
+
+	raw_spin_lock_irq(&pool->lock);
+
+	list_for_each_entry_safe(worker, tmp, &pool->idle_cull_list, entry)
 		destroy_worker(worker);
+
+	/* Re-arm the idle timer if necessary */
+	if (pool->nr_idle) {
+		unsigned long expires;
+
+		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);
 	}
 
 	raw_spin_unlock_irq(&pool->lock);
@@ -3482,9 +3539,11 @@ static int init_worker_pool(struct worker_pool *pool)
 	pool->watchdog_ts = jiffies;
 	INIT_LIST_HEAD(&pool->worklist);
 	INIT_LIST_HEAD(&pool->idle_list);
+	INIT_LIST_HEAD(&pool->idle_cull_list);
 	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);
 
@@ -3632,6 +3691,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] 10+ messages in thread

* [PATCH v5 5/5] workqueue: Unbind kworkers before sending them to exit()
  2022-11-22 19:29 [PATCH v5 0/5] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
                   ` (3 preceding siblings ...)
  2022-11-22 19:29 ` [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
@ 2022-11-22 19:29 ` Valentin Schneider
  4 siblings, 0 replies; 10+ messages in thread
From: Valentin Schneider @ 2022-11-22 19:29 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 | 70 ++++++++++++++++++++++++++++++++++++++--------
 1 file changed, 58 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index b744288c58a4b..8ce3b63851e95 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -182,6 +182,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 */
@@ -1913,7 +1914,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);
 
@@ -2002,21 +2003,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) ||
@@ -2027,9 +2051,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);
 }
 
 /*
@@ -2083,16 +2108,29 @@ static void idle_worker_timeout(struct timer_list *t)
 
 /*
  * idle_cull_fn - cull workers that have been idle for too long.
+ *
+ * We don't want to disturbed 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 worker *worker, *tmp;
+	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);
 
 	list_for_each_entry_safe(worker, tmp, &pool->idle_cull_list, entry)
-		destroy_worker(worker);
+		set_worker_dying(worker, &cull_list);
 
 	/* Re-arm the idle timer if necessary */
 	if (pool->nr_idle) {
@@ -2105,6 +2143,8 @@ static void idle_cull_fn(struct work_struct *work)
 	}
 
 	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)
@@ -2468,12 +2508,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;
 	}
@@ -3548,6 +3588,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);
@@ -3648,8 +3689,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)
@@ -3672,17 +3716,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);
+		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] 10+ messages in thread

* Re: [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess
  2022-11-22 19:29 ` [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess Valentin Schneider
@ 2022-11-22 20:17   ` Tejun Heo
  2022-11-28 11:24     ` Valentin Schneider
  0 siblings, 1 reply; 10+ messages in thread
From: Tejun Heo @ 2022-11-22 20:17 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Hello,

On Tue, Nov 22, 2022 at 07:29:35PM +0000, Valentin Schneider wrote:
...
> The function currently returns true when
>   (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy
> thus, the desired number of idle workers is expressed by
>   (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO == nr_busy - 1
> IOW
>    nr_idle == ((nr_busy - 1) / MAX_IDLE_WORKERS_RATIO) + 2
> +/* How many idle workers should we get rid of, if any? */
> +static unsigned int worker_cull_count(struct worker_pool *pool)

Can we name it nr_workers_to_cull()?

>  {
>  	bool managing = pool->flags & POOL_MANAGER_ACTIVE;
>  	int nr_idle = pool->nr_idle + managing; /* manager is considered idle */
>  	int nr_busy = pool->nr_workers - nr_idle;
>  
> -	return nr_idle > 2 && (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy;
> +	lockdep_assert_held(&pool->lock);
> +
> +	/*
> +	 * We keep at least 2 spare idle workers, but overall aim to keep at
> +	 * most (1 / MAX_IDLE_WORKERS_RATIO) workers idle.
> +	 */
> +	return max(0, nr_idle - 2 - ((nr_busy - 1) / MAX_IDLE_WORKERS_RATIO));

I think we can do away with the subtraction on nr_busy. I don't think it'd
make any material difference, so maybe we can do:

        return max(0, nr_idle - 2 - nr_busy / MAX_IDLE_WORKERS_RATIO);

Thanks.

-- 
tejun

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

* Re: [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct
  2022-11-22 19:29 ` [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
@ 2022-11-22 20:23   ` Tejun Heo
  2022-11-28 11:24     ` Valentin Schneider
  0 siblings, 1 reply; 10+ messages in thread
From: Tejun Heo @ 2022-11-22 20:23 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

Hello,

On Tue, Nov 22, 2022 at 07:29:36PM +0000, Valentin Schneider wrote:
> @@ -2039,12 +2060,48 @@ static void idle_worker_timeout(struct timer_list *t)
>  		worker = list_entry(pool->idle_list.prev, struct worker, entry);
>  		expires = worker->last_active + IDLE_WORKER_TIMEOUT;
>  
> +		/* All remaining entries will be younger than this */
>  		if (time_before(jiffies, expires)) {
> -			mod_timer(&pool->idle_timer, expires);
> +			if (!cull_cnt)
> +				mod_timer(&pool->idle_timer, expires);
>  			break;
>  		}
>  
> +		/*
> +		 * Mark the idle worker ripe for culling.
> +		 * If a preempted idle worker gets to run before the idle cull
> +		 * handles it, it will just pop itself out of that list and
> +		 * continue as normal.
> +		 */
> +		list_move(&worker->entry, &pool->idle_cull_list);
> +	}
> +	raw_spin_unlock_irq(&pool->lock);
> +
> +	if (cull_cnt)
> +		queue_work(system_unbound_wq, &pool->idle_cull_work);
> +}

So, you mentioned this explicitly in the cover letter but I think I'd prefer
if the timer were simpler and all logic were in the work item. It just needs
to pick at the first worker and compare the expiration once, right? If that
bothers you, we can make workers keep track of the oldest idle's timestamp
in, say, wq->first_idle_at as the workers go idle and busy and then the
timer can simply look at the value and decide to schedule the work item or
not.

Thanks.

-- 
tejun

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

* Re: [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct
  2022-11-22 20:23   ` Tejun Heo
@ 2022-11-28 11:24     ` Valentin Schneider
  0 siblings, 0 replies; 10+ messages in thread
From: Valentin Schneider @ 2022-11-28 11:24 UTC (permalink / raw)
  To: Tejun Heo
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On 22/11/22 10:23, Tejun Heo wrote:
> Hello,
>

Thanks for having a look at this so quickly!

> On Tue, Nov 22, 2022 at 07:29:36PM +0000, Valentin Schneider wrote:
>> @@ -2039,12 +2060,48 @@ static void idle_worker_timeout(struct timer_list *t)
>>              worker = list_entry(pool->idle_list.prev, struct worker, entry);
>>              expires = worker->last_active + IDLE_WORKER_TIMEOUT;
>>
>> +		/* All remaining entries will be younger than this */
>>              if (time_before(jiffies, expires)) {
>> -			mod_timer(&pool->idle_timer, expires);
>> +			if (!cull_cnt)
>> +				mod_timer(&pool->idle_timer, expires);
>>                      break;
>>              }
>>
>> +		/*
>> +		 * Mark the idle worker ripe for culling.
>> +		 * If a preempted idle worker gets to run before the idle cull
>> +		 * handles it, it will just pop itself out of that list and
>> +		 * continue as normal.
>> +		 */
>> +		list_move(&worker->entry, &pool->idle_cull_list);
>> +	}
>> +	raw_spin_unlock_irq(&pool->lock);
>> +
>> +	if (cull_cnt)
>> +		queue_work(system_unbound_wq, &pool->idle_cull_work);
>> +}
>
> So, you mentioned this explicitly in the cover letter but I think I'd prefer
> if the timer were simpler and all logic were in the work item. It just needs
> to pick at the first worker and compare the expiration once, right?

Yep exactly. I wasn't fond of repeating the expiration check pattern, and
also it meant the culling worker could see different things than the
timer.

Moving everything in the worker does however mean we can get rid of the new
worker_pool.idle_cull_list, which is my least favourite bit of that
approach, so I'll give that a try.

> If that
> bothers you, we can make workers keep track of the oldest idle's timestamp
> in, say, wq->first_idle_at as the workers go idle and busy and then the
> timer can simply look at the value and decide to schedule the work item or
> not.
>

I don't think the overhead at worker_{enter,leave}_idle() would be worth it.


> Thanks.
>
> --
> tejun


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

* Re: [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess
  2022-11-22 20:17   ` Tejun Heo
@ 2022-11-28 11:24     ` Valentin Schneider
  0 siblings, 0 replies; 10+ messages in thread
From: Valentin Schneider @ 2022-11-28 11:24 UTC (permalink / raw)
  To: Tejun Heo
  Cc: linux-kernel, Lai Jiangshan, Peter Zijlstra, Frederic Weisbecker,
	Juri Lelli, Phil Auld, Marcelo Tosatti

On 22/11/22 10:17, Tejun Heo wrote:
> Hello,
>
> On Tue, Nov 22, 2022 at 07:29:35PM +0000, Valentin Schneider wrote:
> ...
>> The function currently returns true when
>>   (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy
>> thus, the desired number of idle workers is expressed by
>>   (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO == nr_busy - 1
>> IOW
>>    nr_idle == ((nr_busy - 1) / MAX_IDLE_WORKERS_RATIO) + 2
>> +/* How many idle workers should we get rid of, if any? */
>> +static unsigned int worker_cull_count(struct worker_pool *pool)
>
> Can we name it nr_workers_to_cull()?
>

Ack

>>  {
>>  	bool managing = pool->flags & POOL_MANAGER_ACTIVE;
>>  	int nr_idle = pool->nr_idle + managing; /* manager is considered idle */
>>  	int nr_busy = pool->nr_workers - nr_idle;
>>  
>> -	return nr_idle > 2 && (nr_idle - 2) * MAX_IDLE_WORKERS_RATIO >= nr_busy;
>> +	lockdep_assert_held(&pool->lock);
>> +
>> +	/*
>> +	 * We keep at least 2 spare idle workers, but overall aim to keep at
>> +	 * most (1 / MAX_IDLE_WORKERS_RATIO) workers idle.
>> +	 */
>> +	return max(0, nr_idle - 2 - ((nr_busy - 1) / MAX_IDLE_WORKERS_RATIO));
>
> I think we can do away with the subtraction on nr_busy. I don't think it'd
> make any material difference, so maybe we can do:
>
>         return max(0, nr_idle - 2 - nr_busy / MAX_IDLE_WORKERS_RATIO);
>

I'll do that if this survives in the next revision :)

> Thanks.
>
> -- 
> tejun


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

end of thread, other threads:[~2022-11-28 11:25 UTC | newest]

Thread overview: 10+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-11-22 19:29 [PATCH v5 0/5] workqueue: destroy_worker() vs isolated CPUs Valentin Schneider
2022-11-22 19:29 ` [PATCH v5 1/5] workqueue: Protects wq_unbound_cpumask with wq_pool_attach_mutex Valentin Schneider
2022-11-22 19:29 ` [PATCH v5 2/5] workqueue: Factorize unbind/rebind_workers() logic Valentin Schneider
2022-11-22 19:29 ` [PATCH v5 3/5] workqueue: Make too_many_workers() return the worker excess Valentin Schneider
2022-11-22 20:17   ` Tejun Heo
2022-11-28 11:24     ` Valentin Schneider
2022-11-22 19:29 ` [PATCH v5 4/5] workqueue: Convert the idle_timer to a timer + work_struct Valentin Schneider
2022-11-22 20:23   ` Tejun Heo
2022-11-28 11:24     ` Valentin Schneider
2022-11-22 19:29 ` [PATCH v5 5/5] workqueue: Unbind kworkers before sending them to exit() 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).