@@ -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
@@ -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
@@ -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
@@ -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;
@@ -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(¤t->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(¤t->blocked_lock);
+#endif
raw_spin_unlock(&lock->wait_lock);
wake_up_q(&wake_q);
@@ -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.
@@ -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);
}
@@ -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;
}
@@ -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);
}
@@ -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);
}