linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/8] sched: Fix hot-unplug regressions
@ 2021-01-16 11:30 Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 1/8] sched/core: Print out straggler tasks in sched_cpu_dying() Peter Zijlstra
                   ` (9 more replies)
  0 siblings, 10 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

Hi,

These patches (no longer 4), seems to fix all the hotplug regressions as per
nearly a 100 18*SRCU-P runs over-night.

I did clean up the patches, so possibly I wrecked it again. I've started new
runs and will again leave them running over-night.

Paul, if you could please also throw your monster machine at it.



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

* [PATCH 1/8] sched/core: Print out straggler tasks in sched_cpu_dying()
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 2/8] workqueue: Use cpu_possible_mask instead of cpu_active_mask to break affinity Peter Zijlstra
                   ` (8 subsequent siblings)
  9 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

From: Valentin Schneider <valentin.schneider@arm.com>

Since commit

  1cf12e08bc4d ("sched/hotplug: Consolidate task migration on CPU unplug")

tasks are expected to move themselves out of a out-going CPU. For most
tasks this will be done automagically via BALANCE_PUSH, but percpu kthreads
will have to cooperate and move themselves away one way or another.

Currently, some percpu kthreads (workqueues being a notable exemple) do not
cooperate nicely and can end up on an out-going CPU at the time
sched_cpu_dying() is invoked.

Print the dying rq's tasks to shed some light on the stragglers.

Signed-off-by: Valentin Schneider <valentin.schneider@arm.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lkml.kernel.org/r/20210113183141.11974-1-valentin.schneider@arm.com
---
 kernel/sched/core.c |   24 +++++++++++++++++++++++-
 1 file changed, 23 insertions(+), 1 deletion(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -7580,6 +7580,25 @@ static void calc_load_migrate(struct rq
 		atomic_long_add(delta, &calc_load_tasks);
 }
 
+static void dump_rq_tasks(struct rq *rq, const char *loglvl)
+{
+	struct task_struct *g, *p;
+	int cpu = cpu_of(rq);
+
+	lockdep_assert_held(&rq->lock);
+
+	printk("%sCPU%d enqueued tasks (%u total):\n", loglvl, cpu, rq->nr_running);
+	for_each_process_thread(g, p) {
+		if (task_cpu(p) != cpu)
+			continue;
+
+		if (!task_on_rq_queued(p))
+			continue;
+
+		printk("%s\tpid: %d, name: %s\n", loglvl, p->pid, p->comm);
+	}
+}
+
 int sched_cpu_dying(unsigned int cpu)
 {
 	struct rq *rq = cpu_rq(cpu);
@@ -7589,7 +7608,10 @@ int sched_cpu_dying(unsigned int cpu)
 	sched_tick_stop(cpu);
 
 	rq_lock_irqsave(rq, &rf);
-	BUG_ON(rq->nr_running != 1 || rq_has_pinned_tasks(rq));
+	if (rq->nr_running != 1 || rq_has_pinned_tasks(rq)) {
+		WARN(true, "Dying CPU not properly vacated!");
+		dump_rq_tasks(rq, KERN_WARNING);
+	}
 	rq_unlock_irqrestore(rq, &rf);
 
 	calc_load_migrate(rq);



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

* [PATCH 2/8] workqueue: Use cpu_possible_mask instead of cpu_active_mask to break affinity
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 1/8] sched/core: Print out straggler tasks in sched_cpu_dying() Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 3/8] sched: Dont run cpu-online with balance_push() enabled Peter Zijlstra
                   ` (7 subsequent siblings)
  9 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

From: Lai Jiangshan <laijs@linux.alibaba.com>

The scheduler won't break affinity for us any more, and we should
"emulate" the same behavior when the scheduler breaks affinity for
us.  The behavior is "changing the cpumask to cpu_possible_mask".

And there might be some other CPUs online later while the worker is
still running with the pending work items.  The worker should be allowed
to use the later online CPUs as before and process the work items ASAP.
If we use cpu_active_mask here, we can't achieve this goal but
using cpu_possible_mask can.

Fixes: 06249738a41a ("workqueue: Manually break affinity on hotplug")
Signed-off-by: Lai Jiangshan <laijs@linux.alibaba.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Acked-by: Tejun Heo <tj@kernel.org>
Tested-by: Paul E. McKenney <paulmck@kernel.org>
Link: https://lkml.kernel.org/r/20210111152638.2417-4-jiangshanlai@gmail.com
---
 kernel/workqueue.c |    2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -4920,7 +4920,7 @@ static void unbind_workers(int cpu)
 		raw_spin_unlock_irq(&pool->lock);
 
 		for_each_pool_worker(worker, pool)
-			WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, cpu_active_mask) < 0);
+			WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, cpu_possible_mask) < 0);
 
 		mutex_unlock(&wq_pool_attach_mutex);
 



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

* [PATCH 3/8] sched: Dont run cpu-online with balance_push() enabled
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 1/8] sched/core: Print out straggler tasks in sched_cpu_dying() Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 2/8] workqueue: Use cpu_possible_mask instead of cpu_active_mask to break affinity Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 15:27   ` Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 4/8] kthread: Extract KTHREAD_IS_PER_CPU Peter Zijlstra
                   ` (6 subsequent siblings)
  9 siblings, 1 reply; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

We don't need to push away tasks when we come online, mark the push
complete right before the CPU dies.

XXX hotplug state machine has trouble with rollback here.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/sched/core.c |   16 ++++++++++++++--
 1 file changed, 14 insertions(+), 2 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -7320,10 +7320,12 @@ static void balance_push_set(int cpu, bo
 	struct rq_flags rf;
 
 	rq_lock_irqsave(rq, &rf);
-	if (on)
+	if (on) {
+		WARN_ON_ONCE(rq->balance_callback);
 		rq->balance_callback = &balance_push_callback;
-	else
+	} else if (rq->balance_callback == &balance_push_callback) {
 		rq->balance_callback = NULL;
+	}
 	rq_unlock_irqrestore(rq, &rf);
 }
 
@@ -7441,6 +7443,10 @@ int sched_cpu_activate(unsigned int cpu)
 	struct rq *rq = cpu_rq(cpu);
 	struct rq_flags rf;
 
+	/*
+	 * Make sure that when the hotplug state machine does a roll-back
+	 * we clear balance_push. Ideally that would happen earlier...
+	 */
 	balance_push_set(cpu, false);
 
 #ifdef CONFIG_SCHED_SMT
@@ -7608,6 +7614,12 @@ int sched_cpu_dying(unsigned int cpu)
 	}
 	rq_unlock_irqrestore(rq, &rf);
 
+	/*
+	 * Should really be after we clear cpu_online(), but we're in
+	 * stop_machine(), so it all works.
+	 */
+	balance_push_set(cpu, false);
+
 	calc_load_migrate(rq);
 	update_max_interval();
 	nohz_balance_exit_idle(rq);



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

* [PATCH 4/8] kthread: Extract KTHREAD_IS_PER_CPU
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (2 preceding siblings ...)
  2021-01-16 11:30 ` [PATCH 3/8] sched: Dont run cpu-online with balance_push() enabled Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 5/8] workqueue: Tag bound workers with KTHREAD_IS_PER_CPU Peter Zijlstra
                   ` (5 subsequent siblings)
  9 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

There is a need to distinguish geniune per-cpu kthreads from kthreads
that happen to have a single CPU affinity.

Geniune per-cpu kthreads are kthreads that are CPU affine for
correctness, these will obviously have PF_KTHREAD set, but must also
have PF_NO_SETAFFINITY set, lest userspace modify their affinity and
ruins things.

However, these two things are not sufficient, PF_NO_SETAFFINITY is
also set on other tasks that have their affinities controlled through
other means, like for instance workqueues.

Therefore another bit is needed; it turns out kthread_create_per_cpu()
already has such a bit: KTHREAD_IS_PER_CPU, which is used to make
kthread_park()/kthread_unpark() work correctly.

Expose this flag and remove the implicit setting of it from
kthread_create_on_cpu(); the io_uring usage of it seems dubious at
best.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/linux/kthread.h |    3 +++
 kernel/kthread.c        |   30 ++++++++++++++++++++++++++++--
 kernel/smpboot.c        |    1 +
 3 files changed, 32 insertions(+), 2 deletions(-)

--- a/include/linux/kthread.h
+++ b/include/linux/kthread.h
@@ -33,6 +33,9 @@ struct task_struct *kthread_create_on_cp
 					  unsigned int cpu,
 					  const char *namefmt);
 
+void kthread_set_per_cpu(struct task_struct *k, int cpu);
+bool kthread_is_per_cpu(struct task_struct *k);
+
 /**
  * kthread_run - create and wake a thread.
  * @threadfn: the function to run until signal_pending(current).
--- a/kernel/kthread.c
+++ b/kernel/kthread.c
@@ -493,11 +494,36 @@ struct task_struct *kthread_create_on_cp
 		return p;
 	kthread_bind(p, cpu);
 	/* CPU hotplug need to bind once again when unparking the thread. */
-	set_bit(KTHREAD_IS_PER_CPU, &to_kthread(p)->flags);
 	to_kthread(p)->cpu = cpu;
 	return p;
 }
 
+void kthread_set_per_cpu(struct task_struct *k, int cpu)
+{
+	struct kthread *kthread = to_kthread(k);
+	if (!kthread)
+		return;
+
+	WARN_ON_ONCE(!(k->flags & PF_NO_SETAFFINITY));
+
+	if (cpu < 0) {
+		clear_bit(KTHREAD_IS_PER_CPU, &kthread->flags);
+		return;
+	}
+
+	kthread->cpu = cpu;
+	set_bit(KTHREAD_IS_PER_CPU, &kthread->flags);
+}
+
+bool kthread_is_per_cpu(struct task_struct *k)
+{
+	struct kthread *kthread = to_kthread(k);
+	if (!kthread)
+		return false;
+
+	return test_bit(KTHREAD_IS_PER_CPU, &kthread->flags);
+}
+
 /**
  * kthread_unpark - unpark a thread created by kthread_create().
  * @k:		thread created by kthread_create().
--- a/kernel/smpboot.c
+++ b/kernel/smpboot.c
@@ -188,6 +188,7 @@ __smpboot_create_thread(struct smp_hotpl
 		kfree(td);
 		return PTR_ERR(tsk);
 	}
+	kthread_set_per_cpu(tsk, cpu);
 	/*
 	 * Park the thread so that it could start right on the CPU
 	 * when it is available.



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

* [PATCH 5/8] workqueue: Tag bound workers with KTHREAD_IS_PER_CPU
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (3 preceding siblings ...)
  2021-01-16 11:30 ` [PATCH 4/8] kthread: Extract KTHREAD_IS_PER_CPU Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 6/8] workqueue: Restrict affinity change to rescuer Peter Zijlstra
                   ` (4 subsequent siblings)
  9 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

Mark the per-cpu workqueue workers as KTHREAD_IS_PER_CPU.

Workqueues have unfortunate semantics in that per-cpu workers are not
default flushed and parked during hotplug, however a subset does
manual flush on hotplug and hard relies on them for correctness.

Therefore play silly games..

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/workqueue.c |   25 ++++++++++++++++++-------
 1 file changed, 18 insertions(+), 7 deletions(-)

--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1861,6 +1861,8 @@ static void worker_attach_to_pool(struct
 	 */
 	if (pool->flags & POOL_DISASSOCIATED)
 		worker->flags |= WORKER_UNBOUND;
+	else
+		kthread_set_per_cpu(worker->task, pool->cpu);
 
 	list_add_tail(&worker->node, &pool->workers);
 	worker->pool = pool;
@@ -1883,6 +1885,7 @@ static void worker_detach_from_pool(stru
 
 	mutex_lock(&wq_pool_attach_mutex);
 
+	kthread_set_per_cpu(worker->task, -1);
 	list_del(&worker->node);
 	worker->pool = NULL;
 
@@ -2368,6 +2371,7 @@ static int worker_thread(void *__worker)
 	/* tell the scheduler that this is a workqueue worker */
 	set_pf_worker(true);
 woke_up:
+	kthread_parkme();
 	raw_spin_lock_irq(&pool->lock);
 
 	/* am I supposed to die? */
@@ -2425,7 +2429,7 @@ static int worker_thread(void *__worker)
 			move_linked_works(work, &worker->scheduled, NULL);
 			process_scheduled_works(worker);
 		}
-	} while (keep_working(pool));
+	} while (keep_working(pool) && !kthread_should_park());
 
 	worker_set_flags(worker, WORKER_PREP);
 sleep:
@@ -2437,9 +2441,12 @@ static int worker_thread(void *__worker)
 	 * event.
 	 */
 	worker_enter_idle(worker);
-	__set_current_state(TASK_IDLE);
+	set_current_state(TASK_IDLE);
 	raw_spin_unlock_irq(&pool->lock);
-	schedule();
+
+	if (!kthread_should_park())
+		schedule();
+
 	goto woke_up;
 }
 
@@ -4919,8 +4926,10 @@ static void unbind_workers(int cpu)
 
 		raw_spin_unlock_irq(&pool->lock);
 
-		for_each_pool_worker(worker, pool)
+		for_each_pool_worker(worker, pool) {
+			kthread_set_per_cpu(worker->task, -1);
 			WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task, cpu_possible_mask) < 0);
+		}
 
 		mutex_unlock(&wq_pool_attach_mutex);
 
@@ -4972,9 +4981,11 @@ static void rebind_workers(struct worker
 	 * 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)
-		WARN_ON_ONCE(set_cpus_allowed_ptr(worker->task,
-						  pool->attrs->cpumask) < 0);
+	for_each_pool_worker(worker, pool) {
+		WARN_ON_ONCE(kthread_park(worker->task) < 0);
+		kthread_set_per_cpu(worker->task, pool->cpu);
+		kthread_unpark(worker->task);
+	}
 
 	raw_spin_lock_irq(&pool->lock);
 



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

* [PATCH 6/8] workqueue: Restrict affinity change to rescuer
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (4 preceding siblings ...)
  2021-01-16 11:30 ` [PATCH 5/8] workqueue: Tag bound workers with KTHREAD_IS_PER_CPU Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 11:30 ` [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread() Peter Zijlstra
                   ` (3 subsequent siblings)
  9 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

create_worker() will already set the right affinity using
kthread_bind_mask(), this means only the rescuer will need to change
it's affinity.

Howveer, while in cpu-hot-unplug a regular task is not allowed to run
on online&&!active as it would be pushed away quite agressively. We
need KTHREAD_IS_PER_CPU to survive in that environment.

Therefore set the affinity after getting that magic flag.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/workqueue.c |    9 +++------
 1 file changed, 3 insertions(+), 6 deletions(-)

--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1849,12 +1849,6 @@ static void worker_attach_to_pool(struct
 	mutex_lock(&wq_pool_attach_mutex);
 
 	/*
-	 * set_cpus_allowed_ptr() will fail if the cpumask doesn't have any
-	 * online CPUs.  It'll be re-applied when any of the CPUs come up.
-	 */
-	set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
-
-	/*
 	 * The wq_pool_attach_mutex ensures %POOL_DISASSOCIATED remains
 	 * stable across this function.  See the comments above the flag
 	 * definition for details.
@@ -1864,6 +1858,9 @@ static void worker_attach_to_pool(struct
 	else
 		kthread_set_per_cpu(worker->task, pool->cpu);
 
+	if (worker->rescue_wq)
+		set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
+
 	list_add_tail(&worker->node, &pool->workers);
 	worker->pool = pool;
 



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

* [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread()
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (5 preceding siblings ...)
  2021-01-16 11:30 ` [PATCH 6/8] workqueue: Restrict affinity change to rescuer Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-17 16:57   ` Valentin Schneider
  2021-01-16 11:30 ` [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics Peter Zijlstra
                   ` (2 subsequent siblings)
  9 siblings, 1 reply; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

Prior to commit 1cf12e08bc4d ("sched/hotplug: Consolidate task
migration on CPU unplug") we'd leave any task on the dying CPU and
break affinity and force them off at the very end.

This scheme had to change in order to enable migrate_disable(). One
cannot wait for migrate_disable() to complete while stuck in
stop_machine(). Furthermore, since we need at the very least: idle,
hotplug and stop threads at any point before stop_machine, we can't
break affinity and/or push those away.

Under the assumption that all per-cpu kthreads are sanely handled by
CPU hotplug, the new code no long breaks affinity or migrates any of
them (which then includes the critical ones above).

However, there's an important difference between per-cpu kthreads and
kthreads that happen to have a single CPU affinity which is lost. The
latter class very much relies on the forced affinity breaking and
migration semantics previously provided.

Use the new kthread_is_per_cpu() infrastructure to tighten
is_per_cpu_kthread() and fix the hot-unplug problems stemming from the
change.

Fixes: 1cf12e08bc4d ("sched/hotplug: Consolidate task migration on CPU unplug")
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/sched/core.c |   29 +++++++++++++++++++++++++----
 1 file changed, 25 insertions(+), 4 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -1796,13 +1796,28 @@ static inline bool rq_has_pinned_tasks(s
  */
 static inline bool is_cpu_allowed(struct task_struct *p, int cpu)
 {
+	/* When not in the task's cpumask, no point in looking further. */
 	if (!cpumask_test_cpu(cpu, p->cpus_ptr))
 		return false;
 
-	if (is_per_cpu_kthread(p) || is_migration_disabled(p))
+	/* migrate_disabled() must be allowed to finish. */
+	if (is_migration_disabled(p))
 		return cpu_online(cpu);
 
-	return cpu_active(cpu);
+	/* Non kernel threads are not allowed during either online or offline. */
+	if (!(p->flags & PF_KTHREAD))
+		return cpu_active(cpu);
+
+	/* KTHREAD_IS_PER_CPU is always allowed. */
+	if (kthread_is_per_cpu(p))
+		return cpu_online(cpu);
+
+	/* Regular kernel threads don't get to stay during offline. */
+	if (cpu_rq(cpu)->balance_callback == &balance_push_callback)
+		return cpu_active(cpu);
+
+	/* But are allowed during online. */
+	return cpu_online(cpu);
 }
 
 /*
@@ -7276,8 +7291,14 @@ static void balance_push(struct rq *rq)
 	/*
 	 * Both the cpu-hotplug and stop task are in this case and are
 	 * required to complete the hotplug process.
+	 *
+	 * XXX: the idle task does not match kthread_is_per_cpu() due to
+	 * histerical raisins.
 	 */
-	if (is_per_cpu_kthread(push_task) || is_migration_disabled(push_task)) {
+	if (rq->idle == push_task ||
+	    ((push_task->flags & PF_KTHREAD) && kthread_is_per_cpu(push_task)) ||
+	    is_migration_disabled(push_task)) {
+
 		/*
 		 * If this is the idle task on the outgoing CPU try to wake
 		 * up the hotplug control thread which might wait for the
@@ -7309,7 +7330,7 @@ static void balance_push(struct rq *rq)
 	/*
 	 * At this point need_resched() is true and we'll take the loop in
 	 * schedule(). The next pick is obviously going to be the stop task
-	 * which is_per_cpu_kthread() and will push this task away.
+	 * which kthread_is_per_cpu() and will push this task away.
 	 */
 	raw_spin_lock(&rq->lock);
 }



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

* [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (6 preceding siblings ...)
  2021-01-16 11:30 ` [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread() Peter Zijlstra
@ 2021-01-16 11:30 ` Peter Zijlstra
  2021-01-16 14:39   ` Lai Jiangshan
  2021-01-16 15:25 ` [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
  2021-01-16 15:48 ` Paul E. McKenney
  9 siblings, 1 reply; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 11:30 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj,
	peterz

Now that we have KTHREAD_IS_PER_CPU to denote the critical per-cpu
tasks to retain during CPU offline, we can relax the warning in
set_cpus_allowed_ptr(). Any spurious kthread that wants to get on at
the last minute will get pushed off before it can run.

While during CPU online there is no harm, and actual benefit, to
allowing kthreads back on early, it simplifies hotplug code.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/sched/core.c |   20 +++++++++-----------
 1 file changed, 9 insertions(+), 11 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -2342,7 +2342,9 @@ static int __set_cpus_allowed_ptr(struct
 
 	if (p->flags & PF_KTHREAD || is_migration_disabled(p)) {
 		/*
-		 * Kernel threads are allowed on online && !active CPUs.
+		 * Kernel threads are allowed on online && !active CPUs,
+		 * however, during cpu-hot-unplug, even these might get pushed
+		 * away if not KTHREAD_IS_PER_CPU.
 		 *
 		 * Specifically, migration_disabled() tasks must not fail the
 		 * cpumask_any_and_distribute() pick below, esp. so on
@@ -2386,16 +2388,6 @@ static int __set_cpus_allowed_ptr(struct
 
 	__do_set_cpus_allowed(p, new_mask, flags);
 
-	if (p->flags & PF_KTHREAD) {
-		/*
-		 * For kernel threads that do indeed end up on online &&
-		 * !active we want to ensure they are strict per-CPU threads.
-		 */
-		WARN_ON(cpumask_intersects(new_mask, cpu_online_mask) &&
-			!cpumask_intersects(new_mask, cpu_active_mask) &&
-			p->nr_cpus_allowed != 1);
-	}
-
 	return affine_move_task(rq, p, &rf, dest_cpu, flags);
 
 out:
@@ -7519,6 +7511,12 @@ int sched_cpu_deactivate(unsigned int cp
 	 */
 	synchronize_rcu();
 
+	/*
+	 * From this point forward, this CPU will refuse to run any task that
+	 * is not: migrate_disable() or KTHREAD_IS_PER_CPU, and will actively
+	 * push those tasks away until this gets cleared, see
+	 * sched_cpu_dying().
+	 */
 	balance_push_set(cpu, true);
 
 	rq_lock_irqsave(rq, &rf);



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

* Re: [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics
  2021-01-16 11:30 ` [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics Peter Zijlstra
@ 2021-01-16 14:39   ` Lai Jiangshan
  2021-01-16 15:19     ` Peter Zijlstra
  0 siblings, 1 reply; 19+ messages in thread
From: Lai Jiangshan @ 2021-01-16 14:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ingo Molnar, Thomas Gleixner, LKML, Valentin Schneider, Qian Cai,
	Vincent Donnefort, Dexuan Cui, Paul E. McKenney, Vincent Guittot,
	Steven Rostedt, Tejun Heo

On Sat, Jan 16, 2021 at 7:43 PM Peter Zijlstra <peterz@infradead.org> wrote:
>
> Now that we have KTHREAD_IS_PER_CPU to denote the critical per-cpu
> tasks to retain during CPU offline, we can relax the warning in
> set_cpus_allowed_ptr(). Any spurious kthread that wants to get on at
> the last minute will get pushed off before it can run.
>
> While during CPU online there is no harm, and actual benefit, to
> allowing kthreads back on early, it simplifies hotplug code.
>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>

Thanks!

Relaxing set_cpus_allowed_ptr() was also one of the choices I listed,
which can really simplify hotplug code in the workqueue and may be
other hotplug code.

Reviewed-by: Lai jiangshan <jiangshanlai@gmail.com>

> ---
>  kernel/sched/core.c |   20 +++++++++-----------
>  1 file changed, 9 insertions(+), 11 deletions(-)
>
> --- a/kernel/sched/core.c
> +++ b/kernel/sched/core.c
> @@ -2342,7 +2342,9 @@ static int __set_cpus_allowed_ptr(struct
>
>         if (p->flags & PF_KTHREAD || is_migration_disabled(p)) {
>                 /*
> -                * Kernel threads are allowed on online && !active CPUs.
> +                * Kernel threads are allowed on online && !active CPUs,
> +                * however, during cpu-hot-unplug, even these might get pushed
> +                * away if not KTHREAD_IS_PER_CPU.
>                  *
>                  * Specifically, migration_disabled() tasks must not fail the
>                  * cpumask_any_and_distribute() pick below, esp. so on
> @@ -2386,16 +2388,6 @@ static int __set_cpus_allowed_ptr(struct
>
>         __do_set_cpus_allowed(p, new_mask, flags);
>
> -       if (p->flags & PF_KTHREAD) {
> -               /*
> -                * For kernel threads that do indeed end up on online &&
> -                * !active we want to ensure they are strict per-CPU threads.
> -                */
> -               WARN_ON(cpumask_intersects(new_mask, cpu_online_mask) &&
> -                       !cpumask_intersects(new_mask, cpu_active_mask) &&
> -                       p->nr_cpus_allowed != 1);
> -       }
> -
>         return affine_move_task(rq, p, &rf, dest_cpu, flags);
>
>  out:
> @@ -7519,6 +7511,12 @@ int sched_cpu_deactivate(unsigned int cp
>          */
>         synchronize_rcu();
>
> +       /*
> +        * From this point forward, this CPU will refuse to run any task that
> +        * is not: migrate_disable() or KTHREAD_IS_PER_CPU, and will actively
> +        * push those tasks away until this gets cleared, see
> +        * sched_cpu_dying().
> +        */
>         balance_push_set(cpu, true);
>
>         rq_lock_irqsave(rq, &rf);
>
>

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

* Re: [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics
  2021-01-16 14:39   ` Lai Jiangshan
@ 2021-01-16 15:19     ` Peter Zijlstra
  0 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 15:19 UTC (permalink / raw)
  To: Lai Jiangshan
  Cc: Ingo Molnar, Thomas Gleixner, LKML, Valentin Schneider, Qian Cai,
	Vincent Donnefort, Dexuan Cui, Paul E. McKenney, Vincent Guittot,
	Steven Rostedt, Tejun Heo

On Sat, Jan 16, 2021 at 10:39:03PM +0800, Lai Jiangshan wrote:
> On Sat, Jan 16, 2021 at 7:43 PM Peter Zijlstra <peterz@infradead.org> wrote:
> >
> > Now that we have KTHREAD_IS_PER_CPU to denote the critical per-cpu
> > tasks to retain during CPU offline, we can relax the warning in
> > set_cpus_allowed_ptr(). Any spurious kthread that wants to get on at
> > the last minute will get pushed off before it can run.
> >
> > While during CPU online there is no harm, and actual benefit, to
> > allowing kthreads back on early, it simplifies hotplug code.
> >
> > Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> 
> Thanks!
> 
> Relaxing set_cpus_allowed_ptr() was also one of the choices I listed,
> which can really simplify hotplug code in the workqueue and may be
> other hotplug code.

Indeed you did. Having that KTHREAD_IS_PER_CPU for the offline side of
things made it possible to relax (as per the Changelog above).

> Reviewed-by: Lai jiangshan <jiangshanlai@gmail.com>

Thanks!

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

* Re: [PATCH 0/8] sched: Fix hot-unplug regressions
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (7 preceding siblings ...)
  2021-01-16 11:30 ` [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics Peter Zijlstra
@ 2021-01-16 15:25 ` Peter Zijlstra
  2021-01-16 15:45   ` Paul E. McKenney
  2021-01-16 15:48 ` Paul E. McKenney
  9 siblings, 1 reply; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 15:25 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj

On Sat, Jan 16, 2021 at 12:30:33PM +0100, Peter Zijlstra wrote:
> Hi,
> 
> These patches (no longer 4), seems to fix all the hotplug regressions as per
> nearly a 100 18*SRCU-P runs over-night.
> 
> I did clean up the patches, so possibly I wrecked it again. I've started new
> runs and will again leave them running over-night.

Hurph... I've got one splat from this version, one I've not seen before:

[   68.712848] Dying CPU not properly vacated!
...
[   68.744448] CPU1 enqueued tasks (2 total):
[   68.745018]  pid: 14, name: rcu_preempt
[   68.745557]  pid: 18, name: migration/1

Paul, rcu_preempt, is from rcu_spawn_gp_kthread(), right? Afaict that
doesn't even have affinity.. /me wonders HTH that ended up on the
runqueue so late.


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

* Re: [PATCH 3/8] sched: Dont run cpu-online with balance_push() enabled
  2021-01-16 11:30 ` [PATCH 3/8] sched: Dont run cpu-online with balance_push() enabled Peter Zijlstra
@ 2021-01-16 15:27   ` Peter Zijlstra
  0 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 15:27 UTC (permalink / raw)
  To: mingo, tglx
  Cc: linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, paulmck, vincent.guittot, rostedt, tj

On Sat, Jan 16, 2021 at 12:30:36PM +0100, Peter Zijlstra wrote:
> @@ -7608,6 +7614,12 @@ int sched_cpu_dying(unsigned int cpu)
>  	}
>  	rq_unlock_irqrestore(rq, &rf);
>  
> +	/*
> +	 * Should really be after we clear cpu_online(), but we're in
> +	 * stop_machine(), so it all works.
> +	 */
> +	balance_push_set(cpu, false);

Looking at the RCU thing just now made me realize we run all the DYING
notifiers with cpu_online() already false, so the above comment is wrong
and ordering in fact perfect.



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

* Re: [PATCH 0/8] sched: Fix hot-unplug regressions
  2021-01-16 15:25 ` [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
@ 2021-01-16 15:45   ` Paul E. McKenney
  2021-01-16 18:51     ` Peter Zijlstra
  0 siblings, 1 reply; 19+ messages in thread
From: Paul E. McKenney @ 2021-01-16 15:45 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, vincent.guittot, rostedt, tj

On Sat, Jan 16, 2021 at 04:25:58PM +0100, Peter Zijlstra wrote:
> On Sat, Jan 16, 2021 at 12:30:33PM +0100, Peter Zijlstra wrote:
> > Hi,
> > 
> > These patches (no longer 4), seems to fix all the hotplug regressions as per
> > nearly a 100 18*SRCU-P runs over-night.
> > 
> > I did clean up the patches, so possibly I wrecked it again. I've started new
> > runs and will again leave them running over-night.
> 
> Hurph... I've got one splat from this version, one I've not seen before:
> 
> [   68.712848] Dying CPU not properly vacated!
> ...
> [   68.744448] CPU1 enqueued tasks (2 total):
> [   68.745018]  pid: 14, name: rcu_preempt
> [   68.745557]  pid: 18, name: migration/1
> 
> Paul, rcu_preempt, is from rcu_spawn_gp_kthread(), right? Afaict that
> doesn't even have affinity.. /me wonders HTH that ended up on the
> runqueue so late.

Yes, rcu_preempt is from rcu_spawn_gp_kthread(), and you are right that
the kernel code does not bind it anywhere.  If this is rcutorture,
there isn't enough of a userspace to do the binding there, eihter.
Wakeups for the rcu_preempt task can happen in odd places, though.

Grasping at straws...  Would Frederic's series help?  This is in
-rcu here:

cfd941c rcu/nocb: Detect unsafe checks for offloaded rdp
028d407 rcu: Remove superfluous rdp fetch
38e216a rcu: Pull deferred rcuog wake up to rcu_eqs_enter() callers
53775fd rcu/nocb: Perform deferred wake up before last idle's need_resched() check
1fbabce rcu/nocb: Trigger self-IPI on late deferred wake up before user resume
2856844 entry: Explicitly flush pending rcuog wakeup before last rescheduling points
4d959df sched: Report local wake up on resched blind zone within idle loop
2617331 entry: Report local wake up on resched blind zone while resuming to user
79acd12 timer: Report ignored local enqueue in nohz mode

I have been including these in all of my tests of your patches.

							Thanx, Paul

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

* Re: [PATCH 0/8] sched: Fix hot-unplug regressions
  2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
                   ` (8 preceding siblings ...)
  2021-01-16 15:25 ` [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
@ 2021-01-16 15:48 ` Paul E. McKenney
  2021-01-18  5:28   ` Paul E. McKenney
  9 siblings, 1 reply; 19+ messages in thread
From: Paul E. McKenney @ 2021-01-16 15:48 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, vincent.guittot, rostedt, tj

On Sat, Jan 16, 2021 at 12:30:33PM +0100, Peter Zijlstra wrote:
> Hi,
> 
> These patches (no longer 4), seems to fix all the hotplug regressions as per
> nearly a 100 18*SRCU-P runs over-night.

Nice!!!

> I did clean up the patches, so possibly I wrecked it again. I've started new
> runs and will again leave them running over-night.
> 
> Paul, if you could please also throw your monster machine at it.

Will do as soon as the tests I started yesterday complete, which should
be this afternoon, Pacific Time.

My thought is to do the full set of scenarios overnight, then try
hammering either SRCU-P and/or whatever else shows up in the overnight
test.  Seem reasonable?

							Thanx, Paul

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

* Re: [PATCH 0/8] sched: Fix hot-unplug regressions
  2021-01-16 15:45   ` Paul E. McKenney
@ 2021-01-16 18:51     ` Peter Zijlstra
  0 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-16 18:51 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: mingo, tglx, linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, vincent.guittot, rostedt, tj

On Sat, Jan 16, 2021 at 07:45:42AM -0800, Paul E. McKenney wrote:
> On Sat, Jan 16, 2021 at 04:25:58PM +0100, Peter Zijlstra wrote:
> > On Sat, Jan 16, 2021 at 12:30:33PM +0100, Peter Zijlstra wrote:
> > > Hi,
> > > 
> > > These patches (no longer 4), seems to fix all the hotplug regressions as per
> > > nearly a 100 18*SRCU-P runs over-night.
> > > 
> > > I did clean up the patches, so possibly I wrecked it again. I've started new
> > > runs and will again leave them running over-night.
> > 
> > Hurph... I've got one splat from this version, one I've not seen before:
> > 
> > [   68.712848] Dying CPU not properly vacated!
> > ...
> > [   68.744448] CPU1 enqueued tasks (2 total):
> > [   68.745018]  pid: 14, name: rcu_preempt
> > [   68.745557]  pid: 18, name: migration/1
> > 
> > Paul, rcu_preempt, is from rcu_spawn_gp_kthread(), right? Afaict that
> > doesn't even have affinity.. /me wonders HTH that ended up on the
> > runqueue so late.
> 
> Yes, rcu_preempt is from rcu_spawn_gp_kthread(), and you are right that
> the kernel code does not bind it anywhere.  If this is rcutorture,
> there isn't enough of a userspace to do the binding there, eihter.
> Wakeups for the rcu_preempt task can happen in odd places, though.
> 
> Grasping at straws...

My current straw is that the wakeup lands on the wakelist before ttwu()
will refuse to wake to the CPU, and then lands on the RQ after we've
waited. Which seems near impossible..

I'll keep staring..

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

* Re: [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread()
  2021-01-16 11:30 ` [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread() Peter Zijlstra
@ 2021-01-17 16:57   ` Valentin Schneider
  2021-01-18  9:30     ` Peter Zijlstra
  0 siblings, 1 reply; 19+ messages in thread
From: Valentin Schneider @ 2021-01-17 16:57 UTC (permalink / raw)
  To: Peter Zijlstra, mingo, tglx
  Cc: linux-kernel, jiangshanlai, cai, vincent.donnefort, decui,
	paulmck, vincent.guittot, rostedt, tj, peterz

On 16/01/21 12:30, Peter Zijlstra wrote:
> @@ -1796,13 +1796,28 @@ static inline bool rq_has_pinned_tasks(s
>   */
>  static inline bool is_cpu_allowed(struct task_struct *p, int cpu)
>  {
> +	/* When not in the task's cpumask, no point in looking further. */
>       if (!cpumask_test_cpu(cpu, p->cpus_ptr))
>               return false;
>
> -	if (is_per_cpu_kthread(p) || is_migration_disabled(p))
> +	/* migrate_disabled() must be allowed to finish. */
> +	if (is_migration_disabled(p))
>               return cpu_online(cpu);
>
> -	return cpu_active(cpu);
> +	/* Non kernel threads are not allowed during either online or offline. */
> +	if (!(p->flags & PF_KTHREAD))
> +		return cpu_active(cpu);
> +
> +	/* KTHREAD_IS_PER_CPU is always allowed. */
> +	if (kthread_is_per_cpu(p))
> +		return cpu_online(cpu);
> +
> +	/* Regular kernel threads don't get to stay during offline. */
> +	if (cpu_rq(cpu)->balance_callback == &balance_push_callback)
> +		return cpu_active(cpu);

is_cpu_allowed(, cpu) isn't guaranteed to have cpu_rq(cpu)'s rq_lock
held, so this can race with balance_push_set(, true). This shouldn't
matter under normal circumstances as we'll have sched_cpu_wait_empty()
further down the line.

This might get ugly with the rollback faff - this is jumping the gun a
bit, but that's something we'll have to address, and I think what I'm
concerned about is close to what you mentioned in

  http://lore.kernel.org/r/YAM1t2Qzr7Rib3bN@hirez.programming.kicks-ass.net

Here's what I'm thinking of:

_cpu_up()                            ttwu()
                                       select_task_rq()
                                         is_cpu_allowed()
                                           rq->balance_callback != balance_push_callback
  smpboot_unpark_threads() // FAIL
  (now going down, set push here)
  sched_cpu_wait_empty()
  ...                                  ttwu_queue()
  sched_cpu_dying()
  *ARGH*

I've written some horrors on top of this series here:

  https://gitlab.arm.com/linux-arm/linux-vs/-/commits/mainline/migrate_disable/stragglers/

Also, my TX2 is again in need of CPR, so in the meantime I'm running
tests on a (much) smaller machine...

> +
> +	/* But are allowed during online. */
> +	return cpu_online(cpu);
>  }

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

* Re: [PATCH 0/8] sched: Fix hot-unplug regressions
  2021-01-16 15:48 ` Paul E. McKenney
@ 2021-01-18  5:28   ` Paul E. McKenney
  0 siblings, 0 replies; 19+ messages in thread
From: Paul E. McKenney @ 2021-01-18  5:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, linux-kernel, jiangshanlai, valentin.schneider, cai,
	vincent.donnefort, decui, vincent.guittot, rostedt, tj

On Sat, Jan 16, 2021 at 07:48:59AM -0800, Paul E. McKenney wrote:
> On Sat, Jan 16, 2021 at 12:30:33PM +0100, Peter Zijlstra wrote:
> > Hi,
> > 
> > These patches (no longer 4), seems to fix all the hotplug regressions as per
> > nearly a 100 18*SRCU-P runs over-night.
> 
> Nice!!!
> 
> > I did clean up the patches, so possibly I wrecked it again. I've started new
> > runs and will again leave them running over-night.
> > 
> > Paul, if you could please also throw your monster machine at it.
> 
> Will do as soon as the tests I started yesterday complete, which should
> be this afternoon, Pacific Time.
> 
> My thought is to do the full set of scenarios overnight, then try
> hammering either SRCU-P and/or whatever else shows up in the overnight
> test.  Seem reasonable?

And the SRCU-P runs did fine.  I got some task hangs on TREE03 and (to
a lesser extent) TREE04.  These might well be my fault, so I will try
bisecting tomorrow, Pacific Time.

							Thanx, Paul

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

* Re: [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread()
  2021-01-17 16:57   ` Valentin Schneider
@ 2021-01-18  9:30     ` Peter Zijlstra
  0 siblings, 0 replies; 19+ messages in thread
From: Peter Zijlstra @ 2021-01-18  9:30 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: mingo, tglx, linux-kernel, jiangshanlai, cai, vincent.donnefort,
	decui, paulmck, vincent.guittot, rostedt, tj

On Sun, Jan 17, 2021 at 04:57:27PM +0000, Valentin Schneider wrote:
> On 16/01/21 12:30, Peter Zijlstra wrote:
> > @@ -1796,13 +1796,28 @@ static inline bool rq_has_pinned_tasks(s
> >   */
> >  static inline bool is_cpu_allowed(struct task_struct *p, int cpu)
> >  {
> > +	/* When not in the task's cpumask, no point in looking further. */
> >       if (!cpumask_test_cpu(cpu, p->cpus_ptr))
> >               return false;
> >
> > +	/* migrate_disabled() must be allowed to finish. */
> > +	if (is_migration_disabled(p))
> >               return cpu_online(cpu);
> >
> > +	/* Non kernel threads are not allowed during either online or offline. */
> > +	if (!(p->flags & PF_KTHREAD))
> > +		return cpu_active(cpu);
> > +
> > +	/* KTHREAD_IS_PER_CPU is always allowed. */
> > +	if (kthread_is_per_cpu(p))
> > +		return cpu_online(cpu);
> > +
> > +	/* Regular kernel threads don't get to stay during offline. */
> > +	if (cpu_rq(cpu)->balance_callback == &balance_push_callback)
> > +		return cpu_active(cpu);
> 
> is_cpu_allowed(, cpu) isn't guaranteed to have cpu_rq(cpu)'s rq_lock
> held, so this can race with balance_push_set(, true). This shouldn't
> matter under normal circumstances as we'll have sched_cpu_wait_empty()
> further down the line.
> 
> This might get ugly with the rollback faff - this is jumping the gun a
> bit, but that's something we'll have to address, and I think what I'm
> concerned about is close to what you mentioned in
> 
>   http://lore.kernel.org/r/YAM1t2Qzr7Rib3bN@hirez.programming.kicks-ass.net
> 
> Here's what I'm thinking of:
> 
> _cpu_up()                            ttwu()
>                                        select_task_rq()
>                                          is_cpu_allowed()
>                                            rq->balance_callback != balance_push_callback
>   smpboot_unpark_threads() // FAIL
>   (now going down, set push here)
>   sched_cpu_wait_empty()
>   ...                                  ttwu_queue()
>   sched_cpu_dying()
>   *ARGH*
> 

Let me try this then...

diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 5057054b1cff..9b045296d646 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -7495,6 +7495,8 @@ int sched_cpu_activate(unsigned int cpu)
 	return 0;
 }
 
+unsigned long sched_cpu_rcu_state;
+
 int sched_cpu_deactivate(unsigned int cpu)
 {
 	struct rq *rq = cpu_rq(cpu);
@@ -7519,6 +7521,11 @@ int sched_cpu_deactivate(unsigned int cpu)
 	 */
 	balance_push_set(cpu, true);
 
+	/*
+	 * See sched_cpu_wait_empty().
+	 */
+	sched_cpu_rcu_state = get_state_synchronize_rcu();
+
 	rq_lock_irqsave(rq, &rf);
 	if (rq->rd) {
 		update_rq_clock(rq);
@@ -7578,6 +7585,12 @@ int sched_cpu_starting(unsigned int cpu)
  */
 int sched_cpu_wait_empty(unsigned int cpu)
 {
+	/*
+	 * Guarantee that TTWU will observe balance_push_set(true),
+	 * such that all wakeups will refuse this CPU.
+	 */
+	cond_synchronize_rcu(sched_cpu_rcu_state);
+
 	balance_hotplug_wait();
 	return 0;
 }

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

end of thread, other threads:[~2021-01-18  9:47 UTC | newest]

Thread overview: 19+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-01-16 11:30 [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
2021-01-16 11:30 ` [PATCH 1/8] sched/core: Print out straggler tasks in sched_cpu_dying() Peter Zijlstra
2021-01-16 11:30 ` [PATCH 2/8] workqueue: Use cpu_possible_mask instead of cpu_active_mask to break affinity Peter Zijlstra
2021-01-16 11:30 ` [PATCH 3/8] sched: Dont run cpu-online with balance_push() enabled Peter Zijlstra
2021-01-16 15:27   ` Peter Zijlstra
2021-01-16 11:30 ` [PATCH 4/8] kthread: Extract KTHREAD_IS_PER_CPU Peter Zijlstra
2021-01-16 11:30 ` [PATCH 5/8] workqueue: Tag bound workers with KTHREAD_IS_PER_CPU Peter Zijlstra
2021-01-16 11:30 ` [PATCH 6/8] workqueue: Restrict affinity change to rescuer Peter Zijlstra
2021-01-16 11:30 ` [PATCH 7/8] sched: Fix CPU hotplug / tighten is_per_cpu_kthread() Peter Zijlstra
2021-01-17 16:57   ` Valentin Schneider
2021-01-18  9:30     ` Peter Zijlstra
2021-01-16 11:30 ` [PATCH 8/8] sched: Relax the set_cpus_allowed_ptr() semantics Peter Zijlstra
2021-01-16 14:39   ` Lai Jiangshan
2021-01-16 15:19     ` Peter Zijlstra
2021-01-16 15:25 ` [PATCH 0/8] sched: Fix hot-unplug regressions Peter Zijlstra
2021-01-16 15:45   ` Paul E. McKenney
2021-01-16 18:51     ` Peter Zijlstra
2021-01-16 15:48 ` Paul E. McKenney
2021-01-18  5:28   ` Paul E. McKenney

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