All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC][PATCH] sched: Better document ttwu()
@ 2020-07-02 12:52 Peter Zijlstra
  2020-07-02 13:13 ` Phil Auld
                   ` (4 more replies)
  0 siblings, 5 replies; 9+ messages in thread
From: Peter Zijlstra @ 2020-07-02 12:52 UTC (permalink / raw)
  To: Ingo Molnar; +Cc: linux-kernel, vincent.guittot, mgorman, Oleg Nesterov, david


Dave hit the problem fixed by commit:

  b6e13e85829f ("sched/core: Fix ttwu() race")

and failed to understand much of the code involved. Per his request a
few comments to (hopefully) clarify things.

Requested-by: Dave Chinner <david@fromorbit.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/linux/sched.h |  12 ++--
 kernel/sched/core.c   | 195 +++++++++++++++++++++++++++++++++++++++++++-------
 kernel/sched/sched.h  |  11 +++
 3 files changed, 187 insertions(+), 31 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 9bd073a10224..ad36f70bef24 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -158,24 +158,24 @@ struct task_group;
  *
  *   for (;;) {
  *	set_current_state(TASK_UNINTERRUPTIBLE);
- *	if (!need_sleep)
- *		break;
+ *	if (CONDITION)
+ *	   break;
  *
  *	schedule();
  *   }
  *   __set_current_state(TASK_RUNNING);
  *
  * If the caller does not need such serialisation (because, for instance, the
- * condition test and condition change and wakeup are under the same lock) then
+ * CONDITION test and condition change and wakeup are under the same lock) then
  * use __set_current_state().
  *
  * The above is typically ordered against the wakeup, which does:
  *
- *   need_sleep = false;
+ *   CONDITION = 1;
  *   wake_up_state(p, TASK_UNINTERRUPTIBLE);
  *
- * where wake_up_state() executes a full memory barrier before accessing the
- * task state.
+ * where wake_up_state()/try_to_wake_up() executes a full memory barrier before
+ * accessing p->state.
  *
  * Wakeup will do: if (@state & p->state) p->state = TASK_RUNNING, that is,
  * once it observes the TASK_UNINTERRUPTIBLE store the waking CPU can issue a
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 1d3d2d67f398..0cd6c336029f 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -77,6 +77,97 @@ __read_mostly int scheduler_running;
  */
 int sysctl_sched_rt_runtime = 950000;
 
+
+/*
+ * Serialization rules:
+ *
+ * Lock order:
+ *
+ *   p->pi_lock
+ *     rq->lock
+ *       hrtimer_cpu_base->lock (hrtimer_start() for bandwidth controls)
+ *
+ *  rq1->lock
+ *    rq2->lock  where: rq1 < rq2
+ *
+ * Regular state:
+ *
+ * Normal scheduling state is serialized by rq->lock. __schedule() takes the
+ * local CPU's rq->lock, it optionally removes the task from the runqueue and
+ * always looks at the local rq data structures to find the most elegible task
+ * to run next.
+ *
+ * Task enqueue is also under rq->lock, possibly taken from another CPU.
+ * Wakeups from another LLC domain might use an IPI to transfer the enqueue to
+ * the local CPU to avoid bouncing the runqueue state around [ see
+ * ttwu_queue_wakelist() ]
+ *
+ * Task wakeup, specifically wakeups that involve migration, are horribly
+ * complicated to avoid having to take two rq->locks.
+ *
+ * Special state:
+ *
+ * System-calls and anything external will use task_rq_lock() which acquires
+ * both p->lock and rq->lock. As a consequence the state they change is stable
+ * while holding either lock:
+ *
+ *  - sched_setaffinity():	p->cpus_ptr
+ *  - set_user_nice():		p->se.load, p->static_prio
+ *  - __sched_setscheduler():	p->sched_class, p->policy, p->*prio, p->se.load,
+ *				p->dl.dl_{runtime, deadline, period, flags, bw, density}
+ *  - sched_setnuma():		p->numa_preferred_nid
+ *  - sched_move_task()/
+ *    cpu_cgroup_fork():	p->sched_task_group
+ *
+ * p->state <- TASK_*:
+ *
+ *   is changed locklessly using set_current_state(), __set_current_state() or
+ *   set_special_state(), see their respective comments, or by
+ *   try_to_wake_up(). This latter uses p->pi_lock to serialize against
+ *   concurrent self.
+ *
+ * p->on_rq <- { 0, 1 = TASK_ON_RQ_QUEUED, 2 = TASK_ON_RQ_MIGRATING }:
+ *
+ *   is set by activate_task() and cleared by deactivate_task(), under
+ *   rq->lock. Non-zero indicates the task is runnable, the special
+ *   ON_RQ_MIGRATING state is used for migration without holding both
+ *   rq->locks. It indicates task_cpu() is not stable, see task_rq_lock().
+ *
+ * p->on_cpu <- { 0, 1 }:
+ *
+ *   is set by prepare_task() and cleared by finish_task() such that it will be
+ *   set before p is scheduled-in and cleared after p is scheduled-out, both
+ *   under rq->lock. Non-zero indicates the task is running on it's CPU.
+ *
+ *   [ The astute reader will observe that it is possible for two tasks on one
+ *     CPU to have ->on_cpu = 1 at the same time. ]
+ *
+ * task_cpu(p): is changed by set_task_cpu(), the rules are:
+ *
+ *  - Don't call set_task_cpu() on a blocked task:
+ *
+ *    We don't care what CPU we're not running on, this simplifies hotplug,
+ *    the CPU assignment of blocked tasks isn't required to be valid.
+ *
+ *  - for try_to_wake_up(), called under p->pi_lock:
+ *
+ *    This allows try_to_wake_up() to only take one rq->lock, see its comment.
+ *
+ *  - for migration called under rq->lock:
+ *    [ see task_on_rq_migrating() in task_rq_lock() ]
+ *
+ *    o move_queued_task()
+ *    o __migrate_swap_task()
+ *    o detach_task()
+ *
+ *  - for migration called under double_rq_lock():
+ *
+ *    o push_rt_task() / pull_rt_task()
+ *    o push_dl_task() / pull_dl_task()
+ *    o dl_task_offline_migration()
+ *
+ */
+
 /*
  * __task_rq_lock - lock the rq @p resides on.
  */
@@ -1466,8 +1557,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
 {
 	lockdep_assert_held(&rq->lock);
 
-	WRITE_ONCE(p->on_rq, TASK_ON_RQ_MIGRATING);
-	dequeue_task(rq, p, DEQUEUE_NOCLOCK);
+	deactivate_task(rq, p, DEQUEUE_NOCLOCK);
 	set_task_cpu(p, new_cpu);
 	rq_unlock(rq, rf);
 
@@ -1475,8 +1565,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
 
 	rq_lock(rq, rf);
 	BUG_ON(task_cpu(p) != new_cpu);
-	enqueue_task(rq, p, 0);
-	p->on_rq = TASK_ON_RQ_QUEUED;
+	activate_task(rq, p, 0);
 	check_preempt_curr(rq, p, 0);
 
 	return rq;
@@ -2241,12 +2330,31 @@ ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
 }
 
 /*
- * Called in case the task @p isn't fully descheduled from its runqueue,
- * in this case we must do a remote wakeup. Its a 'light' wakeup though,
- * since all we need to do is flip p->state to TASK_RUNNING, since
- * the task is still ->on_rq.
+ * Consider @p being inside a wait loop:
+ *
+ *   for (;;) {
+ *   	set_current_state(TASK_UNINTERRUPTIBLE);
+ *
+ *   	if (CONDITION)
+ *         break;
+ *
+ *   	schedule();
+ *   }
+ *   __set_current_state(TASK_RUNNING);
+ *
+ * between set_current_state() and schedule(). In this case @p is still
+ * runnable, so all that needs doing is change p->state back to TASK_RUNNING in
+ * an atomic manner.
+ *
+ * By taking task_rq(p)->lock we serialize against schedule(), if @p->on_rq
+ * then schedule() must still happen and p->state can be changed to
+ * TASK_RUNNING. Otherwise we lost the race, schedule() has happened, and we
+ * need to do a full wakeup with enqueue.
+ *
+ * Returns: %true when the wakeup is done,
+ *          %false otherwise.
  */
-static int ttwu_remote(struct task_struct *p, int wake_flags)
+static int ttwu_runnable(struct task_struct *p, int wake_flags)
 {
 	struct rq_flags rf;
 	struct rq *rq;
@@ -2387,6 +2495,14 @@ static bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
 
 	return false;
 }
+
+#else /* !CONFIG_SMP */
+
+static inline bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
+{
+	return false;
+}
+
 #endif /* CONFIG_SMP */
 
 static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
@@ -2394,10 +2510,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
 	struct rq *rq = cpu_rq(cpu);
 	struct rq_flags rf;
 
-#if defined(CONFIG_SMP)
 	if (ttwu_queue_wakelist(p, cpu, wake_flags))
 		return;
-#endif
 
 	rq_lock(rq, &rf);
 	update_rq_clock(rq);
@@ -2453,8 +2567,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
  * migration. However the means are completely different as there is no lock
  * chain to provide order. Instead we do:
  *
- *   1) smp_store_release(X->on_cpu, 0)
- *   2) smp_cond_load_acquire(!X->on_cpu)
+ *   1) smp_store_release(X->on_cpu, 0)   -- finish_task()
+ *   2) smp_cond_load_acquire(!X->on_cpu) -- try_to_wake_up()
  *
  * Example:
  *
@@ -2494,15 +2608,41 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
  * @state: the mask of task states that can be woken
  * @wake_flags: wake modifier flags (WF_*)
  *
- * If (@state & @p->state) @p->state = TASK_RUNNING.
+ * Conceptually does:
+ *
+ *   If (@state & @p->state) @p->state = TASK_RUNNING.
  *
  * If the task was not queued/runnable, also place it back on a runqueue.
  *
- * Atomic against schedule() which would dequeue a task, also see
- * set_current_state().
+ * This function:
+ *  - is atomic against schedule() which would dequeue the task;
+ *  - issues a full memory barrier before accessing @p->state.
+ * See the comment with set_current_state().
+ *
+ * Uses p->pi_lock to serialize against concurrent wake-ups.
+ *
+ * Relies on p->pi_lock stabilizing:
+ *  - p->sched_class
+ *  - p->cpus_ptr
+ *  - p->sched_task_group
+ * in order to do migration, see its use of select_task_rq()/set_task_cpu().
  *
- * This function executes a full memory barrier before accessing the task
- * state; see set_current_state().
+ * Tries really hard to only take one task_rq(p)->lock for performance.
+ * Takes rq->lock in:
+ *  - ttwu_runnable()    -- old rq, unavoidable, see comment there;
+ *  - ttwu_queue()       -- new rq, for enqueue of the task;
+ *  - psi_ttwu_dequeue() -- much sadness :-( accounting will kill us.
+ *
+ * As a concequence we race really badly with just about everything. See the
+ * many memory barriers and their comments for details. The basic order of
+ * reading things is:
+ *
+ *   LOAD p->state
+ *   RMB
+ *   LOAD p->on_rq
+ *   RMB
+ *   LOAD-ACQUIRE p->on_cpu
+ *   LOAD task_cpu()
  *
  * Return: %true if @p->state changes (an actual wakeup was done),
  *	   %false otherwise.
@@ -2518,7 +2658,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 		/*
 		 * We're waking current, this means 'p->on_rq' and 'task_cpu(p)
 		 * == smp_processor_id()'. Together this means we can special
-		 * case the whole 'p->on_rq && ttwu_remote()' case below
+		 * case the whole 'p->on_rq && ttwu_runnable()' case below
 		 * without taking any locks.
 		 *
 		 * In particular:
@@ -2539,8 +2679,8 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 	/*
 	 * If we are going to wake up a thread waiting for CONDITION we
 	 * need to ensure that CONDITION=1 done by the caller can not be
-	 * reordered with p->state check below. This pairs with mb() in
-	 * set_current_state() the waiting thread does.
+	 * reordered with p->state check below. This pairs with smp_store_mb()
+	 * in set_current_state() that the waiting thread does.
 	 */
 	raw_spin_lock_irqsave(&p->pi_lock, flags);
 	smp_mb__after_spinlock();
@@ -2575,7 +2715,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 	 * A similar smb_rmb() lives in try_invoke_on_locked_down_task().
 	 */
 	smp_rmb();
-	if (p->on_rq && ttwu_remote(p, wake_flags))
+	if (p->on_rq && ttwu_runnable(p, wake_flags))
 		goto unlock;
 
 	if (p->in_iowait) {
@@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
 	/*
 	 * Claim the task as running, we do this before switching to it
 	 * such that any running task will have this set.
+	 *
+	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
+	 * store against prior state change of @next, also see
+	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).
 	 */
-	next->on_cpu = 1;
+	WRITE_ONCE(next->on_cpu, 1);
 #endif
 }
 
@@ -3143,8 +3287,9 @@ static inline void finish_task(struct task_struct *prev)
 {
 #ifdef CONFIG_SMP
 	/*
-	 * After ->on_cpu is cleared, the task can be moved to a different CPU.
-	 * We must ensure this doesn't happen until the switch is completely
+	 * This must be the very last reference to @prev from this CPU. After
+	 * p->on_cpu is cleared, the task can be moved to a different CPU. We
+	 * must ensure this doesn't happen until the switch is completely
 	 * finished.
 	 *
 	 * In particular, the load of prev->state in finish_task_switch() must
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 5aa6661ecaf1..73c0c5d0034b 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -1197,6 +1197,17 @@ struct rq_flags {
 #endif
 };
 
+/*
+ * Lockdep annotation that avoid accidental unlock; any
+ * raw_spin_unlock(&rq->lock) without preceding rq_unpin_lock() with the
+ * correct cookie will result in a WARN.
+ *
+ * This avoids code that has access to 'struct rq *rq' (basically everything in
+ * the scheduler) from accidentally unlocking the rq if they do not also have a
+ * copy of the (on-stack) 'struct rq_flags rf'.
+ *
+ * Also see Documentation/locking/lockdep-design.rst.
+ */
 static inline void rq_pin_lock(struct rq *rq, struct rq_flags *rf)
 {
 	rf->cookie = lockdep_pin_lock(&rq->lock);

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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-02 12:52 [RFC][PATCH] sched: Better document ttwu() Peter Zijlstra
@ 2020-07-02 13:13 ` Phil Auld
  2020-07-02 15:23   ` Peter Zijlstra
  2020-07-02 18:39 ` Valentin Schneider
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 9+ messages in thread
From: Phil Auld @ 2020-07-02 13:13 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ingo Molnar, linux-kernel, vincent.guittot, mgorman,
	Oleg Nesterov, david


Hi Peter,

On Thu, Jul 02, 2020 at 02:52:11PM +0200 Peter Zijlstra wrote:
> 
> Dave hit the problem fixed by commit:
> 
>   b6e13e85829f ("sched/core: Fix ttwu() race")
> 
> and failed to understand much of the code involved. Per his request a
> few comments to (hopefully) clarify things.
> 
> Requested-by: Dave Chinner <david@fromorbit.com>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  include/linux/sched.h |  12 ++--
>  kernel/sched/core.c   | 195 +++++++++++++++++++++++++++++++++++++++++++-------
>  kernel/sched/sched.h  |  11 +++
>  3 files changed, 187 insertions(+), 31 deletions(-)
> 
> diff --git a/include/linux/sched.h b/include/linux/sched.h
> index 9bd073a10224..ad36f70bef24 100644
> --- a/include/linux/sched.h
> +++ b/include/linux/sched.h
> @@ -158,24 +158,24 @@ struct task_group;
>   *
>   *   for (;;) {
>   *	set_current_state(TASK_UNINTERRUPTIBLE);
> - *	if (!need_sleep)
> - *		break;
> + *	if (CONDITION)
> + *	   break;
>   *
>   *	schedule();
>   *   }
>   *   __set_current_state(TASK_RUNNING);
>   *
>   * If the caller does not need such serialisation (because, for instance, the
> - * condition test and condition change and wakeup are under the same lock) then
> + * CONDITION test and condition change and wakeup are under the same lock) then
>   * use __set_current_state().
>   *
>   * The above is typically ordered against the wakeup, which does:
>   *
> - *   need_sleep = false;
> + *   CONDITION = 1;
>   *   wake_up_state(p, TASK_UNINTERRUPTIBLE);
>   *
> - * where wake_up_state() executes a full memory barrier before accessing the
> - * task state.
> + * where wake_up_state()/try_to_wake_up() executes a full memory barrier before
> + * accessing p->state.
>   *
>   * Wakeup will do: if (@state & p->state) p->state = TASK_RUNNING, that is,
>   * once it observes the TASK_UNINTERRUPTIBLE store the waking CPU can issue a
> diff --git a/kernel/sched/core.c b/kernel/sched/core.c
> index 1d3d2d67f398..0cd6c336029f 100644
> --- a/kernel/sched/core.c
> +++ b/kernel/sched/core.c
> @@ -77,6 +77,97 @@ __read_mostly int scheduler_running;
>   */
>  int sysctl_sched_rt_runtime = 950000;
>  
> +
> +/*
> + * Serialization rules:
> + *
> + * Lock order:
> + *
> + *   p->pi_lock
> + *     rq->lock
> + *       hrtimer_cpu_base->lock (hrtimer_start() for bandwidth controls)
> + *
> + *  rq1->lock
> + *    rq2->lock  where: rq1 < rq2
> + *
> + * Regular state:
> + *
> + * Normal scheduling state is serialized by rq->lock. __schedule() takes the
> + * local CPU's rq->lock, it optionally removes the task from the runqueue and
> + * always looks at the local rq data structures to find the most elegible task
> + * to run next.
> + *
> + * Task enqueue is also under rq->lock, possibly taken from another CPU.
> + * Wakeups from another LLC domain might use an IPI to transfer the enqueue to
> + * the local CPU to avoid bouncing the runqueue state around [ see
> + * ttwu_queue_wakelist() ]
> + *
> + * Task wakeup, specifically wakeups that involve migration, are horribly
> + * complicated to avoid having to take two rq->locks.
> + *
> + * Special state:
> + *
> + * System-calls and anything external will use task_rq_lock() which acquires
> + * both p->lock and rq->lock. As a consequence the state they change is stable
> + * while holding either lock:
> + *
> + *  - sched_setaffinity():	p->cpus_ptr
> + *  - set_user_nice():		p->se.load, p->static_prio
> + *  - __sched_setscheduler():	p->sched_class, p->policy, p->*prio, p->se.load,
> + *				p->dl.dl_{runtime, deadline, period, flags, bw, density}
> + *  - sched_setnuma():		p->numa_preferred_nid
> + *  - sched_move_task()/
> + *    cpu_cgroup_fork():	p->sched_task_group
> + *
> + * p->state <- TASK_*:
> + *
> + *   is changed locklessly using set_current_state(), __set_current_state() or
> + *   set_special_state(), see their respective comments, or by
> + *   try_to_wake_up(). This latter uses p->pi_lock to serialize against
> + *   concurrent self.
> + *
> + * p->on_rq <- { 0, 1 = TASK_ON_RQ_QUEUED, 2 = TASK_ON_RQ_MIGRATING }:
> + *
> + *   is set by activate_task() and cleared by deactivate_task(), under
> + *   rq->lock. Non-zero indicates the task is runnable, the special
> + *   ON_RQ_MIGRATING state is used for migration without holding both
> + *   rq->locks. It indicates task_cpu() is not stable, see task_rq_lock().
> + *
> + * p->on_cpu <- { 0, 1 }:
> + *
> + *   is set by prepare_task() and cleared by finish_task() such that it will be
> + *   set before p is scheduled-in and cleared after p is scheduled-out, both
> + *   under rq->lock. Non-zero indicates the task is running on it's CPU.

s/it's/its/

> + *
> + *   [ The astute reader will observe that it is possible for two tasks on one
> + *     CPU to have ->on_cpu = 1 at the same time. ]
> + *
> + * task_cpu(p): is changed by set_task_cpu(), the rules are:
> + *
> + *  - Don't call set_task_cpu() on a blocked task:
> + *
> + *    We don't care what CPU we're not running on, this simplifies hotplug,
> + *    the CPU assignment of blocked tasks isn't required to be valid.
> + *
> + *  - for try_to_wake_up(), called under p->pi_lock:
> + *
> + *    This allows try_to_wake_up() to only take one rq->lock, see its comment.
> + *
> + *  - for migration called under rq->lock:
> + *    [ see task_on_rq_migrating() in task_rq_lock() ]
> + *
> + *    o move_queued_task()
> + *    o __migrate_swap_task()
> + *    o detach_task()
> + *
> + *  - for migration called under double_rq_lock():
> + *
> + *    o push_rt_task() / pull_rt_task()
> + *    o push_dl_task() / pull_dl_task()
> + *    o dl_task_offline_migration()
> + *
> + */
> +
>  /*
>   * __task_rq_lock - lock the rq @p resides on.
>   */
> @@ -1466,8 +1557,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
>  {
>  	lockdep_assert_held(&rq->lock);
>  
> -	WRITE_ONCE(p->on_rq, TASK_ON_RQ_MIGRATING);
> -	dequeue_task(rq, p, DEQUEUE_NOCLOCK);
> +	deactivate_task(rq, p, DEQUEUE_NOCLOCK);
>  	set_task_cpu(p, new_cpu);
>  	rq_unlock(rq, rf);
>  
> @@ -1475,8 +1565,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
>  
>  	rq_lock(rq, rf);
>  	BUG_ON(task_cpu(p) != new_cpu);
> -	enqueue_task(rq, p, 0);
> -	p->on_rq = TASK_ON_RQ_QUEUED;
> +	activate_task(rq, p, 0);
>  	check_preempt_curr(rq, p, 0);
>  
>  	return rq;
> @@ -2241,12 +2330,31 @@ ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
>  }
>  
>  /*
> - * Called in case the task @p isn't fully descheduled from its runqueue,
> - * in this case we must do a remote wakeup. Its a 'light' wakeup though,
> - * since all we need to do is flip p->state to TASK_RUNNING, since
> - * the task is still ->on_rq.
> + * Consider @p being inside a wait loop:
> + *
> + *   for (;;) {
> + *   	set_current_state(TASK_UNINTERRUPTIBLE);
> + *
> + *   	if (CONDITION)
> + *         break;
> + *
> + *   	schedule();
> + *   }
> + *   __set_current_state(TASK_RUNNING);
> + *
> + * between set_current_state() and schedule(). In this case @p is still
> + * runnable, so all that needs doing is change p->state back to TASK_RUNNING in
> + * an atomic manner.
> + *
> + * By taking task_rq(p)->lock we serialize against schedule(), if @p->on_rq
> + * then schedule() must still happen and p->state can be changed to
> + * TASK_RUNNING. Otherwise we lost the race, schedule() has happened, and we
> + * need to do a full wakeup with enqueue.
> + *
> + * Returns: %true when the wakeup is done,
> + *          %false otherwise.
>   */
> -static int ttwu_remote(struct task_struct *p, int wake_flags)
> +static int ttwu_runnable(struct task_struct *p, int wake_flags)
>  {
>  	struct rq_flags rf;
>  	struct rq *rq;
> @@ -2387,6 +2495,14 @@ static bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
>  
>  	return false;
>  }
> +
> +#else /* !CONFIG_SMP */
> +
> +static inline bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
> +{
> +	return false;
> +}
> +
>  #endif /* CONFIG_SMP */
>  
>  static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
> @@ -2394,10 +2510,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>  	struct rq *rq = cpu_rq(cpu);
>  	struct rq_flags rf;
>  
> -#if defined(CONFIG_SMP)
>  	if (ttwu_queue_wakelist(p, cpu, wake_flags))
>  		return;
> -#endif
>  
>  	rq_lock(rq, &rf);
>  	update_rq_clock(rq);
> @@ -2453,8 +2567,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>   * migration. However the means are completely different as there is no lock
>   * chain to provide order. Instead we do:
>   *
> - *   1) smp_store_release(X->on_cpu, 0)
> - *   2) smp_cond_load_acquire(!X->on_cpu)
> + *   1) smp_store_release(X->on_cpu, 0)   -- finish_task()
> + *   2) smp_cond_load_acquire(!X->on_cpu) -- try_to_wake_up()
>   *
>   * Example:
>   *
> @@ -2494,15 +2608,41 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>   * @state: the mask of task states that can be woken
>   * @wake_flags: wake modifier flags (WF_*)
>   *
> - * If (@state & @p->state) @p->state = TASK_RUNNING.
> + * Conceptually does:
> + *
> + *   If (@state & @p->state) @p->state = TASK_RUNNING.
>   *
>   * If the task was not queued/runnable, also place it back on a runqueue.
>   *
> - * Atomic against schedule() which would dequeue a task, also see
> - * set_current_state().
> + * This function:
> + *  - is atomic against schedule() which would dequeue the task;
> + *  - issues a full memory barrier before accessing @p->state.
> + * See the comment with set_current_state().

I think these two above should not be " - " indented to match the other
partial sentences below (or all the ones below should be bullets, but I
think that is messier). But this is just a style quibble :)

> + *
> + * Uses p->pi_lock to serialize against concurrent wake-ups.
> + *
> + * Relies on p->pi_lock stabilizing:
> + *  - p->sched_class
> + *  - p->cpus_ptr
> + *  - p->sched_task_group
> + * in order to do migration, see its use of select_task_rq()/set_task_cpu().
>   *
> - * This function executes a full memory barrier before accessing the task
> - * state; see set_current_state().
> + * Tries really hard to only take one task_rq(p)->lock for performance.
> + * Takes rq->lock in:
> + *  - ttwu_runnable()    -- old rq, unavoidable, see comment there;
> + *  - ttwu_queue()       -- new rq, for enqueue of the task;
> + *  - psi_ttwu_dequeue() -- much sadness :-( accounting will kill us.
> + *
> + * As a concequence we race really badly with just about everything. See the
> + * many memory barriers and their comments for details. The basic order of
> + * reading things is:
> + *
> + *   LOAD p->state
> + *   RMB
> + *   LOAD p->on_rq
> + *   RMB
> + *   LOAD-ACQUIRE p->on_cpu
> + *   LOAD task_cpu()
>   *
>   * Return: %true if @p->state changes (an actual wakeup was done),
>   *	   %false otherwise.
> @@ -2518,7 +2658,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>  		/*
>  		 * We're waking current, this means 'p->on_rq' and 'task_cpu(p)
>  		 * == smp_processor_id()'. Together this means we can special
> -		 * case the whole 'p->on_rq && ttwu_remote()' case below
> +		 * case the whole 'p->on_rq && ttwu_runnable()' case below
>  		 * without taking any locks.
>  		 *
>  		 * In particular:
> @@ -2539,8 +2679,8 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>  	/*
>  	 * If we are going to wake up a thread waiting for CONDITION we
>  	 * need to ensure that CONDITION=1 done by the caller can not be
> -	 * reordered with p->state check below. This pairs with mb() in
> -	 * set_current_state() the waiting thread does.
> +	 * reordered with p->state check below. This pairs with smp_store_mb()
> +	 * in set_current_state() that the waiting thread does.
>  	 */
>  	raw_spin_lock_irqsave(&p->pi_lock, flags);
>  	smp_mb__after_spinlock();
> @@ -2575,7 +2715,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>  	 * A similar smb_rmb() lives in try_invoke_on_locked_down_task().
>  	 */
>  	smp_rmb();
> -	if (p->on_rq && ttwu_remote(p, wake_flags))
> +	if (p->on_rq && ttwu_runnable(p, wake_flags))
>  		goto unlock;
>  
>  	if (p->in_iowait) {
> @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
>  	/*
>  	 * Claim the task as running, we do this before switching to it
>  	 * such that any running task will have this set.
> +	 *
> +	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> +	 * store against prior state change of @next, also see
> +	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).
>  	 */
> -	next->on_cpu = 1;
> +	WRITE_ONCE(next->on_cpu, 1);

This is more than a documentation change.


Beyond that this looks good. I've got to read it again in context to digest it fully.
Thank you for doing this. Very helpful. 


Cheers,
Phil

>  #endif
>  }
>  
> @@ -3143,8 +3287,9 @@ static inline void finish_task(struct task_struct *prev)
>  {
>  #ifdef CONFIG_SMP
>  	/*
> -	 * After ->on_cpu is cleared, the task can be moved to a different CPU.
> -	 * We must ensure this doesn't happen until the switch is completely
> +	 * This must be the very last reference to @prev from this CPU. After
> +	 * p->on_cpu is cleared, the task can be moved to a different CPU. We
> +	 * must ensure this doesn't happen until the switch is completely
>  	 * finished.
>  	 *
>  	 * In particular, the load of prev->state in finish_task_switch() must
> diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
> index 5aa6661ecaf1..73c0c5d0034b 100644
> --- a/kernel/sched/sched.h
> +++ b/kernel/sched/sched.h
> @@ -1197,6 +1197,17 @@ struct rq_flags {
>  #endif
>  };
>  
> +/*
> + * Lockdep annotation that avoid accidental unlock; any
> + * raw_spin_unlock(&rq->lock) without preceding rq_unpin_lock() with the
> + * correct cookie will result in a WARN.
> + *
> + * This avoids code that has access to 'struct rq *rq' (basically everything in
> + * the scheduler) from accidentally unlocking the rq if they do not also have a
> + * copy of the (on-stack) 'struct rq_flags rf'.
> + *
> + * Also see Documentation/locking/lockdep-design.rst.
> + */
>  static inline void rq_pin_lock(struct rq *rq, struct rq_flags *rf)
>  {
>  	rf->cookie = lockdep_pin_lock(&rq->lock);
> 

-- 


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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-02 13:13 ` Phil Auld
@ 2020-07-02 15:23   ` Peter Zijlstra
  0 siblings, 0 replies; 9+ messages in thread
From: Peter Zijlstra @ 2020-07-02 15:23 UTC (permalink / raw)
  To: Phil Auld
  Cc: Ingo Molnar, linux-kernel, vincent.guittot, mgorman,
	Oleg Nesterov, david

On Thu, Jul 02, 2020 at 09:13:19AM -0400, Phil Auld wrote:
> On Thu, Jul 02, 2020 at 02:52:11PM +0200 Peter Zijlstra wrote:

> > + * p->on_cpu <- { 0, 1 }:
> > + *
> > + *   is set by prepare_task() and cleared by finish_task() such that it will be
> > + *   set before p is scheduled-in and cleared after p is scheduled-out, both
> > + *   under rq->lock. Non-zero indicates the task is running on it's CPU.
> 
> s/it's/its/

Fixed.

> > @@ -2494,15 +2608,41 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
> >   * @state: the mask of task states that can be woken
> >   * @wake_flags: wake modifier flags (WF_*)
> >   *
> > - * If (@state & @p->state) @p->state = TASK_RUNNING.
> > + * Conceptually does:
> > + *
> > + *   If (@state & @p->state) @p->state = TASK_RUNNING.
> >   *
> >   * If the task was not queued/runnable, also place it back on a runqueue.
> >   *
> > - * Atomic against schedule() which would dequeue a task, also see
> > - * set_current_state().
> > + * This function:
> > + *  - is atomic against schedule() which would dequeue the task;
> > + *  - issues a full memory barrier before accessing @p->state.
> > + * See the comment with set_current_state().
> 
> I think these two above should not be " - " indented to match the other
> partial sentences below (or all the ones below should be bullets, but I
> think that is messier). But this is just a style quibble :)

Fair enough; I'll go rework that.

> > @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
> >  	/*
> >  	 * Claim the task as running, we do this before switching to it
> >  	 * such that any running task will have this set.
> > +	 *
> > +	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> > +	 * store against prior state change of @next, also see
> > +	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).
> >  	 */
> > -	next->on_cpu = 1;
> > +	WRITE_ONCE(next->on_cpu, 1);
> 
> This is more than a documentation change.

It had better be an effective no-change though; as documented we only
ever write 0 or 1, so even if the compiler is evil and tears our write,
it is impossible to get this wrong.

The reason I made it WRITE_ONCE() is because the other write is
smp_store_release() and the two loads are smp_load_acquire(), so a plain
store is 'weird'.

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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-02 12:52 [RFC][PATCH] sched: Better document ttwu() Peter Zijlstra
  2020-07-02 13:13 ` Phil Auld
@ 2020-07-02 18:39 ` Valentin Schneider
  2020-07-03  8:30   ` Peter Zijlstra
  2020-07-03 10:12 ` Dietmar Eggemann
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 9+ messages in thread
From: Valentin Schneider @ 2020-07-02 18:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ingo Molnar, linux-kernel, vincent.guittot, mgorman,
	Oleg Nesterov, david


Hi,

On 02/07/20 13:52, Peter Zijlstra wrote:
> Dave hit the problem fixed by commit:
>
>   b6e13e85829f ("sched/core: Fix ttwu() race")
>
> and failed to understand much of the code involved. Per his request a
> few comments to (hopefully) clarify things.
>

All of the below is already tremendously helpful! I remember it took me
quite some time to figure out e.g. how we could observe p->on_cpu &&
!p->on_rq and why we cared about it in ttwu(). On the bright side, I'm
happy that my notes aren't completely off - there may be hope for me yet.

Have some small comments below.

> Requested-by: Dave Chinner <david@fromorbit.com>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  include/linux/sched.h |  12 ++--
>  kernel/sched/core.c   | 195 +++++++++++++++++++++++++++++++++++++++++++-------
>  kernel/sched/sched.h  |  11 +++
>  3 files changed, 187 insertions(+), 31 deletions(-)
>
> diff --git a/include/linux/sched.h b/include/linux/sched.h
> index 9bd073a10224..ad36f70bef24 100644
> --- a/include/linux/sched.h
> +++ b/include/linux/sched.h
[...]
> + * Special state:
> + *
> + * System-calls and anything external will use task_rq_lock() which acquires
> + * both p->lock and rq->lock. As a consequence the state they change is stable
> + * while holding either lock:
> + *
> + *  - sched_setaffinity():	p->cpus_ptr
> + *  - set_user_nice():		p->se.load, p->static_prio
> + *  - __sched_setscheduler():	p->sched_class, p->policy, p->*prio, p->se.load,
> + *				p->dl.dl_{runtime, deadline, period, flags, bw, density}

Only extra thing that comes to mind is p->uclamp*; dunno how exhaustive you
want this list to be.

> + *  - sched_setnuma():		p->numa_preferred_nid
> + *  - sched_move_task()/
> + *    cpu_cgroup_fork():	p->sched_task_group
> + *
> + * p->state <- TASK_*:
> + *
> + *   is changed locklessly using set_current_state(), __set_current_state() or
> + *   set_special_state(), see their respective comments, or by
> + *   try_to_wake_up(). This latter uses p->pi_lock to serialize against
> + *   concurrent self.
> + *
> + * p->on_rq <- { 0, 1 = TASK_ON_RQ_QUEUED, 2 = TASK_ON_RQ_MIGRATING }:
> + *
> + *   is set by activate_task() and cleared by deactivate_task(), under
> + *   rq->lock. Non-zero indicates the task is runnable, the special
> + *   ON_RQ_MIGRATING state is used for migration without holding both
> + *   rq->locks. It indicates task_cpu() is not stable, see task_rq_lock().
> + *
> + * p->on_cpu <- { 0, 1 }:
> + *
> + *   is set by prepare_task() and cleared by finish_task() such that it will be
> + *   set before p is scheduled-in and cleared after p is scheduled-out, both
> + *   under rq->lock. Non-zero indicates the task is running on it's CPU.
> + *
> + *   [ The astute reader will observe that it is possible for two tasks on one
> + *     CPU to have ->on_cpu = 1 at the same time. ]
> + *
> + * task_cpu(p): is changed by set_task_cpu(), the rules are:
> + *
> + *  - Don't call set_task_cpu() on a blocked task:
> + *
> + *    We don't care what CPU we're not running on, this simplifies hotplug,
> + *    the CPU assignment of blocked tasks isn't required to be valid.
> + *

That's more of a good practice rather than a hard rule, right? We do that
with proxy execution (the whole migrate to owner's rq thing), at least in
its current shape.

> + *  - for try_to_wake_up(), called under p->pi_lock:
> + *
> + *    This allows try_to_wake_up() to only take one rq->lock, see its comment.
> + *
> + *  - for migration called under rq->lock:
> + *    [ see task_on_rq_migrating() in task_rq_lock() ]
> + *
> + *    o move_queued_task()
> + *    o __migrate_swap_task()

Isn't that one under double_rq_lock()?

> + *    o detach_task()
> + *
> + *  - for migration called under double_rq_lock():
> + *
> + *    o push_rt_task() / pull_rt_task()
> + *    o push_dl_task() / pull_dl_task()
> + *    o dl_task_offline_migration()
> + *
> + */
> +
>  /*
>   * __task_rq_lock - lock the rq @p resides on.
>   */
[...]
>  }
>
>  /*
> - * Called in case the task @p isn't fully descheduled from its runqueue,
> - * in this case we must do a remote wakeup. Its a 'light' wakeup though,
> - * since all we need to do is flip p->state to TASK_RUNNING, since
> - * the task is still ->on_rq.
> + * Consider @p being inside a wait loop:
> + *
> + *   for (;;) {
> + *           set_current_state(TASK_UNINTERRUPTIBLE);
> + *
> + *           if (CONDITION)
> + *         break;

For some reason the alignment is off in my mail view, but looks okay once
applied.

> + *
> + *           schedule();
> + *   }
> + *   __set_current_state(TASK_RUNNING);
> + *
> + * between set_current_state() and schedule(). In this case @p is still
> + * runnable, so all that needs doing is change p->state back to TASK_RUNNING in
> + * an atomic manner.
> + *

Sorry if I'm being dense; don't you mean "running" here? If it stops being
current inbetween set_current_state() and schedule(), __schedule() will
deactivate() it, so AFAICT it can only be either running or deactivated.

> + * By taking task_rq(p)->lock we serialize against schedule(), if @p->on_rq
> + * then schedule() must still happen and p->state can be changed to
> + * TASK_RUNNING. Otherwise we lost the race, schedule() has happened, and we
> + * need to do a full wakeup with enqueue.
> + *
> + * Returns: %true when the wakeup is done,
> + *          %false otherwise.
>   */
> -static int ttwu_remote(struct task_struct *p, int wake_flags)
> +static int ttwu_runnable(struct task_struct *p, int wake_flags)
>  {
>       struct rq_flags rf;
>       struct rq *rq;
[...]
> @@ -2494,15 +2608,41 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>   * @state: the mask of task states that can be woken
>   * @wake_flags: wake modifier flags (WF_*)
>   *
> - * If (@state & @p->state) @p->state = TASK_RUNNING.
> + * Conceptually does:
> + *
> + *   If (@state & @p->state) @p->state = TASK_RUNNING.
>   *
>   * If the task was not queued/runnable, also place it back on a runqueue.
>   *
> - * Atomic against schedule() which would dequeue a task, also see
> - * set_current_state().
> + * This function:
> + *  - is atomic against schedule() which would dequeue the task;
> + *  - issues a full memory barrier before accessing @p->state.
> + * See the comment with set_current_state().
> + *
> + * Uses p->pi_lock to serialize against concurrent wake-ups.
> + *
> + * Relies on p->pi_lock stabilizing:
> + *  - p->sched_class
> + *  - p->cpus_ptr
> + *  - p->sched_task_group
> + * in order to do migration, see its use of select_task_rq()/set_task_cpu().
>   *
> - * This function executes a full memory barrier before accessing the task
> - * state; see set_current_state().
> + * Tries really hard to only take one task_rq(p)->lock for performance.
> + * Takes rq->lock in:
> + *  - ttwu_runnable()    -- old rq, unavoidable, see comment there;
> + *  - ttwu_queue()       -- new rq, for enqueue of the task;
> + *  - psi_ttwu_dequeue() -- much sadness :-( accounting will kill us.
> + *
> + * As a concequence we race really badly with just about everything. See the

s/concequence/consequence/

> + * many memory barriers and their comments for details. The basic order of
> + * reading things is:
> + *
> + *   LOAD p->state
> + *   RMB
> + *   LOAD p->on_rq
> + *   RMB
> + *   LOAD-ACQUIRE p->on_cpu
> + *   LOAD task_cpu()
>   *
>   * Return: %true if @p->state changes (an actual wakeup was done),
>   *	   %false otherwise.
> @@ -2518,7 +2658,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>               /*
>                * We're waking current, this means 'p->on_rq' and 'task_cpu(p)
>                * == smp_processor_id()'. Together this means we can special
> -		 * case the whole 'p->on_rq && ttwu_remote()' case below
> +		 * case the whole 'p->on_rq && ttwu_runnable()' case below
>                * without taking any locks.
>                *
>                * In particular:
> @@ -2539,8 +2679,8 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>       /*
>        * If we are going to wake up a thread waiting for CONDITION we
>        * need to ensure that CONDITION=1 done by the caller can not be
> -	 * reordered with p->state check below. This pairs with mb() in
> -	 * set_current_state() the waiting thread does.
> +	 * reordered with p->state check below. This pairs with smp_store_mb()
> +	 * in set_current_state() that the waiting thread does.
>        */
>       raw_spin_lock_irqsave(&p->pi_lock, flags);
>       smp_mb__after_spinlock();
> @@ -2575,7 +2715,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>        * A similar smb_rmb() lives in try_invoke_on_locked_down_task().
>        */
>       smp_rmb();
> -	if (p->on_rq && ttwu_remote(p, wake_flags))
> +	if (p->on_rq && ttwu_runnable(p, wake_flags))
>               goto unlock;
>
>       if (p->in_iowait) {
> @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
>       /*
>        * Claim the task as running, we do this before switching to it
>        * such that any running task will have this set.
> +	 *
> +	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> +	 * store against prior state change of @next, also see
> +	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).

smp_*cond*_load_acquire(&p->on_cpu, <blah>)

>        */
> -	next->on_cpu = 1;
> +	WRITE_ONCE(next->on_cpu, 1);
>  #endif
>  }
>
> @@ -3143,8 +3287,9 @@ static inline void finish_task(struct task_struct *prev)
>  {
>  #ifdef CONFIG_SMP
>       /*
> -	 * After ->on_cpu is cleared, the task can be moved to a different CPU.
> -	 * We must ensure this doesn't happen until the switch is completely
> +	 * This must be the very last reference to @prev from this CPU. After
> +	 * p->on_cpu is cleared, the task can be moved to a different CPU. We
> +	 * must ensure this doesn't happen until the switch is completely
>        * finished.
>        *
>        * In particular, the load of prev->state in finish_task_switch() must
> diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
> index 5aa6661ecaf1..73c0c5d0034b 100644
> --- a/kernel/sched/sched.h
> +++ b/kernel/sched/sched.h
> @@ -1197,6 +1197,17 @@ struct rq_flags {
>  #endif
>  };
>
> +/*
> + * Lockdep annotation that avoid accidental unlock; any
> + * raw_spin_unlock(&rq->lock) without preceding rq_unpin_lock() with the
> + * correct cookie will result in a WARN.
> + *

ISTR that being described (by yourself?) as a "sticky/continuous
lockdep_assert_held()", which I think gets the point across.

> + * This avoids code that has access to 'struct rq *rq' (basically everything in
> + * the scheduler) from accidentally unlocking the rq if they do not also have a
> + * copy of the (on-stack) 'struct rq_flags rf'.
> + *
> + * Also see Documentation/locking/lockdep-design.rst.
> + */
>  static inline void rq_pin_lock(struct rq *rq, struct rq_flags *rf)
>  {
>       rf->cookie = lockdep_pin_lock(&rq->lock);

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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-02 18:39 ` Valentin Schneider
@ 2020-07-03  8:30   ` Peter Zijlstra
  2020-07-03 11:36     ` Peter Zijlstra
  0 siblings, 1 reply; 9+ messages in thread
From: Peter Zijlstra @ 2020-07-03  8:30 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: Ingo Molnar, linux-kernel, vincent.guittot, mgorman,
	Oleg Nesterov, david

On Thu, Jul 02, 2020 at 07:39:16PM +0100, Valentin Schneider wrote:

> > + * Special state:
> > + *
> > + * System-calls and anything external will use task_rq_lock() which acquires
> > + * both p->lock and rq->lock. As a consequence the state they change is stable
> > + * while holding either lock:
> > + *
> > + *  - sched_setaffinity():	p->cpus_ptr
> > + *  - set_user_nice():		p->se.load, p->static_prio
> > + *  - __sched_setscheduler():	p->sched_class, p->policy, p->*prio, p->se.load,
> > + *				p->dl.dl_{runtime, deadline, period, flags, bw, density}
> 
> Only extra thing that comes to mind is p->uclamp*; dunno how exhaustive you
> want this list to be.

Indeed, I seem to have missed that one.

> > + *  - sched_setnuma():		p->numa_preferred_nid
> > + *  - sched_move_task()/
> > + *    cpu_cgroup_fork():	p->sched_task_group
> > + *

> > + * task_cpu(p): is changed by set_task_cpu(), the rules are:
> > + *
> > + *  - Don't call set_task_cpu() on a blocked task:
> > + *
> > + *    We don't care what CPU we're not running on, this simplifies hotplug,
> > + *    the CPU assignment of blocked tasks isn't required to be valid.
> > + *
> 
> That's more of a good practice rather than a hard rule, right? We do that
> with proxy execution (the whole migrate to owner's rq thing), at least in
> its current shape.

Yeah, but all of that isn't upstream yet. That said; the distinguishing
feature there is that we create a class of blocked tasks that will still
be 'runnable'. And as such we'll care about their placement.

> > + *  - for try_to_wake_up(), called under p->pi_lock:
> > + *
> > + *    This allows try_to_wake_up() to only take one rq->lock, see its comment.
> > + *
> > + *  - for migration called under rq->lock:
> > + *    [ see task_on_rq_migrating() in task_rq_lock() ]
> > + *
> > + *    o move_queued_task()
> > + *    o __migrate_swap_task()
> 
> Isn't that one under double_rq_lock()?

Indeed, /me moves.

> > + *    o detach_task()
> > + *
> > + *  - for migration called under double_rq_lock():
> > + *
> > + *    o push_rt_task() / pull_rt_task()
> > + *    o push_dl_task() / pull_dl_task()
> > + *    o dl_task_offline_migration()
> > + *
> > + */

> >  /*
> > - * Called in case the task @p isn't fully descheduled from its runqueue,
> > - * in this case we must do a remote wakeup. Its a 'light' wakeup though,
> > - * since all we need to do is flip p->state to TASK_RUNNING, since
> > - * the task is still ->on_rq.
> > + * Consider @p being inside a wait loop:
> > + *
> > + *   for (;;) {
> > + *           set_current_state(TASK_UNINTERRUPTIBLE);
> > + *
> > + *           if (CONDITION)
> > + *         break;
> 
> For some reason the alignment is off in my mail view, but looks okay once
> applied.

I'll go eradicate tabstops :-)

> > + *
> > + *           schedule();
> > + *   }
> > + *   __set_current_state(TASK_RUNNING);
> > + *
> > + * between set_current_state() and schedule(). In this case @p is still
> > + * runnable, so all that needs doing is change p->state back to TASK_RUNNING in
> > + * an atomic manner.
> > + *
> 
> Sorry if I'm being dense; don't you mean "running" here? If it stops being
> current inbetween set_current_state() and schedule(), __schedule() will
> deactivate() it, so AFAICT it can only be either running or deactivated.

Runnable, the task could be preempted. At this point we don't care if it
is actually running or not.

> > + * By taking task_rq(p)->lock we serialize against schedule(), if @p->on_rq
> > + * then schedule() must still happen and p->state can be changed to
> > + * TASK_RUNNING. Otherwise we lost the race, schedule() has happened, and we
> > + * need to do a full wakeup with enqueue.
> > + *
> > + * Returns: %true when the wakeup is done,
> > + *          %false otherwise.
> >   */
> > -static int ttwu_remote(struct task_struct *p, int wake_flags)
> > +static int ttwu_runnable(struct task_struct *p, int wake_flags)
> >  {
> >       struct rq_flags rf;
> >       struct rq *rq;


> > + * Tries really hard to only take one task_rq(p)->lock for performance.
> > + * Takes rq->lock in:
> > + *  - ttwu_runnable()    -- old rq, unavoidable, see comment there;
> > + *  - ttwu_queue()       -- new rq, for enqueue of the task;
> > + *  - psi_ttwu_dequeue() -- much sadness :-( accounting will kill us.
> > + *
> > + * As a concequence we race really badly with just about everything. See the
> 
> s/concequence/consequence/

ta!

> > @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
> >       /*
> >        * Claim the task as running, we do this before switching to it
> >        * such that any running task will have this set.
> > +	 *
> > +	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> > +	 * store against prior state change of @next, also see
> > +	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).
> 
> smp_*cond*_load_acquire(&p->on_cpu, <blah>)

Both, but yeah.. arguably the cond one is the more important one.

> 
> >        */
> > -	next->on_cpu = 1;
> > +	WRITE_ONCE(next->on_cpu, 1);
> >  #endif
> >  }

> > +/*
> > + * Lockdep annotation that avoid accidental unlock; any
> > + * raw_spin_unlock(&rq->lock) without preceding rq_unpin_lock() with the
> > + * correct cookie will result in a WARN.
> > + *
> 
> ISTR that being described (by yourself?) as a "sticky/continuous
> lockdep_assert_held()", which I think gets the point across.

Ah indeed! Clever of my past self :-) I'll go reword it.

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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-02 12:52 [RFC][PATCH] sched: Better document ttwu() Peter Zijlstra
  2020-07-02 13:13 ` Phil Auld
  2020-07-02 18:39 ` Valentin Schneider
@ 2020-07-03 10:12 ` Dietmar Eggemann
  2020-07-03 12:39 ` Vincent Guittot
  2020-07-22  9:12 ` [tip: sched/core] " tip-bot2 for Peter Zijlstra
  4 siblings, 0 replies; 9+ messages in thread
From: Dietmar Eggemann @ 2020-07-03 10:12 UTC (permalink / raw)
  To: Peter Zijlstra, Ingo Molnar
  Cc: linux-kernel, vincent.guittot, mgorman, Oleg Nesterov, david

On 02/07/2020 14:52, Peter Zijlstra wrote:
> 
> Dave hit the problem fixed by commit:
> 
>   b6e13e85829f ("sched/core: Fix ttwu() race")
> 
> and failed to understand much of the code involved. Per his request a
> few comments to (hopefully) clarify things.
> 
> Requested-by: Dave Chinner <david@fromorbit.com>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>

LGTM. Just a couple of nitpicks below.

[...]

> + * Special state:
> + *
> + * System-calls and anything external will use task_rq_lock() which acquires
> + * both p->lock and rq->lock. As a consequence the state they change is stable

s/p->lock/p->pi_lock ?

> + * while holding either lock:
> + *
> + *  - sched_setaffinity():	p->cpus_ptr
> + *  - set_user_nice():		p->se.load, p->static_prio

Doesn't set_user_nice() also change p->prio and p->normal_prio, so
p->*prio ?

> + *  - __sched_setscheduler():	p->sched_class, p->policy, p->*prio, p->se.load,
> + *				p->dl.dl_{runtime, deadline, period, flags, bw, density}

p->rt_priority ?

> + *  - sched_setnuma():		p->numa_preferred_nid
> + *  - sched_move_task()/
> + *    cpu_cgroup_fork():	p->sched_task_group

maybe also: set_cpus_allowed_ptr() -> __set_cpus_allowed_ptr() (like
sched_setaffinity()) ?

[...]

> @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
>  	/*
>  	 * Claim the task as running, we do this before switching to it
>  	 * such that any running task will have this set.
> +	 *
> +	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> +	 * store against prior state change of @next, also see
> +	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).

s/smp_load_acquire/smp_cond_load_acquire ?

[...]

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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-03  8:30   ` Peter Zijlstra
@ 2020-07-03 11:36     ` Peter Zijlstra
  0 siblings, 0 replies; 9+ messages in thread
From: Peter Zijlstra @ 2020-07-03 11:36 UTC (permalink / raw)
  To: Valentin Schneider
  Cc: Ingo Molnar, linux-kernel, vincent.guittot, mgorman,
	Oleg Nesterov, david

On Fri, Jul 03, 2020 at 10:30:12AM +0200, Peter Zijlstra wrote:
> On Thu, Jul 02, 2020 at 07:39:16PM +0100, Valentin Schneider wrote:
> > > @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
> > >       /*
> > >        * Claim the task as running, we do this before switching to it
> > >        * such that any running task will have this set.
> > > +	 *
> > > +	 * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> > > +	 * store against prior state change of @next, also see
> > > +	 * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).
> > 
> > smp_*cond*_load_acquire(&p->on_cpu, <blah>)
> 
> Both, but yeah.. arguably the cond one is the more important one.

Ah no, this one really want to match the WF_ON_CPU case. I'll clarify
non-the-less.

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

* Re: [RFC][PATCH] sched: Better document ttwu()
  2020-07-02 12:52 [RFC][PATCH] sched: Better document ttwu() Peter Zijlstra
                   ` (2 preceding siblings ...)
  2020-07-03 10:12 ` Dietmar Eggemann
@ 2020-07-03 12:39 ` Vincent Guittot
  2020-07-22  9:12 ` [tip: sched/core] " tip-bot2 for Peter Zijlstra
  4 siblings, 0 replies; 9+ messages in thread
From: Vincent Guittot @ 2020-07-03 12:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ingo Molnar, linux-kernel, Mel Gorman, Oleg Nesterov, Dave Chinner

Hi Peter,

On Thu, 2 Jul 2020 at 14:52, Peter Zijlstra <peterz@infradead.org> wrote:
>
>
> Dave hit the problem fixed by commit:
>
>   b6e13e85829f ("sched/core: Fix ttwu() race")
>
> and failed to understand much of the code involved. Per his request a
> few comments to (hopefully) clarify things.
>
> Requested-by: Dave Chinner <david@fromorbit.com>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>

LGTM
I don't have any comment/nit that has not been already mentioned by others

> ---
>  include/linux/sched.h |  12 ++--
>  kernel/sched/core.c   | 195 +++++++++++++++++++++++++++++++++++++++++++-------
>  kernel/sched/sched.h  |  11 +++
>  3 files changed, 187 insertions(+), 31 deletions(-)
>
> diff --git a/include/linux/sched.h b/include/linux/sched.h
> index 9bd073a10224..ad36f70bef24 100644
> --- a/include/linux/sched.h
> +++ b/include/linux/sched.h
> @@ -158,24 +158,24 @@ struct task_group;
>   *
>   *   for (;;) {
>   *     set_current_state(TASK_UNINTERRUPTIBLE);
> - *     if (!need_sleep)
> - *             break;
> + *     if (CONDITION)
> + *        break;
>   *
>   *     schedule();
>   *   }
>   *   __set_current_state(TASK_RUNNING);
>   *
>   * If the caller does not need such serialisation (because, for instance, the
> - * condition test and condition change and wakeup are under the same lock) then
> + * CONDITION test and condition change and wakeup are under the same lock) then
>   * use __set_current_state().
>   *
>   * The above is typically ordered against the wakeup, which does:
>   *
> - *   need_sleep = false;
> + *   CONDITION = 1;
>   *   wake_up_state(p, TASK_UNINTERRUPTIBLE);
>   *
> - * where wake_up_state() executes a full memory barrier before accessing the
> - * task state.
> + * where wake_up_state()/try_to_wake_up() executes a full memory barrier before
> + * accessing p->state.
>   *
>   * Wakeup will do: if (@state & p->state) p->state = TASK_RUNNING, that is,
>   * once it observes the TASK_UNINTERRUPTIBLE store the waking CPU can issue a
> diff --git a/kernel/sched/core.c b/kernel/sched/core.c
> index 1d3d2d67f398..0cd6c336029f 100644
> --- a/kernel/sched/core.c
> +++ b/kernel/sched/core.c
> @@ -77,6 +77,97 @@ __read_mostly int scheduler_running;
>   */
>  int sysctl_sched_rt_runtime = 950000;
>
> +
> +/*
> + * Serialization rules:
> + *
> + * Lock order:
> + *
> + *   p->pi_lock
> + *     rq->lock
> + *       hrtimer_cpu_base->lock (hrtimer_start() for bandwidth controls)
> + *
> + *  rq1->lock
> + *    rq2->lock  where: rq1 < rq2
> + *
> + * Regular state:
> + *
> + * Normal scheduling state is serialized by rq->lock. __schedule() takes the
> + * local CPU's rq->lock, it optionally removes the task from the runqueue and
> + * always looks at the local rq data structures to find the most elegible task
> + * to run next.
> + *
> + * Task enqueue is also under rq->lock, possibly taken from another CPU.
> + * Wakeups from another LLC domain might use an IPI to transfer the enqueue to
> + * the local CPU to avoid bouncing the runqueue state around [ see
> + * ttwu_queue_wakelist() ]
> + *
> + * Task wakeup, specifically wakeups that involve migration, are horribly
> + * complicated to avoid having to take two rq->locks.
> + *
> + * Special state:
> + *
> + * System-calls and anything external will use task_rq_lock() which acquires
> + * both p->lock and rq->lock. As a consequence the state they change is stable
> + * while holding either lock:
> + *
> + *  - sched_setaffinity():     p->cpus_ptr
> + *  - set_user_nice():         p->se.load, p->static_prio
> + *  - __sched_setscheduler():  p->sched_class, p->policy, p->*prio, p->se.load,
> + *                             p->dl.dl_{runtime, deadline, period, flags, bw, density}
> + *  - sched_setnuma():         p->numa_preferred_nid
> + *  - sched_move_task()/
> + *    cpu_cgroup_fork():       p->sched_task_group
> + *
> + * p->state <- TASK_*:
> + *
> + *   is changed locklessly using set_current_state(), __set_current_state() or
> + *   set_special_state(), see their respective comments, or by
> + *   try_to_wake_up(). This latter uses p->pi_lock to serialize against
> + *   concurrent self.
> + *
> + * p->on_rq <- { 0, 1 = TASK_ON_RQ_QUEUED, 2 = TASK_ON_RQ_MIGRATING }:
> + *
> + *   is set by activate_task() and cleared by deactivate_task(), under
> + *   rq->lock. Non-zero indicates the task is runnable, the special
> + *   ON_RQ_MIGRATING state is used for migration without holding both
> + *   rq->locks. It indicates task_cpu() is not stable, see task_rq_lock().
> + *
> + * p->on_cpu <- { 0, 1 }:
> + *
> + *   is set by prepare_task() and cleared by finish_task() such that it will be
> + *   set before p is scheduled-in and cleared after p is scheduled-out, both
> + *   under rq->lock. Non-zero indicates the task is running on it's CPU.
> + *
> + *   [ The astute reader will observe that it is possible for two tasks on one
> + *     CPU to have ->on_cpu = 1 at the same time. ]
> + *
> + * task_cpu(p): is changed by set_task_cpu(), the rules are:
> + *
> + *  - Don't call set_task_cpu() on a blocked task:
> + *
> + *    We don't care what CPU we're not running on, this simplifies hotplug,
> + *    the CPU assignment of blocked tasks isn't required to be valid.
> + *
> + *  - for try_to_wake_up(), called under p->pi_lock:
> + *
> + *    This allows try_to_wake_up() to only take one rq->lock, see its comment.
> + *
> + *  - for migration called under rq->lock:
> + *    [ see task_on_rq_migrating() in task_rq_lock() ]
> + *
> + *    o move_queued_task()
> + *    o __migrate_swap_task()
> + *    o detach_task()
> + *
> + *  - for migration called under double_rq_lock():
> + *
> + *    o push_rt_task() / pull_rt_task()
> + *    o push_dl_task() / pull_dl_task()
> + *    o dl_task_offline_migration()
> + *
> + */
> +
>  /*
>   * __task_rq_lock - lock the rq @p resides on.
>   */
> @@ -1466,8 +1557,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
>  {
>         lockdep_assert_held(&rq->lock);
>
> -       WRITE_ONCE(p->on_rq, TASK_ON_RQ_MIGRATING);
> -       dequeue_task(rq, p, DEQUEUE_NOCLOCK);
> +       deactivate_task(rq, p, DEQUEUE_NOCLOCK);
>         set_task_cpu(p, new_cpu);
>         rq_unlock(rq, rf);
>
> @@ -1475,8 +1565,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
>
>         rq_lock(rq, rf);
>         BUG_ON(task_cpu(p) != new_cpu);
> -       enqueue_task(rq, p, 0);
> -       p->on_rq = TASK_ON_RQ_QUEUED;
> +       activate_task(rq, p, 0);
>         check_preempt_curr(rq, p, 0);
>
>         return rq;
> @@ -2241,12 +2330,31 @@ ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
>  }
>
>  /*
> - * Called in case the task @p isn't fully descheduled from its runqueue,
> - * in this case we must do a remote wakeup. Its a 'light' wakeup though,
> - * since all we need to do is flip p->state to TASK_RUNNING, since
> - * the task is still ->on_rq.
> + * Consider @p being inside a wait loop:
> + *
> + *   for (;;) {
> + *     set_current_state(TASK_UNINTERRUPTIBLE);
> + *
> + *     if (CONDITION)
> + *         break;
> + *
> + *     schedule();
> + *   }
> + *   __set_current_state(TASK_RUNNING);
> + *
> + * between set_current_state() and schedule(). In this case @p is still
> + * runnable, so all that needs doing is change p->state back to TASK_RUNNING in
> + * an atomic manner.
> + *
> + * By taking task_rq(p)->lock we serialize against schedule(), if @p->on_rq
> + * then schedule() must still happen and p->state can be changed to
> + * TASK_RUNNING. Otherwise we lost the race, schedule() has happened, and we
> + * need to do a full wakeup with enqueue.
> + *
> + * Returns: %true when the wakeup is done,
> + *          %false otherwise.
>   */
> -static int ttwu_remote(struct task_struct *p, int wake_flags)
> +static int ttwu_runnable(struct task_struct *p, int wake_flags)
>  {
>         struct rq_flags rf;
>         struct rq *rq;
> @@ -2387,6 +2495,14 @@ static bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
>
>         return false;
>  }
> +
> +#else /* !CONFIG_SMP */
> +
> +static inline bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
> +{
> +       return false;
> +}
> +
>  #endif /* CONFIG_SMP */
>
>  static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
> @@ -2394,10 +2510,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>         struct rq *rq = cpu_rq(cpu);
>         struct rq_flags rf;
>
> -#if defined(CONFIG_SMP)
>         if (ttwu_queue_wakelist(p, cpu, wake_flags))
>                 return;
> -#endif
>
>         rq_lock(rq, &rf);
>         update_rq_clock(rq);
> @@ -2453,8 +2567,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>   * migration. However the means are completely different as there is no lock
>   * chain to provide order. Instead we do:
>   *
> - *   1) smp_store_release(X->on_cpu, 0)
> - *   2) smp_cond_load_acquire(!X->on_cpu)
> + *   1) smp_store_release(X->on_cpu, 0)   -- finish_task()
> + *   2) smp_cond_load_acquire(!X->on_cpu) -- try_to_wake_up()
>   *
>   * Example:
>   *
> @@ -2494,15 +2608,41 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
>   * @state: the mask of task states that can be woken
>   * @wake_flags: wake modifier flags (WF_*)
>   *
> - * If (@state & @p->state) @p->state = TASK_RUNNING.
> + * Conceptually does:
> + *
> + *   If (@state & @p->state) @p->state = TASK_RUNNING.
>   *
>   * If the task was not queued/runnable, also place it back on a runqueue.
>   *
> - * Atomic against schedule() which would dequeue a task, also see
> - * set_current_state().
> + * This function:
> + *  - is atomic against schedule() which would dequeue the task;
> + *  - issues a full memory barrier before accessing @p->state.
> + * See the comment with set_current_state().
> + *
> + * Uses p->pi_lock to serialize against concurrent wake-ups.
> + *
> + * Relies on p->pi_lock stabilizing:
> + *  - p->sched_class
> + *  - p->cpus_ptr
> + *  - p->sched_task_group
> + * in order to do migration, see its use of select_task_rq()/set_task_cpu().
>   *
> - * This function executes a full memory barrier before accessing the task
> - * state; see set_current_state().
> + * Tries really hard to only take one task_rq(p)->lock for performance.
> + * Takes rq->lock in:
> + *  - ttwu_runnable()    -- old rq, unavoidable, see comment there;
> + *  - ttwu_queue()       -- new rq, for enqueue of the task;
> + *  - psi_ttwu_dequeue() -- much sadness :-( accounting will kill us.
> + *
> + * As a concequence we race really badly with just about everything. See the
> + * many memory barriers and their comments for details. The basic order of
> + * reading things is:
> + *
> + *   LOAD p->state
> + *   RMB
> + *   LOAD p->on_rq
> + *   RMB
> + *   LOAD-ACQUIRE p->on_cpu
> + *   LOAD task_cpu()
>   *
>   * Return: %true if @p->state changes (an actual wakeup was done),
>   *        %false otherwise.
> @@ -2518,7 +2658,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>                 /*
>                  * We're waking current, this means 'p->on_rq' and 'task_cpu(p)
>                  * == smp_processor_id()'. Together this means we can special
> -                * case the whole 'p->on_rq && ttwu_remote()' case below
> +                * case the whole 'p->on_rq && ttwu_runnable()' case below
>                  * without taking any locks.
>                  *
>                  * In particular:
> @@ -2539,8 +2679,8 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>         /*
>          * If we are going to wake up a thread waiting for CONDITION we
>          * need to ensure that CONDITION=1 done by the caller can not be
> -        * reordered with p->state check below. This pairs with mb() in
> -        * set_current_state() the waiting thread does.
> +        * reordered with p->state check below. This pairs with smp_store_mb()
> +        * in set_current_state() that the waiting thread does.
>          */
>         raw_spin_lock_irqsave(&p->pi_lock, flags);
>         smp_mb__after_spinlock();
> @@ -2575,7 +2715,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
>          * A similar smb_rmb() lives in try_invoke_on_locked_down_task().
>          */
>         smp_rmb();
> -       if (p->on_rq && ttwu_remote(p, wake_flags))
> +       if (p->on_rq && ttwu_runnable(p, wake_flags))
>                 goto unlock;
>
>         if (p->in_iowait) {
> @@ -3134,8 +3274,12 @@ static inline void prepare_task(struct task_struct *next)
>         /*
>          * Claim the task as running, we do this before switching to it
>          * such that any running task will have this set.
> +        *
> +        * __schedule()'s rq->lock and smp_mb__after_spin_lock() orders this
> +        * store against prior state change of @next, also see
> +        * try_to_wake_up(), specifically smp_load_acquire(&p->on_cpu).
>          */
> -       next->on_cpu = 1;
> +       WRITE_ONCE(next->on_cpu, 1);
>  #endif
>  }
>
> @@ -3143,8 +3287,9 @@ static inline void finish_task(struct task_struct *prev)
>  {
>  #ifdef CONFIG_SMP
>         /*
> -        * After ->on_cpu is cleared, the task can be moved to a different CPU.
> -        * We must ensure this doesn't happen until the switch is completely
> +        * This must be the very last reference to @prev from this CPU. After
> +        * p->on_cpu is cleared, the task can be moved to a different CPU. We
> +        * must ensure this doesn't happen until the switch is completely
>          * finished.
>          *
>          * In particular, the load of prev->state in finish_task_switch() must
> diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
> index 5aa6661ecaf1..73c0c5d0034b 100644
> --- a/kernel/sched/sched.h
> +++ b/kernel/sched/sched.h
> @@ -1197,6 +1197,17 @@ struct rq_flags {
>  #endif
>  };
>
> +/*
> + * Lockdep annotation that avoid accidental unlock; any
> + * raw_spin_unlock(&rq->lock) without preceding rq_unpin_lock() with the
> + * correct cookie will result in a WARN.
> + *
> + * This avoids code that has access to 'struct rq *rq' (basically everything in
> + * the scheduler) from accidentally unlocking the rq if they do not also have a
> + * copy of the (on-stack) 'struct rq_flags rf'.
> + *
> + * Also see Documentation/locking/lockdep-design.rst.
> + */
>  static inline void rq_pin_lock(struct rq *rq, struct rq_flags *rf)
>  {
>         rf->cookie = lockdep_pin_lock(&rq->lock);

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

* [tip: sched/core] sched: Better document ttwu()
  2020-07-02 12:52 [RFC][PATCH] sched: Better document ttwu() Peter Zijlstra
                   ` (3 preceding siblings ...)
  2020-07-03 12:39 ` Vincent Guittot
@ 2020-07-22  9:12 ` tip-bot2 for Peter Zijlstra
  4 siblings, 0 replies; 9+ messages in thread
From: tip-bot2 for Peter Zijlstra @ 2020-07-22  9:12 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Peter Zijlstra (Intel), x86, LKML

The following commit has been merged into the sched/core branch of tip:

Commit-ID:     58877d347b58c9e971112df5eb311c13bb0acb28
Gitweb:        https://git.kernel.org/tip/58877d347b58c9e971112df5eb311c13bb0acb28
Author:        Peter Zijlstra <peterz@infradead.org>
AuthorDate:    Thu, 02 Jul 2020 14:52:11 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Wed, 22 Jul 2020 10:22:03 +02:00

sched: Better document ttwu()

Dave hit the problem fixed by commit:

  b6e13e85829f ("sched/core: Fix ttwu() race")

and failed to understand much of the code involved. Per his request a
few comments to (hopefully) clarify things.

Requested-by: Dave Chinner <david@fromorbit.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lkml.kernel.org/r/20200702125211.GQ4800@hirez.programming.kicks-ass.net
---
 include/linux/sched.h |  12 +--
 kernel/sched/core.c   | 188 +++++++++++++++++++++++++++++++++++------
 kernel/sched/sched.h  |  10 ++-
 3 files changed, 179 insertions(+), 31 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 12b10ce..5033813 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -154,24 +154,24 @@ struct task_group;
  *
  *   for (;;) {
  *	set_current_state(TASK_UNINTERRUPTIBLE);
- *	if (!need_sleep)
- *		break;
+ *	if (CONDITION)
+ *	   break;
  *
  *	schedule();
  *   }
  *   __set_current_state(TASK_RUNNING);
  *
  * If the caller does not need such serialisation (because, for instance, the
- * condition test and condition change and wakeup are under the same lock) then
+ * CONDITION test and condition change and wakeup are under the same lock) then
  * use __set_current_state().
  *
  * The above is typically ordered against the wakeup, which does:
  *
- *   need_sleep = false;
+ *   CONDITION = 1;
  *   wake_up_state(p, TASK_UNINTERRUPTIBLE);
  *
- * where wake_up_state() executes a full memory barrier before accessing the
- * task state.
+ * where wake_up_state()/try_to_wake_up() executes a full memory barrier before
+ * accessing p->state.
  *
  * Wakeup will do: if (@state & p->state) p->state = TASK_RUNNING, that is,
  * once it observes the TASK_UNINTERRUPTIBLE store the waking CPU can issue a
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 08d02ce..12db8fb 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -79,6 +79,100 @@ __read_mostly int scheduler_running;
  */
 int sysctl_sched_rt_runtime = 950000;
 
+
+/*
+ * Serialization rules:
+ *
+ * Lock order:
+ *
+ *   p->pi_lock
+ *     rq->lock
+ *       hrtimer_cpu_base->lock (hrtimer_start() for bandwidth controls)
+ *
+ *  rq1->lock
+ *    rq2->lock  where: rq1 < rq2
+ *
+ * Regular state:
+ *
+ * Normal scheduling state is serialized by rq->lock. __schedule() takes the
+ * local CPU's rq->lock, it optionally removes the task from the runqueue and
+ * always looks at the local rq data structures to find the most elegible task
+ * to run next.
+ *
+ * Task enqueue is also under rq->lock, possibly taken from another CPU.
+ * Wakeups from another LLC domain might use an IPI to transfer the enqueue to
+ * the local CPU to avoid bouncing the runqueue state around [ see
+ * ttwu_queue_wakelist() ]
+ *
+ * Task wakeup, specifically wakeups that involve migration, are horribly
+ * complicated to avoid having to take two rq->locks.
+ *
+ * Special state:
+ *
+ * System-calls and anything external will use task_rq_lock() which acquires
+ * both p->pi_lock and rq->lock. As a consequence the state they change is
+ * stable while holding either lock:
+ *
+ *  - sched_setaffinity()/
+ *    set_cpus_allowed_ptr():	p->cpus_ptr, p->nr_cpus_allowed
+ *  - set_user_nice():		p->se.load, p->*prio
+ *  - __sched_setscheduler():	p->sched_class, p->policy, p->*prio,
+ *				p->se.load, p->rt_priority,
+ *				p->dl.dl_{runtime, deadline, period, flags, bw, density}
+ *  - sched_setnuma():		p->numa_preferred_nid
+ *  - sched_move_task()/
+ *    cpu_cgroup_fork():	p->sched_task_group
+ *  - uclamp_update_active()	p->uclamp*
+ *
+ * p->state <- TASK_*:
+ *
+ *   is changed locklessly using set_current_state(), __set_current_state() or
+ *   set_special_state(), see their respective comments, or by
+ *   try_to_wake_up(). This latter uses p->pi_lock to serialize against
+ *   concurrent self.
+ *
+ * p->on_rq <- { 0, 1 = TASK_ON_RQ_QUEUED, 2 = TASK_ON_RQ_MIGRATING }:
+ *
+ *   is set by activate_task() and cleared by deactivate_task(), under
+ *   rq->lock. Non-zero indicates the task is runnable, the special
+ *   ON_RQ_MIGRATING state is used for migration without holding both
+ *   rq->locks. It indicates task_cpu() is not stable, see task_rq_lock().
+ *
+ * p->on_cpu <- { 0, 1 }:
+ *
+ *   is set by prepare_task() and cleared by finish_task() such that it will be
+ *   set before p is scheduled-in and cleared after p is scheduled-out, both
+ *   under rq->lock. Non-zero indicates the task is running on its CPU.
+ *
+ *   [ The astute reader will observe that it is possible for two tasks on one
+ *     CPU to have ->on_cpu = 1 at the same time. ]
+ *
+ * task_cpu(p): is changed by set_task_cpu(), the rules are:
+ *
+ *  - Don't call set_task_cpu() on a blocked task:
+ *
+ *    We don't care what CPU we're not running on, this simplifies hotplug,
+ *    the CPU assignment of blocked tasks isn't required to be valid.
+ *
+ *  - for try_to_wake_up(), called under p->pi_lock:
+ *
+ *    This allows try_to_wake_up() to only take one rq->lock, see its comment.
+ *
+ *  - for migration called under rq->lock:
+ *    [ see task_on_rq_migrating() in task_rq_lock() ]
+ *
+ *    o move_queued_task()
+ *    o detach_task()
+ *
+ *  - for migration called under double_rq_lock():
+ *
+ *    o __migrate_swap_task()
+ *    o push_rt_task() / pull_rt_task()
+ *    o push_dl_task() / pull_dl_task()
+ *    o dl_task_offline_migration()
+ *
+ */
+
 /*
  * __task_rq_lock - lock the rq @p resides on.
  */
@@ -1543,8 +1637,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
 {
 	lockdep_assert_held(&rq->lock);
 
-	WRITE_ONCE(p->on_rq, TASK_ON_RQ_MIGRATING);
-	dequeue_task(rq, p, DEQUEUE_NOCLOCK);
+	deactivate_task(rq, p, DEQUEUE_NOCLOCK);
 	set_task_cpu(p, new_cpu);
 	rq_unlock(rq, rf);
 
@@ -1552,8 +1645,7 @@ static struct rq *move_queued_task(struct rq *rq, struct rq_flags *rf,
 
 	rq_lock(rq, rf);
 	BUG_ON(task_cpu(p) != new_cpu);
-	enqueue_task(rq, p, 0);
-	p->on_rq = TASK_ON_RQ_QUEUED;
+	activate_task(rq, p, 0);
 	check_preempt_curr(rq, p, 0);
 
 	return rq;
@@ -2318,12 +2410,31 @@ ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
 }
 
 /*
- * Called in case the task @p isn't fully descheduled from its runqueue,
- * in this case we must do a remote wakeup. Its a 'light' wakeup though,
- * since all we need to do is flip p->state to TASK_RUNNING, since
- * the task is still ->on_rq.
+ * Consider @p being inside a wait loop:
+ *
+ *   for (;;) {
+ *      set_current_state(TASK_UNINTERRUPTIBLE);
+ *
+ *      if (CONDITION)
+ *         break;
+ *
+ *      schedule();
+ *   }
+ *   __set_current_state(TASK_RUNNING);
+ *
+ * between set_current_state() and schedule(). In this case @p is still
+ * runnable, so all that needs doing is change p->state back to TASK_RUNNING in
+ * an atomic manner.
+ *
+ * By taking task_rq(p)->lock we serialize against schedule(), if @p->on_rq
+ * then schedule() must still happen and p->state can be changed to
+ * TASK_RUNNING. Otherwise we lost the race, schedule() has happened, and we
+ * need to do a full wakeup with enqueue.
+ *
+ * Returns: %true when the wakeup is done,
+ *          %false otherwise.
  */
-static int ttwu_remote(struct task_struct *p, int wake_flags)
+static int ttwu_runnable(struct task_struct *p, int wake_flags)
 {
 	struct rq_flags rf;
 	struct rq *rq;
@@ -2464,6 +2575,14 @@ static bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
 
 	return false;
 }
+
+#else /* !CONFIG_SMP */
+
+static inline bool ttwu_queue_wakelist(struct task_struct *p, int cpu, int wake_flags)
+{
+	return false;
+}
+
 #endif /* CONFIG_SMP */
 
 static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
@@ -2471,10 +2590,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
 	struct rq *rq = cpu_rq(cpu);
 	struct rq_flags rf;
 
-#if defined(CONFIG_SMP)
 	if (ttwu_queue_wakelist(p, cpu, wake_flags))
 		return;
-#endif
 
 	rq_lock(rq, &rf);
 	update_rq_clock(rq);
@@ -2530,8 +2647,8 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
  * migration. However the means are completely different as there is no lock
  * chain to provide order. Instead we do:
  *
- *   1) smp_store_release(X->on_cpu, 0)
- *   2) smp_cond_load_acquire(!X->on_cpu)
+ *   1) smp_store_release(X->on_cpu, 0)   -- finish_task()
+ *   2) smp_cond_load_acquire(!X->on_cpu) -- try_to_wake_up()
  *
  * Example:
  *
@@ -2571,15 +2688,33 @@ static void ttwu_queue(struct task_struct *p, int cpu, int wake_flags)
  * @state: the mask of task states that can be woken
  * @wake_flags: wake modifier flags (WF_*)
  *
- * If (@state & @p->state) @p->state = TASK_RUNNING.
+ * Conceptually does:
+ *
+ *   If (@state & @p->state) @p->state = TASK_RUNNING.
  *
  * If the task was not queued/runnable, also place it back on a runqueue.
  *
- * Atomic against schedule() which would dequeue a task, also see
- * set_current_state().
+ * This function is atomic against schedule() which would dequeue the task.
+ *
+ * It issues a full memory barrier before accessing @p->state, see the comment
+ * with set_current_state().
+ *
+ * Uses p->pi_lock to serialize against concurrent wake-ups.
  *
- * This function executes a full memory barrier before accessing the task
- * state; see set_current_state().
+ * Relies on p->pi_lock stabilizing:
+ *  - p->sched_class
+ *  - p->cpus_ptr
+ *  - p->sched_task_group
+ * in order to do migration, see its use of select_task_rq()/set_task_cpu().
+ *
+ * Tries really hard to only take one task_rq(p)->lock for performance.
+ * Takes rq->lock in:
+ *  - ttwu_runnable()    -- old rq, unavoidable, see comment there;
+ *  - ttwu_queue()       -- new rq, for enqueue of the task;
+ *  - psi_ttwu_dequeue() -- much sadness :-( accounting will kill us.
+ *
+ * As a consequence we race really badly with just about everything. See the
+ * many memory barriers and their comments for details.
  *
  * Return: %true if @p->state changes (an actual wakeup was done),
  *	   %false otherwise.
@@ -2595,7 +2730,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 		/*
 		 * We're waking current, this means 'p->on_rq' and 'task_cpu(p)
 		 * == smp_processor_id()'. Together this means we can special
-		 * case the whole 'p->on_rq && ttwu_remote()' case below
+		 * case the whole 'p->on_rq && ttwu_runnable()' case below
 		 * without taking any locks.
 		 *
 		 * In particular:
@@ -2616,8 +2751,8 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 	/*
 	 * If we are going to wake up a thread waiting for CONDITION we
 	 * need to ensure that CONDITION=1 done by the caller can not be
-	 * reordered with p->state check below. This pairs with mb() in
-	 * set_current_state() the waiting thread does.
+	 * reordered with p->state check below. This pairs with smp_store_mb()
+	 * in set_current_state() that the waiting thread does.
 	 */
 	raw_spin_lock_irqsave(&p->pi_lock, flags);
 	smp_mb__after_spinlock();
@@ -2652,7 +2787,7 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 	 * A similar smb_rmb() lives in try_invoke_on_locked_down_task().
 	 */
 	smp_rmb();
-	if (READ_ONCE(p->on_rq) && ttwu_remote(p, wake_flags))
+	if (READ_ONCE(p->on_rq) && ttwu_runnable(p, wake_flags))
 		goto unlock;
 
 	if (p->in_iowait) {
@@ -3222,8 +3357,10 @@ static inline void prepare_task(struct task_struct *next)
 	/*
 	 * Claim the task as running, we do this before switching to it
 	 * such that any running task will have this set.
+	 *
+	 * See the ttwu() WF_ON_CPU case and its ordering comment.
 	 */
-	next->on_cpu = 1;
+	WRITE_ONCE(next->on_cpu, 1);
 #endif
 }
 
@@ -3231,8 +3368,9 @@ static inline void finish_task(struct task_struct *prev)
 {
 #ifdef CONFIG_SMP
 	/*
-	 * After ->on_cpu is cleared, the task can be moved to a different CPU.
-	 * We must ensure this doesn't happen until the switch is completely
+	 * This must be the very last reference to @prev from this CPU. After
+	 * p->on_cpu is cleared, the task can be moved to a different CPU. We
+	 * must ensure this doesn't happen until the switch is completely
 	 * finished.
 	 *
 	 * In particular, the load of prev->state in finish_task_switch() must
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 65b72e0..9f33c77 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -1203,6 +1203,16 @@ struct rq_flags {
 #endif
 };
 
+/*
+ * Lockdep annotation that avoids accidental unlocks; it's like a
+ * sticky/continuous lockdep_assert_held().
+ *
+ * This avoids code that has access to 'struct rq *rq' (basically everything in
+ * the scheduler) from accidentally unlocking the rq if they do not also have a
+ * copy of the (on-stack) 'struct rq_flags rf'.
+ *
+ * Also see Documentation/locking/lockdep-design.rst.
+ */
 static inline void rq_pin_lock(struct rq *rq, struct rq_flags *rf)
 {
 	rf->cookie = lockdep_pin_lock(&rq->lock);

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

end of thread, other threads:[~2020-07-22  9:12 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-07-02 12:52 [RFC][PATCH] sched: Better document ttwu() Peter Zijlstra
2020-07-02 13:13 ` Phil Auld
2020-07-02 15:23   ` Peter Zijlstra
2020-07-02 18:39 ` Valentin Schneider
2020-07-03  8:30   ` Peter Zijlstra
2020-07-03 11:36     ` Peter Zijlstra
2020-07-03 10:12 ` Dietmar Eggemann
2020-07-03 12:39 ` Vincent Guittot
2020-07-22  9:12 ` [tip: sched/core] " tip-bot2 for Peter Zijlstra

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.