[PATCH v2 10/12] sched: Add proxy execution

From: John Stultz
Date: Mon Mar 20 2023 - 19:38:35 EST


From: Peter Zijlstra <peterz@xxxxxxxxxxxxx>

A task currently holding a mutex (executing a critical section)
might find benefit in using scheduling contexts of other tasks
blocked on the same mutex if they happen to have higher priority
of the current owner (e.g., to prevent priority inversions).

Proxy execution lets a task do exactly that: if a mutex owner
has waiters, it can use waiters' scheduling context to
potentially continue running if preempted.

The basic mechanism is implemented by this patch, the core of
which resides in the proxy() function. Potential proxies (i.e.,
tasks blocked on a mutex) are not dequeued, so, if one of them
is actually selected by schedule() as the next task to be put to
run on a CPU, proxy() is used to walk the blocked_on relation
and find which task (mutex owner) might be able to use the
proxy's scheduling context.

Here come the tricky bits. In fact, owner task might be in all
sort of states when a proxy is found (blocked, executing on a
different CPU, etc.). Details on how to handle different
situations are to be found in proxy() code comments.

Cc: Joel Fernandes <joelaf@xxxxxxxxxx>
Cc: Qais Yousef <qyousef@xxxxxxxxxx>
Cc: Ingo Molnar <mingo@xxxxxxxxxx>
Cc: Peter Zijlstra <peterz@xxxxxxxxxxxxx>
Cc: Juri Lelli <juri.lelli@xxxxxxxxxx>
Cc: Vincent Guittot <vincent.guittot@xxxxxxxxxx>
Cc: Dietmar Eggemann <dietmar.eggemann@xxxxxxx>
Cc: Valentin Schneider <vschneid@xxxxxxxxxx>
Cc: Steven Rostedt <rostedt@xxxxxxxxxxx>
Cc: Ben Segall <bsegall@xxxxxxxxxx>
Cc: Zimuzo Ezeozue <zezeozue@xxxxxxxxxx>
Cc: Mel Gorman <mgorman@xxxxxxx>
Cc: Daniel Bristot de Oliveira <bristot@xxxxxxxxxx>
Cc: Will Deacon <will@xxxxxxxxxx>
Cc: Waiman Long <longman@xxxxxxxxxx>
Cc: Boqun Feng <boqun.feng@xxxxxxxxx>
Cc: "Paul E . McKenney" <paulmck@xxxxxxxxxx>
Cc: kernel-team@xxxxxxxxxxx
Signed-off-by: Peter Zijlstra (Intel) <peterz@xxxxxxxxxxxxx>
[rebased, added comments and changelog]
Signed-off-by: Juri Lelli <juri.lelli@xxxxxxxxxx>
[Fixed rebase conflicts]
[squashed sched: Ensure blocked_on is always guarded by blocked_lock]
Signed-off-by: Valentin Schneider <valentin.schneider@xxxxxxx>
[fix rebase conflicts, various fixes & tweaks commented inline]
[squashed sched: Use rq->curr vs rq->proxy checks]
Signed-off-by: Connor O'Brien <connoro@xxxxxxxxxx>
[jstultz: Rebased, split up, and folded in changes from Juri
Lelli and Connor O'Brian, added additional locking on
get_task_blocked_on(next) logic, pretty major rework to better
conditionalize logic on CONFIG_PROXY_EXEC and split up the very
large proxy() function - hopefully without changes to logic /
behavior]
Signed-off-by: John Stultz <jstultz@xxxxxxxxxx>
---
v2:
* Numerous changes folded in
* Split out some of the logic into separate patches
* Break up the proxy() function so its a bit easier to read
and is better conditionalized on CONFIG_PROXY_EXEC

TODO: Finish conditionalization edge cases
---
include/linux/sched.h | 1 +
init/Kconfig | 7 +
kernel/Kconfig.locks | 2 +-
kernel/fork.c | 1 +
kernel/locking/mutex.c | 58 +++-
kernel/sched/core.c | 652 +++++++++++++++++++++++++++++++++++++++-
kernel/sched/deadline.c | 2 +-
kernel/sched/fair.c | 13 +-
kernel/sched/rt.c | 3 +-
kernel/sched/sched.h | 21 +-
10 files changed, 744 insertions(+), 16 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index fc75fcc696db..b88303ceacaf 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1141,6 +1141,7 @@ struct task_struct {

struct task_struct *blocked_proxy; /* task that is boosting us */
struct mutex *blocked_on; /* lock we're blocked on */
+ struct list_head blocked_entry; /* tasks blocked on us */
raw_spinlock_t blocked_lock;

#ifdef CONFIG_DEBUG_ATOMIC_SLEEP
diff --git a/init/Kconfig b/init/Kconfig
index 1fb5f313d18f..38cdd2ccc538 100644
--- a/init/Kconfig
+++ b/init/Kconfig
@@ -935,6 +935,13 @@ config NUMA_BALANCING_DEFAULT_ENABLED
If set, automatic NUMA balancing will be enabled if running on a NUMA
machine.

+config PROXY_EXEC
+ bool "Proxy Execution"
+ default n
+ help
+ This option enables proxy execution, a mechanism for mutex owning
+ tasks to inherit the scheduling context of higher priority waiters.
+
menuconfig CGROUPS
bool "Control Group support"
select KERNFS
diff --git a/kernel/Kconfig.locks b/kernel/Kconfig.locks
index 4198f0273ecd..791c98f1d329 100644
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -226,7 +226,7 @@ config ARCH_SUPPORTS_ATOMIC_RMW

config MUTEX_SPIN_ON_OWNER
def_bool y
- depends on SMP && ARCH_SUPPORTS_ATOMIC_RMW
+ depends on SMP && ARCH_SUPPORTS_ATOMIC_RMW && !PROXY_EXEC

config RWSEM_SPIN_ON_OWNER
def_bool y
diff --git a/kernel/fork.c b/kernel/fork.c
index 95410333332f..a56c59c7a4cf 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -2226,6 +2226,7 @@ static __latent_entropy struct task_struct *copy_process(

p->blocked_proxy = NULL; /* nobody is boosting us yet */
p->blocked_on = NULL; /* not blocked yet */
+ INIT_LIST_HEAD(&p->blocked_entry);

#ifdef CONFIG_BCACHE
p->sequential_io = 0;
diff --git a/kernel/locking/mutex.c b/kernel/locking/mutex.c
index ead4213232cc..370687cf79f8 100644
--- a/kernel/locking/mutex.c
+++ b/kernel/locking/mutex.c
@@ -939,10 +939,21 @@ static noinline void __sched __mutex_unlock_slowpath(struct mutex *lock, unsigne
{
struct task_struct *next = NULL;
DEFINE_WAKE_Q(wake_q);
- unsigned long owner;
+ /*
+ * XXX [juril] Proxy Exec forces always an HANDOFF (so that owner is
+ * never empty when there are waiters waiting?). Should we make this
+ * conditional on having proxy exec configured in?
+ */
+ unsigned long owner = MUTEX_FLAG_HANDOFF;

mutex_release(&lock->dep_map, ip);

+ /*
+ * XXX must always handoff the mutex to avoid !owner in proxy().
+ * scheduler delay is minimal since we hand off to the task that
+ * is to be scheduled next.
+ */
+#ifndef CONFIG_PROXY_EXEC
/*
* Release the lock before (potentially) taking the spinlock such that
* other contenders can get on with things ASAP.
@@ -965,10 +976,48 @@ static noinline void __sched __mutex_unlock_slowpath(struct mutex *lock, unsigne
return;
}
}
+#endif

raw_spin_lock(&lock->wait_lock);
debug_mutex_unlock(lock);
- if (!list_empty(&lock->wait_list)) {
+
+#ifdef CONFIG_PROXY_EXEC
+ raw_spin_lock(&current->blocked_lock);
+ /*
+ * If we have a task boosting us, and that task was boosting us through
+ * this lock, hand the lock to that task, as that is the highest
+ * waiter, as selected by the scheduling function.
+ */
+ next = current->blocked_proxy;
+ if (next) {
+ struct mutex *next_lock;
+
+ /*
+ * jstultz: get_task_blocked_on(next) seemed to be missing locking
+ * so I've added it here (which required nesting the locks).
+ */
+ raw_spin_lock_nested(&next->blocked_lock, SINGLE_DEPTH_NESTING);
+ next_lock = get_task_blocked_on(next);
+ raw_spin_unlock(&next->blocked_lock);
+ if (next_lock != lock) {
+ next = NULL;
+ } else {
+ wake_q_add(&wake_q, next);
+ current->blocked_proxy = NULL;
+ }
+ }
+
+ /*
+ * XXX if there was no higher prio proxy, ->blocked_task will not have
+ * been set. Therefore lower prio contending tasks are serviced in
+ * FIFO order.
+ */
+#endif
+
+ /*
+ * Failing that, pick any on the wait list.
+ */
+ if (!next && !list_empty(&lock->wait_list)) {
/* get the first entry from the wait-list: */
struct mutex_waiter *waiter =
list_first_entry(&lock->wait_list,
@@ -983,7 +1032,10 @@ static noinline void __sched __mutex_unlock_slowpath(struct mutex *lock, unsigne
if (owner & MUTEX_FLAG_HANDOFF)
__mutex_handoff(lock, next);

- preempt_disable();
+ preempt_disable(); /* XXX connoro: why disable preemption here? */
+#ifdef CONFIG_PROXY_EXEC
+ raw_spin_unlock(&current->blocked_lock);
+#endif
raw_spin_unlock(&lock->wait_lock);

wake_up_q(&wake_q);
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 82f5b29ae675..d0f86670bdf8 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -505,6 +505,8 @@ sched_core_dequeue(struct rq *rq, struct task_struct *p, int flags) { }
*
* task_cpu(p): is changed by set_task_cpu(), the rules are:
*
+ * XXX connoro: does it matter that ttwu_do_activate now calls __set_task_cpu
+ * on blocked tasks?
* - Don't call set_task_cpu() on a blocked task:
*
* We don't care what CPU we're not running on, this simplifies hotplug,
@@ -2774,8 +2776,15 @@ static int affine_move_task(struct rq *rq, struct task_struct *p, struct rq_flag
struct set_affinity_pending my_pending = { }, *pending = NULL;
bool stop_pending, complete = false;

- /* Can the task run on the task's current CPU? If so, we're done */
- if (cpumask_test_cpu(task_cpu(p), &p->cpus_mask)) {
+ /*
+ * Can the task run on the task's current CPU? If so, we're done
+ *
+ * We are also done if the task is currently acting as proxy (and
+ * potentially has been migrated outside its current or previous
+ * affinity mask)
+ */
+ if (cpumask_test_cpu(task_cpu(p), &p->cpus_mask) ||
+ (task_current_proxy(rq, p) && !task_current(rq, p))) {
struct task_struct *push_task = NULL;

if ((flags & SCA_MIGRATE_ENABLE) &&
@@ -3687,6 +3696,72 @@ static inline void ttwu_do_wakeup(struct task_struct *p)
trace_sched_wakeup(p);
}

+#ifdef CONFIG_PROXY_EXEC
+static void activate_task_and_blocked_ent(struct rq *rq, struct task_struct *p, int en_flags)
+{
+ /*
+ * XXX connoro: By calling activate_task with blocked_lock held, we order against
+ * the proxy() blocked_task case such that no more blocked tasks will
+ * be enqueued on p once we release p->blocked_lock.
+ */
+ raw_spin_lock(&p->blocked_lock);
+ /*
+ * XXX connoro: do we need to check p->on_rq here like we do for pp below?
+ * or does holding p->pi_lock ensure nobody else activates p first?
+ */
+ activate_task(rq, p, en_flags);
+ raw_spin_unlock(&p->blocked_lock);
+
+ /*
+ * A whole bunch of 'proxy' tasks back this blocked task, wake
+ * them all up to give this task its 'fair' share.
+ */
+ while (!list_empty(&p->blocked_entry)) {
+ struct task_struct *pp =
+ list_first_entry(&p->blocked_entry,
+ struct task_struct,
+ blocked_entry);
+ /*
+ * XXX connoro: proxy blocked_task case might be enqueuing more blocked tasks
+ * on pp. If those continue past when we delete pp from the list, we'll get an
+ * active with a non-empty blocked_entry list, which is no good. Locking
+ * pp->blocked_lock ensures either the blocked_task path gets the lock first and
+ * enqueues everything before we ever get the lock, or we get the lock first, the
+ * other path sees pp->on_rq != 0 and enqueues nothing.
+ */
+ raw_spin_lock(&pp->blocked_lock);
+ BUG_ON(pp->blocked_entry.prev != &p->blocked_entry);
+
+ list_del_init(&pp->blocked_entry);
+ if (READ_ONCE(pp->on_rq)) {
+ /*
+ * XXX connoro: We raced with a non mutex handoff activation of pp. That
+ * activation will also take care of activating all of the tasks after pp in
+ * the blocked_entry list, so we're done here.
+ */
+ raw_spin_unlock(&pp->blocked_lock);
+ break;
+ }
+ /* XXX can't call set_task_cpu() because we are not holding
+ * neither pp->pi_lock nor task's rq lock. This should however
+ * be fine as this task can't be woken up as it is blocked on
+ * this mutex atm.
+ * A problem however might be that __set_task_cpu() calls
+ * set_task_rq() which deals with groups and such...
+ */
+ __set_task_cpu(pp, cpu_of(rq));
+ activate_task(rq, pp, en_flags);
+ resched_curr(rq);
+ raw_spin_unlock(&pp->blocked_lock);
+ }
+}
+#else
+static inline void activate_task_and_blocked_ent(struct rq *rq, struct task_struct *p, int en_flags)
+{
+ activate_task(rq, p, en_flags);
+}
+#endif
+
static void
ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
struct rq_flags *rf)
@@ -3708,7 +3783,8 @@ ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
atomic_dec(&task_rq(p)->nr_iowait);
}

- activate_task(rq, p, en_flags);
+ activate_task_and_blocked_ent(rq, p, en_flags);
+
check_preempt_curr(rq, p, wake_flags);

ttwu_do_wakeup(p);
@@ -3741,6 +3817,95 @@ ttwu_do_activate(struct rq *rq, struct task_struct *p, int wake_flags,
#endif
}

+#ifdef CONFIG_PROXY_EXEC
+/* XXX jstultz: This needs a better name! */
+bool ttwu_proxy_skip_wakeup(struct rq *rq, struct task_struct *p)
+{
+ /*
+ * XXX connoro: wrap this case with #ifdef CONFIG_PROXY_EXEC?
+ */
+ if (task_current(rq, p)) {
+ bool ret = true;
+ /*
+ * XXX connoro: p is currently running. 3 cases are possible:
+ * 1) p is blocked on a lock it owns, but we got the rq lock before
+ * it could schedule out. Kill blocked_on relation and call
+ * ttwu_do_wakeup
+ * 2) p is blocked on a lock it does not own. Leave blocked_on
+ * unchanged, don't call ttwu_do_wakeup, and return 0.
+ * 3) p is unblocked, but unless we hold onto blocked_lock while
+ * calling ttwu_do_wakeup, we could race with it becoming
+ * blocked and overwrite the correct p->__state with TASK_RUNNING.
+ */
+ raw_spin_lock(&p->blocked_lock);
+ if (task_is_blocked(p) && mutex_owner(p->blocked_on) == p)
+ set_task_blocked_on(p, NULL);
+ if (!task_is_blocked(p))
+ ret = false;
+ raw_spin_unlock(&p->blocked_lock);
+ return ret;
+ }
+
+ /*
+ * Since we don't dequeue for blocked-on relations, we'll always
+ * trigger the on_rq_queued() clause for them.
+ */
+ if (task_is_blocked(p)) {
+ raw_spin_lock(&p->blocked_lock);
+
+ if (mutex_owner(p->blocked_on) != p) {
+ /*
+ * XXX connoro: p already woke, ran and blocked on
+ * another mutex. Since a successful wakeup already
+ * happened, we're done.
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ return true;
+ }
+
+ set_task_blocked_on(p, NULL);
+ if (!cpumask_test_cpu(cpu_of(rq), p->cpus_ptr)) {
+ /*
+ * proxy stuff moved us outside of the affinity mask
+ * 'sleep' now and fail the direct wakeup so that the
+ * normal wakeup path will fix things.
+ */
+ deactivate_task(rq, p, DEQUEUE_SLEEP | DEQUEUE_NOCLOCK);
+ if (task_current_proxy(rq, p)) {
+ /*
+ * XXX connoro: If p is the proxy, then remove lingering
+ * references to it from rq and sched_class structs after
+ * dequeueing.
+ * can we get here while rq is inside __schedule?
+ * do any assumptions break if so?
+ */
+ put_prev_task(rq, p);
+ rq_set_proxy(rq, rq->idle);
+ }
+ resched_curr(rq);
+ raw_spin_unlock(&p->blocked_lock);
+ return true;
+ }
+ /* connoro: perhaps deq/enq here to get our task into the pushable task
+ * list again now that it's unblocked? Does that break if we're the proxy or
+ * does holding the rq lock make that OK?
+ */
+ /*
+ * Must resched after killing a blocked_on relation. The currently
+ * executing context might not be the most elegible anymore.
+ */
+ resched_curr(rq);
+ raw_spin_unlock(&p->blocked_lock);
+ }
+ return false;
+}
+#else
+static inline bool ttwu_proxy_skip_wakeup(struct rq *rq, struct task_struct *p)
+{
+ return false;
+}
+#endif
+
/*
* Consider @p being inside a wait loop:
*
@@ -3773,9 +3938,15 @@ static int ttwu_runnable(struct task_struct *p, int wake_flags)
int ret = 0;

rq = __task_rq_lock(p, &rf);
- if (!task_on_rq_queued(p))
+ if (!task_on_rq_queued(p)) {
+ BUG_ON(task_is_running(p));
goto out_unlock;
+ }

+ /*
+ * ttwu_do_wakeup()->
+ * check_preempt_curr() may use rq clock
+ */
if (!task_on_cpu(rq, p)) {
/*
* When on_rq && !on_cpu the task is preempted, see if
@@ -3784,8 +3955,14 @@ static int ttwu_runnable(struct task_struct *p, int wake_flags)
update_rq_clock(rq);
check_preempt_curr(rq, p, wake_flags);
}
+
+ /* XXX jstultz: This needs a better name! */
+ if (ttwu_proxy_skip_wakeup(rq, p))
+ goto out_unlock;
+
ttwu_do_wakeup(p);
ret = 1;
+
out_unlock:
__task_rq_unlock(rq, &rf);

@@ -4193,6 +4370,23 @@ try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
if (READ_ONCE(p->on_rq) && ttwu_runnable(p, wake_flags))
goto unlock;

+ if (task_is_blocked(p)) {
+ /*
+ * XXX connoro: we are in one of 2 cases:
+ * 1) p is blocked on a mutex it doesn't own but is still
+ * enqueued on a rq. We definitely don't want to keep going
+ * (and potentially activate it elsewhere without ever
+ * dequeueing) but maybe this is more properly handled by
+ * having ttwu_runnable() return 1 in this case?
+ * 2) p was removed from its rq and added to a blocked_entry
+ * list by proxy(). It should not be woken until the task at
+ * the head of the list gets a mutex handoff wakeup.
+ * Should try_to_wake_up() return 1 in either of these cases?
+ */
+ success = 0;
+ goto unlock;
+ }
+
#ifdef CONFIG_SMP
/*
* Ensure we load p->on_cpu _after_ p->on_rq, otherwise it would be
@@ -5581,6 +5775,18 @@ void scheduler_tick(void)

rq_lock(rq, &rf);

+#ifdef CONFIG_PROXY_EXEC
+ /*
+ * XXX connoro: is this check needed? Why?
+ */
+ if (task_cpu(curr) != cpu) {
+ BUG_ON(!test_preempt_need_resched() &&
+ !tif_need_resched());
+ rq_unlock(rq, &rf);
+ return;
+ }
+#endif
+
update_rq_clock(rq);
thermal_pressure = arch_scale_thermal_pressure(cpu_of(rq));
update_thermal_load_avg(rq_clock_thermal(rq), rq, thermal_pressure);
@@ -6473,6 +6679,397 @@ pick_next_task(struct rq *rq, struct task_struct *prev, struct rq_flags *rf)
# define SM_MASK_PREEMPT SM_PREEMPT
#endif

+#ifdef CONFIG_PROXY_EXEC
+
+static struct task_struct *
+proxy_migrate_task(struct rq *rq, struct task_struct *next,
+ struct rq_flags *rf, struct task_struct *p,
+ int that_cpu, bool curr_in_chain)
+{
+ struct rq *that_rq;
+ LIST_HEAD(migrate_list);
+
+ /*
+ * The blocked-on relation must not cross CPUs, if this happens
+ * migrate @p to the @owner's CPU.
+ *
+ * This is because we must respect the CPU affinity of execution
+ * contexts (@owner) but we can ignore affinity for scheduling
+ * contexts (@p). So we have to move scheduling contexts towards
+ * potential execution contexts.
+ *
+ * XXX [juril] what if @p is not the highest prio task once migrated
+ * to @owner's CPU?
+ *
+ * XXX [juril] also, after @p is migrated it is not migrated back once
+ * @owner releases the lock? Isn't this a potential problem w.r.t.
+ * @owner affinity settings?
+ * [juril] OK. It is migrated back into its affinity mask in
+ * ttwu_remote(), or by using wake_cpu via select_task_rq, guess we
+ * might want to add a comment about that here. :-)
+ *
+ * TODO: could optimize by finding the CPU of the final owner
+ * and migrating things there. Given:
+ *
+ * CPU0 CPU1 CPU2
+ *
+ * a ----> b ----> c
+ *
+ * the current scheme would result in migrating 'a' to CPU1,
+ * then CPU1 would migrate b and a to CPU2. Only then would
+ * CPU2 run c.
+ */
+ that_rq = cpu_rq(that_cpu);
+
+ /*
+ * @owner can disappear, simply migrate to @that_cpu and leave that CPU
+ * to sort things out.
+ */
+
+ /*
+ * Since we're going to drop @rq, we have to put(@next) first,
+ * otherwise we have a reference that no longer belongs to us. Use
+ * @fake_task to fill the void and make the next pick_next_task()
+ * invocation happy.
+ *
+ * XXX double, triple think about this.
+ * XXX put doesn't work with ON_RQ_MIGRATE
+ *
+ * CPU0 CPU1
+ *
+ * B mutex_lock(X)
+ *
+ * A mutex_lock(X) <- B
+ * A __schedule()
+ * A pick->A
+ * A proxy->B
+ * A migrate A to CPU1
+ * B mutex_unlock(X) -> A
+ * B __schedule()
+ * B pick->A
+ * B switch_to (A)
+ * A ... does stuff
+ * A ... is still running here
+ *
+ * * BOOM *
+ */
+ put_prev_task(rq, next);
+ if (curr_in_chain) {
+ rq_set_proxy(rq, rq->idle);
+ set_tsk_need_resched(rq->idle);
+ /*
+ * XXX [juril] don't we still need to migrate @next to
+ * @owner's CPU?
+ */
+ return rq->idle;
+ }
+ rq_set_proxy(rq, rq->idle);
+
+ for (; p; p = p->blocked_proxy) {
+ int wake_cpu = p->wake_cpu;
+
+ WARN_ON(p == rq_curr(rq));
+
+ deactivate_task(rq, p, 0);
+ set_task_cpu(p, that_cpu);
+ /*
+ * We can abuse blocked_entry to migrate the thing, because @p is
+ * still on the rq.
+ */
+ list_add(&p->blocked_entry, &migrate_list);
+
+ /*
+ * Preserve p->wake_cpu, such that we can tell where it
+ * used to run later.
+ */
+ p->wake_cpu = wake_cpu;
+ }
+
+ rq_unpin_lock(rq, rf);
+ raw_spin_rq_unlock(rq);
+ raw_spin_rq_lock(that_rq);
+
+ while (!list_empty(&migrate_list)) {
+ p = list_first_entry(&migrate_list, struct task_struct, blocked_entry);
+ list_del_init(&p->blocked_entry);
+
+ enqueue_task(that_rq, p, 0);
+ check_preempt_curr(that_rq, p, 0);
+ p->on_rq = TASK_ON_RQ_QUEUED;
+ /*
+ * check_preempt_curr has already called
+ * resched_curr(that_rq) in case it is
+ * needed.
+ */
+ }
+
+ raw_spin_rq_unlock(that_rq);
+ raw_spin_rq_lock(rq);
+ rq_repin_lock(rq, rf);
+
+ return NULL; /* Retry task selection on _this_ CPU. */
+}
+
+static inline struct task_struct *
+proxy_resched_idle(struct rq *rq, struct task_struct *next)
+{
+ put_prev_task(rq, next);
+ rq_set_proxy(rq, rq->idle);
+ set_tsk_need_resched(rq->idle);
+ return rq->idle;
+}
+
+static void proxy_enqueue_on_owner(struct rq *rq, struct task_struct *p,
+ struct task_struct *owner,
+ struct task_struct *next)
+{
+ /*
+ * Walk back up the blocked_proxy relation and enqueue them all on @owner
+ *
+ * ttwu_activate() will pick them up and place them on whatever rq
+ * @owner will run next.
+ * XXX connoro: originally we would jump back into the main proxy() loop
+ * owner->on_rq !=0 path, but if we then end up taking the owned_task path
+ * then we can overwrite p->on_rq after ttwu_do_activate sets it to 1 which breaks
+ * the assumptions made in ttwu_do_activate.
+ *
+ * Perhaps revisit whether retry is now possible given the changes to the
+ * owned_task path since I wrote the prior comment...
+ */
+ if (!owner->on_rq) {
+ /* jstultz: Err, do we need to hold a lock on p? (we gave it up for owner) */
+ for (; p; p = p->blocked_proxy) {
+ if (p == owner)
+ continue;
+ BUG_ON(!p->on_rq);
+ deactivate_task(rq, p, DEQUEUE_SLEEP);
+ if (task_current_proxy(rq, p)) {
+ put_prev_task(rq, next);
+ rq_set_proxy(rq, rq->idle);
+ }
+ /*
+ * XXX connoro: need to verify this is necessary. The rationale is that
+ * ttwu_do_activate must not have a chance to activate p elsewhere before
+ * it's fully extricated from its old rq.
+ */
+ smp_mb();
+ list_add(&p->blocked_entry, &owner->blocked_entry);
+ }
+ }
+}
+
+/*
+ * Find who @next (currently blocked on a mutex) can proxy for.
+ *
+ * Follow the blocked-on relation:
+ *
+ * ,-> task
+ * | | blocked-on
+ * | v
+ * proxied-by | mutex
+ * | | owner
+ * | v
+ * `-- task
+ *
+ * and set the proxied-by relation, this latter is used by the mutex code
+ * to find which (blocked) task to hand-off to.
+ *
+ * Lock order:
+ *
+ * p->pi_lock
+ * rq->lock
+ * mutex->wait_lock
+ * p->blocked_lock
+ *
+ * Returns the task that is going to be used as execution context (the one
+ * that is actually going to be put to run on cpu_of(rq)).
+ */
+static struct task_struct *
+proxy(struct rq *rq, struct task_struct *next, struct rq_flags *rf)
+{
+ struct task_struct *p = next;
+ struct task_struct *owner = NULL;
+ bool curr_in_chain = false;
+ int this_cpu, that_cpu;
+ struct mutex *mutex;
+
+ this_cpu = cpu_of(rq);
+
+ /*
+ * Follow blocked_on chain.
+ *
+ * TODO: deadlock detection
+ */
+ for (p = next; p->blocked_on; p = owner) {
+ mutex = p->blocked_on;
+ /* Something changed in the chain, pick_again */
+ if (!mutex)
+ return NULL;
+
+ /*
+ * By taking mutex->wait_lock we hold off concurrent mutex_unlock()
+ * and ensure @owner sticks around.
+ */
+ raw_spin_lock(&mutex->wait_lock);
+ raw_spin_lock(&p->blocked_lock);
+
+ /* Check again that p is blocked with blocked_lock held */
+ if (!task_is_blocked(p) || mutex != p->blocked_on) {
+ /*
+ * Something changed in the blocked_on chain and
+ * we don't know if only at this level. So, let's
+ * just bail out completely and let __schedule
+ * figure things out (pick_again loop).
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+ return NULL;
+ }
+
+ if (task_current(rq, p))
+ curr_in_chain = true;
+
+ owner = mutex_owner(mutex);
+ if (task_cpu(owner) != this_cpu) {
+ that_cpu = task_cpu(owner);
+ /*
+ * @owner can disappear, simply migrate to @that_cpu and leave that CPU
+ * to sort things out.
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+
+ return proxy_migrate_task(rq, next, rf, p, that_cpu, curr_in_chain);
+ }
+
+ if (task_on_rq_migrating(owner)) {
+ /*
+ * XXX connoro: one of the chain of mutex owners is currently
+ * migrating to this CPU, but has not yet been enqueued because
+ * we are holding the rq lock. As a simple solution, just schedule
+ * rq->idle to give the migration a chance to complete. Much like
+ * the migrate_task case we should end up back in proxy(), this
+ * time hopefully with all relevant tasks already enqueued.
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+ return proxy_resched_idle(rq, next);
+ }
+
+ if (!owner->on_rq) {
+ /*
+ * XXX connoro: rq->curr must not be added to the blocked_entry list
+ * or else ttwu_do_activate could enqueue it elsewhere before it
+ * switches out here. The approach to avoiding this is the same as in
+ * the migrate_task case.
+ */
+ if (curr_in_chain) {
+ /*
+ * This is identical to the owned_task handling, probably should
+ * fold them together...
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+ return proxy_resched_idle(rq, next);
+ }
+
+ /*
+ * If !@owner->on_rq, holding @rq->lock will not pin the task,
+ * so we cannot drop @mutex->wait_lock until we're sure its a blocked
+ * task on this rq.
+ *
+ * We use @owner->blocked_lock to serialize against ttwu_activate().
+ * Either we see its new owner->on_rq or it will see our list_add().
+ */
+ if (owner != p) {
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_lock(&owner->blocked_lock);
+ }
+
+ proxy_enqueue_on_owner(rq, p, owner, next);
+
+ if (task_current_proxy(rq, next)) {
+ put_prev_task(rq, next);
+ rq_set_proxy(rq, rq->idle);
+ }
+ raw_spin_unlock(&owner->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+
+ return NULL; /* retry task selection */
+ }
+
+ if (owner == p) {
+ /*
+ * Its possible we interleave with mutex_unlock like:
+ *
+ * lock(&rq->lock);
+ * proxy()
+ * mutex_unlock()
+ * lock(&wait_lock);
+ * next(owner) = current->blocked_proxy;
+ * unlock(&wait_lock);
+ *
+ * wake_up_q();
+ * ...
+ * ttwu_runnable()
+ * __task_rq_lock()
+ * lock(&wait_lock);
+ * owner == p
+ *
+ * Which leaves us to finish the ttwu_runnable() and make it go.
+ *
+ * XXX is this happening in case of an HANDOFF to p?
+ * In any case, reading of the owner in __mutex_unlock_slowpath is
+ * done atomically outside wait_lock (only adding waiters to wake_q is
+ * done inside the critical section).
+ * Does this means we can get to proxy _w/o an owner_ if that was
+ * cleared before grabbing wait_lock? Do we account for this case?
+ * OK we actually do (see PROXY_EXEC ifdeffery in unlock function).
+ */
+
+ /*
+ * XXX connoro: prior versions would clear p->blocked_on here, but I think
+ * that can race with the handoff wakeup path. If a wakeup reaches the
+ * call to ttwu_runnable after this point and finds that p is enqueued
+ * and marked as unblocked, it will happily do a ttwu_do_wakeup() call
+ * with zero regard for whether the task's affinity actually allows
+ * running it on this CPU.
+ */
+
+ /*
+ * XXX connoro: previous versions would immediately run owner here if
+ * it's allowed to run on this CPU, but this creates potential races
+ * with the wakeup logic. Instead we can just take the blocked_task path
+ * when owner is already !on_rq, or else schedule rq->idle so that
+ * ttwu_runnable can get the rq lock and mark owner as running.
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+ return proxy_resched_idle(rq, next);
+ }
+
+ /*
+ * OK, now we're absolutely sure @owner is not blocked _and_
+ * on this rq, therefore holding @rq->lock is sufficient to
+ * guarantee its existence, as per ttwu_remote().
+ */
+ raw_spin_unlock(&p->blocked_lock);
+ raw_spin_unlock(&mutex->wait_lock);
+
+ owner->blocked_proxy = p;
+ }
+
+ WARN_ON_ONCE(!owner->on_rq);
+ return owner;
+}
+#else /* PROXY_EXEC */
+static struct task_struct *
+proxy(struct rq *rq, struct task_struct *next, struct rq_flags *rf)
+{
+ return next;
+}
+#endif /* PROXY_EXEC */
+
/*
* __schedule() is the main scheduler function.
*
@@ -6520,6 +7117,7 @@ static void __sched notrace __schedule(unsigned int sched_mode)
struct rq_flags rf;
struct rq *rq;
int cpu;
+ bool preserve_need_resched = false;

cpu = smp_processor_id();
rq = cpu_rq(cpu);
@@ -6565,7 +7163,7 @@ static void __sched notrace __schedule(unsigned int sched_mode)
if (!(sched_mode & SM_MASK_PREEMPT) && prev_state) {
if (signal_pending_state(prev_state, prev)) {
WRITE_ONCE(prev->__state, TASK_RUNNING);
- } else {
+ } else if (!task_is_blocked(prev)) {
prev->sched_contributes_to_load =
(prev_state & TASK_UNINTERRUPTIBLE) &&
!(prev_state & TASK_NOLOAD) &&
@@ -6591,13 +7189,49 @@ static void __sched notrace __schedule(unsigned int sched_mode)
atomic_inc(&rq->nr_iowait);
delayacct_blkio_start();
}
+ } else {
+ /*
+ * XXX
+ * Let's make this task, which is blocked on
+ * a mutex, (push/pull)able (RT/DL).
+ * Unfortunately we can only deal with that by
+ * means of a dequeue/enqueue cycle. :-/
+ */
+ dequeue_task(rq, prev, 0);
+ enqueue_task(rq, prev, 0);
}
switch_count = &prev->nvcsw;
}

- next = pick_next_task(rq, prev, &rf);
+pick_again:
+ /*
+ * If picked task is actually blocked it means that it can act as a
+ * proxy for the task that is holding the mutex picked task is blocked
+ * on. Get a reference to the blocked (going to be proxy) task here.
+ * Note that if next isn't actually blocked we will have rq->proxy ==
+ * rq->curr == next in the end, which is intended and means that proxy
+ * execution is currently "not in use".
+ */
+ next = pick_next_task(rq, rq_proxy(rq), &rf);
rq_set_proxy(rq, next);
- clear_tsk_need_resched(prev);
+ next->blocked_proxy = NULL;
+ if (unlikely(task_is_blocked(next))) {
+ next = proxy(rq, next, &rf);
+ if (!next)
+ goto pick_again;
+ /*
+ * XXX connoro: when proxy() returns rq->idle it sets the
+ * TIF_NEED_RESCHED flag, but in the case where
+ * rq->idle == rq->prev, the flag would be cleared immediately,
+ * defeating the desired behavior. So, check explicitly for
+ * this case.
+ */
+ if (next == rq->idle && prev == rq->idle)
+ preserve_need_resched = true;
+ }
+
+ if (!preserve_need_resched)
+ clear_tsk_need_resched(prev);
clear_preempt_need_resched();
#ifdef CONFIG_SCHED_DEBUG
rq->last_seen_need_resched_ns = 0;
@@ -6684,6 +7318,10 @@ static inline void sched_submit_work(struct task_struct *tsk)
*/
SCHED_WARN_ON(current->__state & TASK_RTLOCK_WAIT);

+ /* XXX still necessary? tsk_is_pi_blocked check here was deleted... */
+ if (task_is_blocked(tsk))
+ return;
+
/*
* If we are going to sleep and we have plugged IO queued,
* make sure to submit it to avoid deadlocks.
diff --git a/kernel/sched/deadline.c b/kernel/sched/deadline.c
index 6ec40f90317c..c9b6a23a99b3 100644
--- a/kernel/sched/deadline.c
+++ b/kernel/sched/deadline.c
@@ -1740,7 +1740,7 @@ static void enqueue_task_dl(struct rq *rq, struct task_struct *p, int flags)

enqueue_dl_entity(&p->dl, flags);

- if (!task_current(rq, p) && p->nr_cpus_allowed > 1)
+ if (!task_current(rq, p) && p->nr_cpus_allowed > 1 && !task_is_blocked(p))
enqueue_pushable_dl_task(rq, p);
}

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 1471519b95c5..aa8d772efadf 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -7929,7 +7929,9 @@ pick_next_task_fair(struct rq *rq, struct task_struct *prev, struct rq_flags *rf
goto idle;

#ifdef CONFIG_FAIR_GROUP_SCHED
- if (!prev || prev->sched_class != &fair_sched_class)
+ if (!prev ||
+ prev->sched_class != &fair_sched_class ||
+ rq_curr(rq) != rq_proxy(rq))
goto simple;

/*
@@ -8447,6 +8449,9 @@ int can_migrate_task(struct task_struct *p, struct lb_env *env)

lockdep_assert_rq_held(env->src_rq);

+ if (task_is_blocked(p))
+ return 0;
+
/*
* We do not migrate tasks that are:
* 1) throttled_lb_pair, or
@@ -8497,7 +8502,11 @@ int can_migrate_task(struct task_struct *p, struct lb_env *env)
/* Record that we found at least one task that could run on dst_cpu */
env->flags &= ~LBF_ALL_PINNED;

- if (task_on_cpu(env->src_rq, p)) {
+ /*
+ * XXX mutex unlock path may have marked proxy as unblocked allowing us to
+ * reach this point, but we still shouldn't migrate it.
+ */
+ if (task_on_cpu(env->src_rq, p) || task_current_proxy(env->src_rq, p)) {
schedstat_inc(p->stats.nr_failed_migrations_running);
return 0;
}
diff --git a/kernel/sched/rt.c b/kernel/sched/rt.c
index 03e5d8fa67aa..d1c5a022eae4 100644
--- a/kernel/sched/rt.c
+++ b/kernel/sched/rt.c
@@ -1537,7 +1537,8 @@ enqueue_task_rt(struct rq *rq, struct task_struct *p, int flags)

enqueue_rt_entity(rt_se, flags);

- if (!task_current(rq, p) && p->nr_cpus_allowed > 1)
+ if (!task_current(rq, p) && p->nr_cpus_allowed > 1 &&
+ !task_is_blocked(p))
enqueue_pushable_task(rq, p);
}

diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 84e49c2530b0..01f82ace084a 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -2159,6 +2159,19 @@ static inline int task_current_proxy(struct rq *rq, struct task_struct *p)
return rq_proxy(rq) == p;
}

+#ifdef CONFIG_PROXY_EXEC
+static inline bool task_is_blocked(struct task_struct *p)
+{
+ return !!p->blocked_on;
+}
+#else /* !PROXY_EXEC */
+static inline bool task_is_blocked(struct task_struct *p)
+{
+ return false;
+}
+
+#endif /* PROXY_EXEC */
+
static inline int task_on_cpu(struct rq *rq, struct task_struct *p)
{
#ifdef CONFIG_SMP
@@ -2316,12 +2329,18 @@ struct sched_class {

static inline void put_prev_task(struct rq *rq, struct task_struct *prev)
{
- WARN_ON_ONCE(rq_proxy(rq) != prev);
+ WARN_ON_ONCE(rq_curr(rq) != prev && prev != rq_proxy(rq));
+
+ /* XXX connoro: is this check necessary? */
+ if (prev == rq_proxy(rq) && task_cpu(prev) != cpu_of(rq))
+ return;
+
prev->sched_class->put_prev_task(rq, prev);
}

static inline void set_next_task(struct rq *rq, struct task_struct *next)
{
+ WARN_ON_ONCE(!task_current_proxy(rq, next));
next->sched_class->set_next_task(rq, next, false);
}

--
2.40.0.rc1.284.g88254d51c5-goog