2011-02-01 14:54:21

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 0/7] directed yield for Pause Loop Exiting

When running SMP virtual machines, it is possible for one VCPU to be
spinning on a spinlock, while the VCPU that holds the spinlock is not
currently running, because the host scheduler preempted it to run
something else.

Both Intel and AMD CPUs have a feature that detects when a virtual
CPU is spinning on a lock and will trap to the host.

The current KVM code sleeps for a bit whenever that happens, which
results in eg. a 64 VCPU Windows guest taking forever and a bit to
boot up. This is because the VCPU holding the lock is actually
running and not sleeping, so the pause is counter-productive.

In other workloads a pause can also be counter-productive, with
spinlock detection resulting in one guest giving up its CPU time
to the others. Instead of spinning, it ends up simply not running
much at all.

This patch series aims to fix that, by having a VCPU that spins
give the remainder of its timeslice to another VCPU in the same
guest before yielding the CPU - one that is runnable but got
preempted, hopefully the lock holder.

v8:
- some more changes and cleanups suggested by Peter
v7:
- move the vcpu to pid mapping to inside the vcpu->mutex
- rename ->yield to ->skip
- merge patch 5 into patch 4
v6:
- implement yield_task_fair in a way that works with task groups,
this allows me to actually get a performance improvement!
- fix another race Avi pointed out, the code should be good now
v5:
- fix the race condition Avi pointed out, by tracking vcpu->pid
- also allows us to yield to vcpu tasks that got preempted while in qemu
userspace
v4:
- change to newer version of Mike Galbraith's yield_to implementation
- chainsaw out some code from Mike that looked like a great idea, but
turned out to give weird interactions in practice
v3:
- more cleanups
- change to Mike Galbraith's yield_to implementation
- yield to spinning VCPUs, this seems to work better in some
situations and has little downside potential
v2:
- make lots of cleanups and improvements suggested
- do not implement timeslice scheduling or fairness stuff
yet, since it is not entirely clear how to do that right
(suggestions welcome)


Benchmark results:

Two 4-CPU KVM guests are pinned to the same 4 physical CPUs.

One guest runs the AMQP performance test, the other guest runs
0, 2 or 4 infinite loops, for CPU overcommit factors of 0, 1.5
and 4.

The AMQP perftest is run 30 times, with message payloads of 8 and 16 bytes.

size8 no overcommit 1.5x overcommit 2x overcommit

no PLE 223801 135137 104951
PLE 224135 141105 118744

size16 no overcommit 1.5x overcommit 2x overcommit

no PLE 222424 126175 105299
PLE 222534 138082 132945

Note: this is with the KVM guests NOT running inside cgroups. There
seems to be a CPU load balancing issue with cgroup fair group scheduling,
which often results in one guest getting only 80% CPU time and the other
guest 320%. That will have to be fixed to get meaningful results with
cgroups.

CPU time division between the AMQP guest and the infinite loop guest
were not exactly fair, but the guests got close to the same amount
of CPU time in each test run.

There is a substantial amount of randomness in CPU time division between
guests, but the performance improvement is consistent between multiple
runs.

--
All rights reversed.


2011-02-01 14:54:07

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 1/7] sched: check the right ->nr_running in yield_task_fair

With CONFIG_FAIR_GROUP_SCHED, each task_group has its own cfs_rq.
Yielding to a task from another cfs_rq may be worthwhile, since
a process calling yield typically cannot use the CPU right now.

Therefor, we want to check the per-cpu nr_running, not the
cgroup local one.

Signed-off-by: Rik van Riel <[email protected]>
---
kernel/sched_fair.c | 2 +-
1 files changed, 1 insertions(+), 1 deletions(-)

diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index c62ebae..7b338ac 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -1304,7 +1304,7 @@ static void yield_task_fair(struct rq *rq)
/*
* Are we the only task in the tree?
*/
- if (unlikely(cfs_rq->nr_running == 1))
+ if (unlikely(rq->nr_running == 1))
return;

clear_buddies(cfs_rq, se);

2011-02-01 14:54:15

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 5/7] export pid symbols needed for kvm_vcpu_on_spin

Export the symbols required for a race-free kvm_vcpu_on_spin.

Signed-off-by: Rik van Riel <[email protected]>

diff --git a/kernel/fork.c b/kernel/fork.c
index 3b159c5..adc8f47 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -191,6 +191,7 @@ void __put_task_struct(struct task_struct *tsk)
if (!profile_handoff_task(tsk))
free_task(tsk);
}
+EXPORT_SYMBOL_GPL(__put_task_struct);

/*
* macro override instead of weak attribute alias, to workaround
diff --git a/kernel/pid.c b/kernel/pid.c
index 39b65b6..02f2212 100644
--- a/kernel/pid.c
+++ b/kernel/pid.c
@@ -435,6 +435,7 @@ struct pid *get_task_pid(struct task_struct *task, enum pid_type type)
rcu_read_unlock();
return pid;
}
+EXPORT_SYMBOL_GPL(get_task_pid);

struct task_struct *get_pid_task(struct pid *pid, enum pid_type type)
{
@@ -446,6 +447,7 @@ struct task_struct *get_pid_task(struct pid *pid, enum pid_type type)
rcu_read_unlock();
return result;
}
+EXPORT_SYMBOL_GPL(get_pid_task);

struct pid *find_get_pid(pid_t nr)
{

2011-02-01 14:54:13

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 4/7] sched: Add yield_to(task, preempt) functionality

From: Mike Galbraith <[email protected]>

Currently only implemented for fair class tasks.

Add a yield_to_task method() to the fair scheduling class. allowing the
caller of yield_to() to accelerate another thread in it's thread group,
task group.

Implemented via a scheduler hint, using cfs_rq->next to encourage the
target being selected. We can rely on pick_next_entity to keep things
fair, so noone can accelerate a thread that has already used its fair
share of CPU time.

This also means callers should only call yield_to when they really
mean it. Calling it too often can result in the scheduler just
ignoring the hint.

Signed-off-by: Rik van Riel <[email protected]>
Signed-off-by: Marcelo Tosatti <[email protected]>
Signed-off-by: Mike Galbraith <[email protected]>

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 2c79e92..6c43fc4 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1047,6 +1047,7 @@ struct sched_class {
void (*enqueue_task) (struct rq *rq, struct task_struct *p, int flags);
void (*dequeue_task) (struct rq *rq, struct task_struct *p, int flags);
void (*yield_task) (struct rq *rq);
+ bool (*yield_to_task) (struct rq *rq, struct task_struct *p, bool preempt);

void (*check_preempt_curr) (struct rq *rq, struct task_struct *p, int flags);

@@ -1943,6 +1944,7 @@ static inline int rt_mutex_getprio(struct task_struct *p)
# define rt_mutex_adjust_pi(p) do { } while (0)
#endif

+extern bool yield_to(struct task_struct *p, bool preempt);
extern void set_user_nice(struct task_struct *p, long nice);
extern int task_prio(const struct task_struct *p);
extern int task_nice(const struct task_struct *p);
diff --git a/kernel/sched.c b/kernel/sched.c
index 7ff53e2..bcc1f65 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -5270,6 +5270,61 @@ void __sched yield(void)
}
EXPORT_SYMBOL(yield);

+/**
+ * yield_to - yield the current processor to another thread in
+ * your thread group, or accelerate that thread toward the
+ * processor it's on.
+ *
+ * It's the caller's job to ensure that the target task struct
+ * can't go away on us before we can do any checks.
+ *
+ * Returns true if we indeed boosted the target task.
+ */
+bool __sched yield_to(struct task_struct *p, bool preempt)
+{
+ struct task_struct *curr = current;
+ struct rq *rq, *p_rq;
+ unsigned long flags;
+ bool yielded = 0;
+
+ local_irq_save(flags);
+ rq = this_rq();
+
+again:
+ p_rq = task_rq(p);
+ double_rq_lock(rq, p_rq);
+ while (task_rq(p) != p_rq) {
+ double_rq_unlock(rq, p_rq);
+ goto again;
+ }
+
+ if (!curr->sched_class->yield_to_task)
+ goto out;
+
+ if (curr->sched_class != p->sched_class)
+ goto out;
+
+ if (task_running(p_rq, p) || p->state)
+ goto out;
+
+ yielded = curr->sched_class->yield_to_task(rq, p, preempt);
+
+ if (yielded) {
+ schedstat_inc(rq, yld_count);
+ current->sched_class->yield_task(rq);
+ }
+
+out:
+ double_rq_unlock(rq, p_rq);
+ local_irq_restore(flags);
+
+ if (yielded)
+ schedule();
+
+ return yielded;
+}
+EXPORT_SYMBOL_GPL(yield_to);
+
/*
* This task is about to go to sleep on IO. Increment rq->nr_iowait so
* that process accounting knows that this is a task in IO wait state.
diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index bd32d2a..728d3e8 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -1794,6 +1794,23 @@ static void yield_task_fair(struct rq *rq)
set_skip_buddy(se);
}

+static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preempt)
+{
+ struct sched_entity *se = &p->se;
+
+ if (!se->on_rq)
+ return false;
+
+ /* Tell the scheduler that we'd really like pse to run next. */
+ set_next_buddy(se);
+
+ /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
+ if (preempt)
+ resched_task(rq->curr);
+
+ return true;
+}
+
#ifdef CONFIG_SMP
/**************************************************
* Fair scheduling class load-balancing methods:
@@ -3987,6 +4004,7 @@ static const struct sched_class fair_sched_class = {
.enqueue_task = enqueue_task_fair,
.dequeue_task = dequeue_task_fair,
.yield_task = yield_task_fair,
+ .yield_to_task = yield_to_task_fair,

.check_preempt_curr = check_preempt_wakeup,

2011-02-01 14:54:34

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 2/7] sched: limit the scope of clear_buddies

The clear_buddies function does not seem to play well with the concept
of hierarchical runqueues. In the following tree, task groups are
represented by 'G', tasks by 'T', next by 'n' and last by 'l'.

(nl)
/ \
G(nl) G
/ \ \
T(l) T(n) T

This situation can arise when a task is woken up T(n), and the previously
running task T(l) is marked last.

When clear_buddies is called from either T(l) or T(n), the next and last
buddies of the group G(nl) will be cleared. This is not the desired
result, since we would like to be able to find the other type of buddy
in many cases.

This especially a worry when implementing yield_task_fair through the
buddy system.

The fix is simple: only clear the buddy type that the task itself
is indicated to be. As an added bonus, we stop walking up the tree
when the buddy has already been cleared or pointed elsewhere.

Signed-off-by: Rik van Riel <[email protected]>
---
kernel/sched_fair.c | 30 +++++++++++++++++++++++-------
1 files changed, 23 insertions(+), 7 deletions(-)

diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index f4ee445..0321473 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -784,19 +784,35 @@ enqueue_entity(struct cfs_rq *cfs_rq, struct sched_entity *se, int flags)
__enqueue_entity(cfs_rq, se);
}

-static void __clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)
+static void __clear_buddies_last(struct sched_entity *se)
{
- if (!se || cfs_rq->last == se)
- cfs_rq->last = NULL;
+ for_each_sched_entity(se) {
+ struct cfs_rq *cfs_rq = cfs_rq_of(se);
+ if (cfs_rq->last == se)
+ cfs_rq->last = NULL;
+ else
+ break;
+ }
+}

- if (!se || cfs_rq->next == se)
- cfs_rq->next = NULL;
+static void __clear_buddies_next(struct sched_entity *se)
+{
+ for_each_sched_entity(se) {
+ struct cfs_rq *cfs_rq = cfs_rq_of(se);
+ if (cfs_rq->next == se)
+ cfs_rq->next = NULL;
+ else
+ break;
+ }
}

static void clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)
{
- for_each_sched_entity(se)
- __clear_buddies(cfs_rq_of(se), se);
+ if (cfs_rq->last == se)
+ __clear_buddies_last(se);
+
+ if (cfs_rq->next == se)
+ __clear_buddies_next(se);
}

static void

2011-02-01 14:55:12

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 6/7] kvm: keep track of which task is running a KVM vcpu

Keep track of which task is running a KVM vcpu. This helps us
figure out later what task to wake up if we want to boost a
vcpu that got preempted.

Unfortunately there are no guarantees that the same task
always keeps the same vcpu, so we can only track the task
across a single "run" of the vcpu.

Signed-off-by: Rik van Riel <[email protected]>

diff --git a/include/linux/kvm_host.h b/include/linux/kvm_host.h
index a055742..9d56ed5 100644
--- a/include/linux/kvm_host.h
+++ b/include/linux/kvm_host.h
@@ -81,6 +81,7 @@ struct kvm_vcpu {
#endif
int vcpu_id;
struct mutex mutex;
+ struct pid *pid;
int cpu;
atomic_t guest_mode;
struct kvm_run *run;
diff --git a/virt/kvm/kvm_main.c b/virt/kvm/kvm_main.c
index 5225052..0fa9a48 100644
--- a/virt/kvm/kvm_main.c
+++ b/virt/kvm/kvm_main.c
@@ -117,6 +117,14 @@ void vcpu_load(struct kvm_vcpu *vcpu)
int cpu;

mutex_lock(&vcpu->mutex);
+ if (unlikely(vcpu->pid != current->pids[PIDTYPE_PID].pid)) {
+ /* The thread running this VCPU changed. */
+ struct pid *oldpid = vcpu->pid;
+ struct pid *newpid = get_task_pid(current, PIDTYPE_PID);
+ rcu_assign_pointer(vcpu->pid, newpid);
+ synchronize_rcu();
+ put_pid(oldpid);
+ }
cpu = get_cpu();
preempt_notifier_register(&vcpu->preempt_notifier);
kvm_arch_vcpu_load(vcpu, cpu);
@@ -185,6 +193,7 @@ int kvm_vcpu_init(struct kvm_vcpu *vcpu, struct kvm *kvm, unsigned id)
vcpu->cpu = -1;
vcpu->kvm = kvm;
vcpu->vcpu_id = id;
+ vcpu->pid = NULL;
init_waitqueue_head(&vcpu->wq);

page = alloc_page(GFP_KERNEL | __GFP_ZERO);
@@ -208,6 +217,7 @@ EXPORT_SYMBOL_GPL(kvm_vcpu_init);

void kvm_vcpu_uninit(struct kvm_vcpu *vcpu)
{
+ put_pid(vcpu->pid);
kvm_arch_vcpu_uninit(vcpu);
free_page((unsigned long)vcpu->run);
}

2011-02-01 14:55:15

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 3/7] sched: use a buddy to implement yield_task_fair

Use the buddy mechanism to implement yield_task_fair. This
allows us to skip onto the next highest priority se at every
level in the CFS tree, unless doing so would introduce gross
unfairness in CPU time distribution.

We order the buddy selection in pick_next_entity to check
yield first, then last, then next. We need next to be able
to override yield, because it is possible for the "next" and
"yield" task to be different processen in the same sub-tree
of the CFS tree. When they are, we need to go into that
sub-tree regardless of the "yield" hint, and pick the correct
entity once we get to the right level.

Signed-off-by: Rik van Riel <[email protected]>

diff --git a/kernel/sched.c b/kernel/sched.c
index dc91a4d..7ff53e2 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -327,7 +327,7 @@ struct cfs_rq {
* 'curr' points to currently running entity on this cfs_rq.
* It is set to NULL otherwise (i.e when none are currently running).
*/
- struct sched_entity *curr, *next, *last;
+ struct sched_entity *curr, *next, *last, *skip;

unsigned int nr_spread_over;

diff --git a/kernel/sched_debug.c b/kernel/sched_debug.c
index 2e1b0d1..f9a2721 100644
--- a/kernel/sched_debug.c
+++ b/kernel/sched_debug.c
@@ -183,7 +183,7 @@ void print_cfs_rq(struct seq_file *m, int cpu, struct cfs_rq *cfs_rq)

raw_spin_lock_irqsave(&rq->lock, flags);
if (cfs_rq->rb_leftmost)
- MIN_vruntime = (__pick_next_entity(cfs_rq))->vruntime;
+ MIN_vruntime = (__pick_first_entity(cfs_rq))->vruntime;
last = __pick_last_entity(cfs_rq);
if (last)
max_vruntime = last->vruntime;
diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index ad946fd..bd32d2a 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -374,7 +374,7 @@ static void __dequeue_entity(struct cfs_rq *cfs_rq, struct sched_entity *se)
rb_erase(&se->run_node, &cfs_rq->tasks_timeline);
}

-static struct sched_entity *__pick_next_entity(struct cfs_rq *cfs_rq)
+static struct sched_entity *__pick_first_entity(struct cfs_rq *cfs_rq)
{
struct rb_node *left = cfs_rq->rb_leftmost;

@@ -384,6 +384,16 @@ static struct sched_entity *__pick_next_entity(struct cfs_rq *cfs_rq)
return rb_entry(left, struct sched_entity, run_node);
}

+static struct sched_entity *__pick_next_entity(struct sched_entity *se)
+{
+ struct rb_node *next = rb_next(&se->run_node);
+
+ if (!next)
+ return NULL;
+
+ return rb_entry(next, struct sched_entity, run_node);
+}
+
static struct sched_entity *__pick_last_entity(struct cfs_rq *cfs_rq)
{
struct rb_node *last = rb_last(&cfs_rq->tasks_timeline);
@@ -806,6 +816,17 @@ static void __clear_buddies_next(struct sched_entity *se)
}
}

+static void __clear_buddies_skip(struct sched_entity *se)
+{
+ for_each_sched_entity(se) {
+ struct cfs_rq *cfs_rq = cfs_rq_of(se);
+ if (cfs_rq->skip == se)
+ cfs_rq->skip = NULL;
+ else
+ break;
+ }
+}
+
static void clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)
{
if (cfs_rq->last == se)
@@ -813,6 +834,9 @@ static void clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)

if (cfs_rq->next == se)
__clear_buddies_next(se);
+
+ if (cfs_rq->skip == se)
+ __clear_buddies_skip(se);
}

static void
@@ -885,7 +909,7 @@ check_preempt_tick(struct cfs_rq *cfs_rq, struct sched_entity *curr)
return;

if (cfs_rq->nr_running > 1) {
- struct sched_entity *se = __pick_next_entity(cfs_rq);
+ struct sched_entity *se = __pick_first_entity(cfs_rq);
s64 delta = curr->vruntime - se->vruntime;

if (delta > ideal_runtime)
@@ -926,13 +950,27 @@ set_next_entity(struct cfs_rq *cfs_rq, struct sched_entity *se)
static int
wakeup_preempt_entity(struct sched_entity *curr, struct sched_entity *se);

+/*
+ * Pick the next process, keeping these things in mind, in this order:
+ * 1) keep things fair between processes/task groups
+ * 2) pick the "next" process, since someone really wants that to run
+ * 3) pick the "last" process, for cache locality
+ * 4) do not run the "skip" process, if something else is available
+ */
static struct sched_entity *pick_next_entity(struct cfs_rq *cfs_rq)
{
- struct sched_entity *se = __pick_next_entity(cfs_rq);
+ struct sched_entity *se = __pick_first_entity(cfs_rq);
struct sched_entity *left = se;

- if (cfs_rq->next && wakeup_preempt_entity(cfs_rq->next, left) < 1)
- se = cfs_rq->next;
+ /*
+ * Avoid running the skip buddy, if running something else can
+ * be done without getting too unfair.
+ */
+ if (cfs_rq->skip == se) {
+ struct sched_entity *second = __pick_next_entity(se);
+ if (second && wakeup_preempt_entity(second, left) < 1)
+ se = second;
+ }

/*
* Prefer last buddy, try to return the CPU to a preempted task.
@@ -940,6 +978,12 @@ static struct sched_entity *pick_next_entity(struct cfs_rq *cfs_rq)
if (cfs_rq->last && wakeup_preempt_entity(cfs_rq->last, left) < 1)
se = cfs_rq->last;

+ /*
+ * Someone really wants this to run. If it's not unfair, run it.
+ */
+ if (cfs_rq->next && wakeup_preempt_entity(cfs_rq->next, left) < 1)
+ se = cfs_rq->next;
+
clear_buddies(cfs_rq, se);

return se;
@@ -1096,52 +1140,6 @@ static void dequeue_task_fair(struct rq *rq, struct task_struct *p, int flags)
hrtick_update(rq);
}

-/*
- * sched_yield() support is very simple - we dequeue and enqueue.
- *
- * If compat_yield is turned on then we requeue to the end of the tree.
- */
-static void yield_task_fair(struct rq *rq)
-{
- struct task_struct *curr = rq->curr;
- struct cfs_rq *cfs_rq = task_cfs_rq(curr);
- struct sched_entity *rightmost, *se = &curr->se;
-
- /*
- * Are we the only task in the tree?
- */
- if (unlikely(rq->nr_running == 1))
- return;
-
- clear_buddies(cfs_rq, se);
-
- if (likely(!sysctl_sched_compat_yield) && curr->policy != SCHED_BATCH) {
- update_rq_clock(rq);
- /*
- * Update run-time statistics of the 'current'.
- */
- update_curr(cfs_rq);
-
- return;
- }
- /*
- * Find the rightmost entry in the rbtree:
- */
- rightmost = __pick_last_entity(cfs_rq);
- /*
- * Already in the rightmost position?
- */
- if (unlikely(!rightmost || entity_before(rightmost, se)))
- return;
-
- /*
- * Minimally necessary key value to be last in the tree:
- * Upon rescheduling, sched_class::put_prev_task() will place
- * 'current' within the tree based on its new key value.
- */
- se->vruntime = rightmost->vruntime + 1;
-}
-
#ifdef CONFIG_SMP

static void task_waking_fair(struct rq *rq, struct task_struct *p)
@@ -1660,6 +1658,14 @@ static void set_next_buddy(struct sched_entity *se)
}
}

+static void set_skip_buddy(struct sched_entity *se)
+{
+ if (likely(task_of(se)->policy != SCHED_IDLE)) {
+ for_each_sched_entity(se)
+ cfs_rq_of(se)->skip = se;
+ }
+}
+
/*
* Preempt the current task with a newly woken task if needed:
*/
@@ -1758,6 +1764,36 @@ static void put_prev_task_fair(struct rq *rq, struct task_struct *prev)
}
}

+/*
+ * sched_yield() is very simple
+ *
+ * The magic of dealing with the ->skip buddy is in pick_next_entity.
+ */
+static void yield_task_fair(struct rq *rq)
+{
+ struct task_struct *curr = rq->curr;
+ struct cfs_rq *cfs_rq = task_cfs_rq(curr);
+ struct sched_entity *se = &curr->se;
+
+ /*
+ * Are we the only task in the tree?
+ */
+ if (unlikely(rq->nr_running == 1))
+ return;
+
+ clear_buddies(cfs_rq, se);
+
+ if (curr->policy != SCHED_BATCH) {
+ update_rq_clock(rq);
+ /*
+ * Update run-time statistics of the 'current'.
+ */
+ update_curr(cfs_rq);
+ }
+
+ set_skip_buddy(se);
+}
+
#ifdef CONFIG_SMP
/**************************************************
* Fair scheduling class load-balancing methods:
diff --git a/kernel/sysctl.c b/kernel/sysctl.c
index 5abfa15..6212b4e 100644
--- a/kernel/sysctl.c
+++ b/kernel/sysctl.c
@@ -375,13 +375,6 @@ static struct ctl_table kern_table[] = {
.mode = 0644,
.proc_handler = sched_rt_handler,
},
- {
- .procname = "sched_compat_yield",
- .data = &sysctl_sched_compat_yield,
- .maxlen = sizeof(unsigned int),
- .mode = 0644,
- .proc_handler = proc_dointvec,
- },
#ifdef CONFIG_PROVE_LOCKING
{
.procname = "prove_locking",

2011-02-01 14:55:14

by Rik van Riel

[permalink] [raw]
Subject: [PATCH -v8a 7/7] kvm: use yield_to instead of sleep in kvm_vcpu_on_spin

Instead of sleeping in kvm_vcpu_on_spin, which can cause gigantic
slowdowns of certain workloads, we instead use yield_to to get
another VCPU in the same KVM guest to run sooner.

This seems to give a 10-15% speedup in certain workloads.

Signed-off-by: Rik van Riel <[email protected]>
Signed-off-by: Marcelo Tosatti <[email protected]>

diff --git a/include/linux/kvm_host.h b/include/linux/kvm_host.h
index 9d56ed5..fab2250 100644
--- a/include/linux/kvm_host.h
+++ b/include/linux/kvm_host.h
@@ -187,6 +187,7 @@ struct kvm {
#endif
struct kvm_vcpu *vcpus[KVM_MAX_VCPUS];
atomic_t online_vcpus;
+ int last_boosted_vcpu;
struct list_head vm_list;
struct mutex lock;
struct kvm_io_bus *buses[KVM_NR_BUSES];
diff --git a/virt/kvm/kvm_main.c b/virt/kvm/kvm_main.c
index 86c4905..8b761ba 100644
--- a/virt/kvm/kvm_main.c
+++ b/virt/kvm/kvm_main.c
@@ -1292,18 +1292,55 @@ void kvm_resched(struct kvm_vcpu *vcpu)
}
EXPORT_SYMBOL_GPL(kvm_resched);

-void kvm_vcpu_on_spin(struct kvm_vcpu *vcpu)
+void kvm_vcpu_on_spin(struct kvm_vcpu *me)
{
- ktime_t expires;
- DEFINE_WAIT(wait);
-
- prepare_to_wait(&vcpu->wq, &wait, TASK_INTERRUPTIBLE);
-
- /* Sleep for 100 us, and hope lock-holder got scheduled */
- expires = ktime_add_ns(ktime_get(), 100000UL);
- schedule_hrtimeout(&expires, HRTIMER_MODE_ABS);
+ struct kvm *kvm = me->kvm;
+ struct kvm_vcpu *vcpu;
+ int last_boosted_vcpu = me->kvm->last_boosted_vcpu;
+ int yielded = 0;
+ int pass;
+ int i;

- finish_wait(&vcpu->wq, &wait);
+ /*
+ * We boost the priority of a VCPU that is runnable but not
+ * currently running, because it got preempted by something
+ * else and called schedule in __vcpu_run. Hopefully that
+ * VCPU is holding the lock that we need and will release it.
+ * We approximate round-robin by starting at the last boosted VCPU.
+ */
+ for (pass = 0; pass < 2 && !yielded; pass++) {
+ kvm_for_each_vcpu(i, vcpu, kvm) {
+ struct task_struct *task = NULL;
+ struct pid *pid;
+ if (!pass && i < last_boosted_vcpu) {
+ i = last_boosted_vcpu;
+ continue;
+ } else if (pass && i > last_boosted_vcpu)
+ break;
+ if (vcpu == me)
+ continue;
+ if (waitqueue_active(&vcpu->wq))
+ continue;
+ rcu_read_lock();
+ pid = rcu_dereference(vcpu->pid);
+ if (pid)
+ task = get_pid_task(vcpu->pid, PIDTYPE_PID);
+ rcu_read_unlock();
+ if (!task)
+ continue;
+ if (task->flags & PF_VCPU) {
+ put_task_struct(task);
+ continue;
+ }
+ if (yield_to(task, 1)) {
+ put_task_struct(task);
+ kvm->last_boosted_vcpu = i;
+ yielded = 1;
+ break;
+ }
+ put_task_struct(task);
+ }
+ }
}
EXPORT_SYMBOL_GPL(kvm_vcpu_on_spin);

2011-02-01 15:51:16

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH -v8a 4/7] sched: Add yield_to(task, preempt) functionality

On Tue, 2011-02-01 at 09:50 -0500, Rik van Riel wrote:

> +/**
> + * yield_to - yield the current processor to another thread in
> + * your thread group, or accelerate that thread toward the
> + * processor it's on.
> + *
> + * It's the caller's job to ensure that the target task struct
> + * can't go away on us before we can do any checks.
> + *
> + * Returns true if we indeed boosted the target task.
> + */
> +bool __sched yield_to(struct task_struct *p, bool preempt)
> +{
> + struct task_struct *curr = current;
> + struct rq *rq, *p_rq;
> + unsigned long flags;
> + bool yielded = 0;
> +
> + local_irq_save(flags);
> + rq = this_rq();
> +
> +again:
> + p_rq = task_rq(p);
> + double_rq_lock(rq, p_rq);
> + while (task_rq(p) != p_rq) {
> + double_rq_unlock(rq, p_rq);
> + goto again;
> + }
> +
> + if (!curr->sched_class->yield_to_task)
> + goto out;
> +
> + if (curr->sched_class != p->sched_class)
> + goto out;
> +
> + if (task_running(p_rq, p) || p->state)
> + goto out;
> +
> + yielded = curr->sched_class->yield_to_task(rq, p, preempt);
> +
> + if (yielded) {
> + schedstat_inc(rq, yld_count);
> + current->sched_class->yield_task(rq);
> + }

We can avoid this second indirect function call by

> +
> +out:
> + double_rq_unlock(rq, p_rq);
> + local_irq_restore(flags);
> +
> + if (yielded)
> + schedule();
> +
> + return yielded;
> +}
> +EXPORT_SYMBOL_GPL(yield_to);


> +static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preempt)
> +{
> + struct sched_entity *se = &p->se;
> +
> + if (!se->on_rq)
> + return false;
> +
> + /* Tell the scheduler that we'd really like pse to run next. */
> + set_next_buddy(se);
> +
> + /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
> + if (preempt)
> + resched_task(rq->curr);

calling: yield_task_fair(rq); here.

> + return true;
> +}


I'll make that change on commit.

2011-02-01 15:52:55

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH -v8a 3/7] sched: use a buddy to implement yield_task_fair

On Tue, 2011-02-01 at 09:51 -0500, Rik van Riel wrote:
> --- a/kernel/sysctl.c
> +++ b/kernel/sysctl.c
> @@ -375,13 +375,6 @@ static struct ctl_table kern_table[] = {
> .mode = 0644,
> .proc_handler = sched_rt_handler,
> },
> - {
> - .procname = "sched_compat_yield",
> - .data = &sysctl_sched_compat_yield,
> - .maxlen = sizeof(unsigned int),
> - .mode = 0644,
> - .proc_handler = proc_dointvec,
> - },
> #ifdef CONFIG_PROVE_LOCKING
> {
> .procname = "prove_locking",
>
There's another instance of sysctl_sched_compat_yield in sched.h, I'll
remove that and its definition in sched_fair.c too ;-)

2011-02-03 12:57:49

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH -v8a 3/7] sched: use a buddy to implement yield_task_fair

On Tue, 2011-02-01 at 09:51 -0500, Rik van Riel wrote:
> -static void yield_task_fair(struct rq *rq)
> -{
> - struct task_struct *curr = rq->curr;
> - struct cfs_rq *cfs_rq = task_cfs_rq(curr);
> - struct sched_entity *rightmost, *se = &curr->se;
> -
> - /*
> - * Are we the only task in the tree?
> - */
> - if (unlikely(rq->nr_running == 1))
> - return;
> -
> - clear_buddies(cfs_rq, se);
> -
> - if (likely(!sysctl_sched_compat_yield) && curr->policy != SCHED_BATCH) {
> - update_rq_clock(rq);
> - /*
> - * Update run-time statistics of the 'current'.
> - */
> - update_curr(cfs_rq);
> -
> - return;
> - }
> - /*
> - * Find the rightmost entry in the rbtree:
> - */
> - rightmost = __pick_last_entity(cfs_rq);
> - /*
> - * Already in the rightmost position?
> - */
> - if (unlikely(!rightmost || entity_before(rightmost, se)))
> - return;
> -
> - /*
> - * Minimally necessary key value to be last in the tree:
> - * Upon rescheduling, sched_class::put_prev_task() will place
> - * 'current' within the tree based on its new key value.
> - */
> - se->vruntime = rightmost->vruntime + 1;
> -}


This also wants the below bit in order to build without warnings for
SCHED_DEBUG=n:


---
Index: linux-2.6/kernel/sched_fair.c
===================================================================
--- linux-2.6.orig/kernel/sched_fair.c
+++ linux-2.6/kernel/sched_fair.c
@@ -431,6 +431,7 @@ static struct sched_entity *__pick_next_
return rb_entry(next, struct sched_entity, run_node);
}

+#ifdef CONFIG_SCHED_DEBUG
static struct sched_entity *__pick_last_entity(struct cfs_rq *cfs_rq)
{
struct rb_node *last = rb_last(&cfs_rq->tasks_timeline);
@@ -445,7 +446,6 @@ static struct sched_entity *__pick_last_
* Scheduling class statistics methods:
*/

-#ifdef CONFIG_SCHED_DEBUG
int sched_proc_update_handler(struct ctl_table *table, int write,
void __user *buffer, size_t *lenp,
loff_t *ppos)

2011-02-03 12:57:59

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH -v8a 4/7] sched: Add yield_to(task, preempt) functionality

On Tue, 2011-02-01 at 09:50 -0500, Rik van Riel wrote:
> +bool __sched yield_to(struct task_struct *p, bool preempt)
> +{
> + struct task_struct *curr = current;
> + struct rq *rq, *p_rq;
> + unsigned long flags;
> + bool yielded = 0;
> +
> + local_irq_save(flags);
> + rq = this_rq();
> +
> +again:
> + p_rq = task_rq(p);
> + double_rq_lock(rq, p_rq);
> + while (task_rq(p) != p_rq) {
> + double_rq_unlock(rq, p_rq);
> + goto again;
> + }
> +
> + if (!curr->sched_class->yield_to_task)
> + goto out;
> +
> + if (curr->sched_class != p->sched_class)
> + goto out;
> +
> + if (task_running(p_rq, p) || p->state)
> + goto out;
> +
> + yielded = curr->sched_class->yield_to_task(rq, p, preempt);
> +
> + if (yielded) {
> + schedstat_inc(rq, yld_count);
> + current->sched_class->yield_task(rq);
> + }
> +
> +out:
> + double_rq_unlock(rq, p_rq);
> + local_irq_restore(flags);
> +
> + if (yielded)
> + schedule();
> +
> + return yielded;
> +}
> +EXPORT_SYMBOL_GPL(yield_to);


This also wants the below bit to successfully build on SMP=n,

---
Index: linux-2.6/kernel/sched.c
===================================================================
--- linux-2.6.orig/kernel/sched.c
+++ linux-2.6/kernel/sched.c
@@ -1686,6 +1686,39 @@ static void double_rq_unlock(struct rq *
__release(rq2->lock);
}

+#else /* CONFIG_SMP */
+
+/*
+ * double_rq_lock - safely lock two runqueues
+ *
+ * Note this does not disable interrupts like task_rq_lock,
+ * you need to do so manually before calling.
+ */
+static void double_rq_lock(struct rq *rq1, struct rq *rq2)
+ __acquires(rq1->lock)
+ __acquires(rq2->lock)
+{
+ BUG_ON(!irqs_disabled());
+ BUG_ON(rq1 != rq2);
+ raw_spin_lock(&rq1->lock);
+ __acquire(rq2->lock); /* Fake it out ;) */
+}
+
+/*
+ * double_rq_unlock - safely unlock two runqueues
+ *
+ * Note this does not restore interrupts like task_rq_unlock,
+ * you need to do so manually after calling.
+ */
+static void double_rq_unlock(struct rq *rq1, struct rq *rq2)
+ __releases(rq1->lock)
+ __releases(rq2->lock)
+{
+ BUG_ON(rq1 != rq2);
+ raw_spin_unlock(&rq1->lock);
+ __release(rq2->lock);
+}
+
#endif

static void calc_load_account_idle(struct rq *this_rq);

2011-02-03 14:12:01

by Rik van Riel

[permalink] [raw]
Subject: [tip:sched/core] sched: Check the right ->nr_running in yield_task_fair()

Commit-ID: 725e7580aaf98e9f7b22b8ccfc640ad0c09e2b08
Gitweb: http://git.kernel.org/tip/725e7580aaf98e9f7b22b8ccfc640ad0c09e2b08
Author: Rik van Riel <[email protected]>
AuthorDate: Tue, 1 Feb 2011 09:47:15 -0500
Committer: Ingo Molnar <[email protected]>
CommitDate: Thu, 3 Feb 2011 14:20:32 +0100

sched: Check the right ->nr_running in yield_task_fair()

With CONFIG_FAIR_GROUP_SCHED, each task_group has its own cfs_rq.
Yielding to a task from another cfs_rq may be worthwhile, since
a process calling yield typically cannot use the CPU right now.

Therefor, we want to check the per-cpu nr_running, not the
cgroup local one.

Signed-off-by: Rik van Riel <[email protected]>
Signed-off-by: Peter Zijlstra <[email protected]>
LKML-Reference: <[email protected]>
Signed-off-by: Ingo Molnar <[email protected]>
---
kernel/sched_fair.c | 2 +-
1 files changed, 1 insertions(+), 1 deletions(-)

diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index 55040f3..4de9905 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -1331,7 +1331,7 @@ static void yield_task_fair(struct rq *rq)
/*
* Are we the only task in the tree?
*/
- if (unlikely(cfs_rq->nr_running == 1))
+ if (unlikely(rq->nr_running == 1))
return;

clear_buddies(cfs_rq, se);

2011-02-03 14:12:23

by Rik van Riel

[permalink] [raw]
Subject: [tip:sched/core] sched: Limit the scope of clear_buddies

Commit-ID: 2c13c919d9e9a3db9896143a501f83dcbbe1ced4
Gitweb: http://git.kernel.org/tip/2c13c919d9e9a3db9896143a501f83dcbbe1ced4
Author: Rik van Riel <[email protected]>
AuthorDate: Tue, 1 Feb 2011 09:48:37 -0500
Committer: Ingo Molnar <[email protected]>
CommitDate: Thu, 3 Feb 2011 14:20:32 +0100

sched: Limit the scope of clear_buddies

The clear_buddies function does not seem to play well with the concept
of hierarchical runqueues. In the following tree, task groups are
represented by 'G', tasks by 'T', next by 'n' and last by 'l'.

(nl)
/ \
G(nl) G
/ \ \
T(l) T(n) T

This situation can arise when a task is woken up T(n), and the previously
running task T(l) is marked last.

When clear_buddies is called from either T(l) or T(n), the next and last
buddies of the group G(nl) will be cleared. This is not the desired
result, since we would like to be able to find the other type of buddy
in many cases.

This especially a worry when implementing yield_task_fair through the
buddy system.

The fix is simple: only clear the buddy type that the task itself
is indicated to be. As an added bonus, we stop walking up the tree
when the buddy has already been cleared or pointed elsewhere.

Signed-off-by: Rik van Riel <[email protected]>
Signed-off-by: Peter Zijlstra <[email protected]>
LKML-Reference: <[email protected]>
Signed-off-by: Ingo Molnar <[email protected]>
---
kernel/sched_fair.c | 30 +++++++++++++++++++++++-------
1 files changed, 23 insertions(+), 7 deletions(-)

diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index 4de9905..a785e08 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -995,19 +995,35 @@ enqueue_entity(struct cfs_rq *cfs_rq, struct sched_entity *se, int flags)
list_add_leaf_cfs_rq(cfs_rq);
}

-static void __clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)
+static void __clear_buddies_last(struct sched_entity *se)
{
- if (!se || cfs_rq->last == se)
- cfs_rq->last = NULL;
+ for_each_sched_entity(se) {
+ struct cfs_rq *cfs_rq = cfs_rq_of(se);
+ if (cfs_rq->last == se)
+ cfs_rq->last = NULL;
+ else
+ break;
+ }
+}

- if (!se || cfs_rq->next == se)
- cfs_rq->next = NULL;
+static void __clear_buddies_next(struct sched_entity *se)
+{
+ for_each_sched_entity(se) {
+ struct cfs_rq *cfs_rq = cfs_rq_of(se);
+ if (cfs_rq->next == se)
+ cfs_rq->next = NULL;
+ else
+ break;
+ }
}

static void clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)
{
- for_each_sched_entity(se)
- __clear_buddies(cfs_rq_of(se), se);
+ if (cfs_rq->last == se)
+ __clear_buddies_last(se);
+
+ if (cfs_rq->next == se)
+ __clear_buddies_next(se);
}

static void

2011-02-03 14:12:50

by Rik van Riel

[permalink] [raw]
Subject: [tip:sched/core] sched: Use a buddy to implement yield_task_fair()

Commit-ID: ac53db596cc08ecb8040cfb6f71ae40c6f2041c4
Gitweb: http://git.kernel.org/tip/ac53db596cc08ecb8040cfb6f71ae40c6f2041c4
Author: Rik van Riel <[email protected]>
AuthorDate: Tue, 1 Feb 2011 09:51:03 -0500
Committer: Ingo Molnar <[email protected]>
CommitDate: Thu, 3 Feb 2011 14:20:33 +0100

sched: Use a buddy to implement yield_task_fair()

Use the buddy mechanism to implement yield_task_fair. This
allows us to skip onto the next highest priority se at every
level in the CFS tree, unless doing so would introduce gross
unfairness in CPU time distribution.

We order the buddy selection in pick_next_entity to check
yield first, then last, then next. We need next to be able
to override yield, because it is possible for the "next" and
"yield" task to be different processen in the same sub-tree
of the CFS tree. When they are, we need to go into that
sub-tree regardless of the "yield" hint, and pick the correct
entity once we get to the right level.

Signed-off-by: Rik van Riel <[email protected]>
Signed-off-by: Peter Zijlstra <[email protected]>
LKML-Reference: <[email protected]>
Signed-off-by: Ingo Molnar <[email protected]>
---
include/linux/sched.h | 2 -
kernel/sched.c | 2 +-
kernel/sched_debug.c | 2 +-
kernel/sched_fair.c | 148 +++++++++++++++++++++++++++++--------------------
kernel/sysctl.c | 7 --
5 files changed, 90 insertions(+), 71 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 0542774..4e9fad2 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1942,8 +1942,6 @@ int sched_rt_handler(struct ctl_table *table, int write,
void __user *buffer, size_t *lenp,
loff_t *ppos);

-extern unsigned int sysctl_sched_compat_yield;
-
#ifdef CONFIG_SCHED_AUTOGROUP
extern unsigned int sysctl_sched_autogroup_enabled;

diff --git a/kernel/sched.c b/kernel/sched.c
index 477e1bc..ae5e1a1 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -324,7 +324,7 @@ struct cfs_rq {
* 'curr' points to currently running entity on this cfs_rq.
* It is set to NULL otherwise (i.e when none are currently running).
*/
- struct sched_entity *curr, *next, *last;
+ struct sched_entity *curr, *next, *last, *skip;

unsigned int nr_spread_over;

diff --git a/kernel/sched_debug.c b/kernel/sched_debug.c
index eb6cb8e..7bacd83 100644
--- a/kernel/sched_debug.c
+++ b/kernel/sched_debug.c
@@ -179,7 +179,7 @@ void print_cfs_rq(struct seq_file *m, int cpu, struct cfs_rq *cfs_rq)

raw_spin_lock_irqsave(&rq->lock, flags);
if (cfs_rq->rb_leftmost)
- MIN_vruntime = (__pick_next_entity(cfs_rq))->vruntime;
+ MIN_vruntime = (__pick_first_entity(cfs_rq))->vruntime;
last = __pick_last_entity(cfs_rq);
if (last)
max_vruntime = last->vruntime;
diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index a785e08..c0fbeb9 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -69,14 +69,6 @@ static unsigned int sched_nr_latency = 8;
unsigned int sysctl_sched_child_runs_first __read_mostly;

/*
- * sys_sched_yield() compat mode
- *
- * This option switches the agressive yield implementation of the
- * old scheduler back on.
- */
-unsigned int __read_mostly sysctl_sched_compat_yield;
-
-/*
* SCHED_OTHER wake-up granularity.
* (default: 1 msec * (1 + ilog(ncpus)), units: nanoseconds)
*
@@ -419,7 +411,7 @@ static void __dequeue_entity(struct cfs_rq *cfs_rq, struct sched_entity *se)
rb_erase(&se->run_node, &cfs_rq->tasks_timeline);
}

-static struct sched_entity *__pick_next_entity(struct cfs_rq *cfs_rq)
+static struct sched_entity *__pick_first_entity(struct cfs_rq *cfs_rq)
{
struct rb_node *left = cfs_rq->rb_leftmost;

@@ -429,6 +421,17 @@ static struct sched_entity *__pick_next_entity(struct cfs_rq *cfs_rq)
return rb_entry(left, struct sched_entity, run_node);
}

+static struct sched_entity *__pick_next_entity(struct sched_entity *se)
+{
+ struct rb_node *next = rb_next(&se->run_node);
+
+ if (!next)
+ return NULL;
+
+ return rb_entry(next, struct sched_entity, run_node);
+}
+
+#ifdef CONFIG_SCHED_DEBUG
static struct sched_entity *__pick_last_entity(struct cfs_rq *cfs_rq)
{
struct rb_node *last = rb_last(&cfs_rq->tasks_timeline);
@@ -443,7 +446,6 @@ static struct sched_entity *__pick_last_entity(struct cfs_rq *cfs_rq)
* Scheduling class statistics methods:
*/

-#ifdef CONFIG_SCHED_DEBUG
int sched_proc_update_handler(struct ctl_table *table, int write,
void __user *buffer, size_t *lenp,
loff_t *ppos)
@@ -1017,6 +1019,17 @@ static void __clear_buddies_next(struct sched_entity *se)
}
}

+static void __clear_buddies_skip(struct sched_entity *se)
+{
+ for_each_sched_entity(se) {
+ struct cfs_rq *cfs_rq = cfs_rq_of(se);
+ if (cfs_rq->skip == se)
+ cfs_rq->skip = NULL;
+ else
+ break;
+ }
+}
+
static void clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)
{
if (cfs_rq->last == se)
@@ -1024,6 +1037,9 @@ static void clear_buddies(struct cfs_rq *cfs_rq, struct sched_entity *se)

if (cfs_rq->next == se)
__clear_buddies_next(se);
+
+ if (cfs_rq->skip == se)
+ __clear_buddies_skip(se);
}

static void
@@ -1099,7 +1115,7 @@ check_preempt_tick(struct cfs_rq *cfs_rq, struct sched_entity *curr)
return;

if (cfs_rq->nr_running > 1) {
- struct sched_entity *se = __pick_next_entity(cfs_rq);
+ struct sched_entity *se = __pick_first_entity(cfs_rq);
s64 delta = curr->vruntime - se->vruntime;

if (delta < 0)
@@ -1143,13 +1159,27 @@ set_next_entity(struct cfs_rq *cfs_rq, struct sched_entity *se)
static int
wakeup_preempt_entity(struct sched_entity *curr, struct sched_entity *se);

+/*
+ * Pick the next process, keeping these things in mind, in this order:
+ * 1) keep things fair between processes/task groups
+ * 2) pick the "next" process, since someone really wants that to run
+ * 3) pick the "last" process, for cache locality
+ * 4) do not run the "skip" process, if something else is available
+ */
static struct sched_entity *pick_next_entity(struct cfs_rq *cfs_rq)
{
- struct sched_entity *se = __pick_next_entity(cfs_rq);
+ struct sched_entity *se = __pick_first_entity(cfs_rq);
struct sched_entity *left = se;

- if (cfs_rq->next && wakeup_preempt_entity(cfs_rq->next, left) < 1)
- se = cfs_rq->next;
+ /*
+ * Avoid running the skip buddy, if running something else can
+ * be done without getting too unfair.
+ */
+ if (cfs_rq->skip == se) {
+ struct sched_entity *second = __pick_next_entity(se);
+ if (second && wakeup_preempt_entity(second, left) < 1)
+ se = second;
+ }

/*
* Prefer last buddy, try to return the CPU to a preempted task.
@@ -1157,6 +1187,12 @@ static struct sched_entity *pick_next_entity(struct cfs_rq *cfs_rq)
if (cfs_rq->last && wakeup_preempt_entity(cfs_rq->last, left) < 1)
se = cfs_rq->last;

+ /*
+ * Someone really wants this to run. If it's not unfair, run it.
+ */
+ if (cfs_rq->next && wakeup_preempt_entity(cfs_rq->next, left) < 1)
+ se = cfs_rq->next;
+
clear_buddies(cfs_rq, se);

return se;
@@ -1333,52 +1369,6 @@ static void dequeue_task_fair(struct rq *rq, struct task_struct *p, int flags)
hrtick_update(rq);
}

-/*
- * sched_yield() support is very simple - we dequeue and enqueue.
- *
- * If compat_yield is turned on then we requeue to the end of the tree.
- */
-static void yield_task_fair(struct rq *rq)
-{
- struct task_struct *curr = rq->curr;
- struct cfs_rq *cfs_rq = task_cfs_rq(curr);
- struct sched_entity *rightmost, *se = &curr->se;
-
- /*
- * Are we the only task in the tree?
- */
- if (unlikely(rq->nr_running == 1))
- return;
-
- clear_buddies(cfs_rq, se);
-
- if (likely(!sysctl_sched_compat_yield) && curr->policy != SCHED_BATCH) {
- update_rq_clock(rq);
- /*
- * Update run-time statistics of the 'current'.
- */
- update_curr(cfs_rq);
-
- return;
- }
- /*
- * Find the rightmost entry in the rbtree:
- */
- rightmost = __pick_last_entity(cfs_rq);
- /*
- * Already in the rightmost position?
- */
- if (unlikely(!rightmost || entity_before(rightmost, se)))
- return;
-
- /*
- * Minimally necessary key value to be last in the tree:
- * Upon rescheduling, sched_class::put_prev_task() will place
- * 'current' within the tree based on its new key value.
- */
- se->vruntime = rightmost->vruntime + 1;
-}
-
#ifdef CONFIG_SMP

static void task_waking_fair(struct rq *rq, struct task_struct *p)
@@ -1849,6 +1839,14 @@ static void set_next_buddy(struct sched_entity *se)
}
}

+static void set_skip_buddy(struct sched_entity *se)
+{
+ if (likely(task_of(se)->policy != SCHED_IDLE)) {
+ for_each_sched_entity(se)
+ cfs_rq_of(se)->skip = se;
+ }
+}
+
/*
* Preempt the current task with a newly woken task if needed:
*/
@@ -1947,6 +1945,36 @@ static void put_prev_task_fair(struct rq *rq, struct task_struct *prev)
}
}

+/*
+ * sched_yield() is very simple
+ *
+ * The magic of dealing with the ->skip buddy is in pick_next_entity.
+ */
+static void yield_task_fair(struct rq *rq)
+{
+ struct task_struct *curr = rq->curr;
+ struct cfs_rq *cfs_rq = task_cfs_rq(curr);
+ struct sched_entity *se = &curr->se;
+
+ /*
+ * Are we the only task in the tree?
+ */
+ if (unlikely(rq->nr_running == 1))
+ return;
+
+ clear_buddies(cfs_rq, se);
+
+ if (curr->policy != SCHED_BATCH) {
+ update_rq_clock(rq);
+ /*
+ * Update run-time statistics of the 'current'.
+ */
+ update_curr(cfs_rq);
+ }
+
+ set_skip_buddy(se);
+}
+
#ifdef CONFIG_SMP
/**************************************************
* Fair scheduling class load-balancing methods:
diff --git a/kernel/sysctl.c b/kernel/sysctl.c
index bc86bb3..cbfda7e 100644
--- a/kernel/sysctl.c
+++ b/kernel/sysctl.c
@@ -360,13 +360,6 @@ static struct ctl_table kern_table[] = {
.mode = 0644,
.proc_handler = sched_rt_handler,
},
- {
- .procname = "sched_compat_yield",
- .data = &sysctl_sched_compat_yield,
- .maxlen = sizeof(unsigned int),
- .mode = 0644,
- .proc_handler = proc_dointvec,
- },
#ifdef CONFIG_SCHED_AUTOGROUP
{
.procname = "sched_autogroup_enabled",

2011-02-03 14:13:11

by Mike Galbraith

[permalink] [raw]
Subject: [tip:sched/core] sched: Add yield_to(task, preempt) functionality

Commit-ID: d95f412200652694e63e64bfd49f0ae274a54479
Gitweb: http://git.kernel.org/tip/d95f412200652694e63e64bfd49f0ae274a54479
Author: Mike Galbraith <[email protected]>
AuthorDate: Tue, 1 Feb 2011 09:50:51 -0500
Committer: Ingo Molnar <[email protected]>
CommitDate: Thu, 3 Feb 2011 14:20:33 +0100

sched: Add yield_to(task, preempt) functionality

Currently only implemented for fair class tasks.

Add a yield_to_task method() to the fair scheduling class. allowing the
caller of yield_to() to accelerate another thread in it's thread group,
task group.

Implemented via a scheduler hint, using cfs_rq->next to encourage the
target being selected. We can rely on pick_next_entity to keep things
fair, so noone can accelerate a thread that has already used its fair
share of CPU time.

This also means callers should only call yield_to when they really
mean it. Calling it too often can result in the scheduler just
ignoring the hint.

Signed-off-by: Rik van Riel <[email protected]>
Signed-off-by: Marcelo Tosatti <[email protected]>
Signed-off-by: Mike Galbraith <[email protected]>
Signed-off-by: Peter Zijlstra <[email protected]>
LKML-Reference: <[email protected]>
Signed-off-by: Ingo Molnar <[email protected]>
---
include/linux/sched.h | 2 +
kernel/sched.c | 85 +++++++++++++++++++++++++++++++++++++++++++++++++
kernel/sched_fair.c | 20 +++++++++++
3 files changed, 107 insertions(+), 0 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 4e9fad2..c88b3bf 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1058,6 +1058,7 @@ struct sched_class {
void (*enqueue_task) (struct rq *rq, struct task_struct *p, int flags);
void (*dequeue_task) (struct rq *rq, struct task_struct *p, int flags);
void (*yield_task) (struct rq *rq);
+ bool (*yield_to_task) (struct rq *rq, struct task_struct *p, bool preempt);

void (*check_preempt_curr) (struct rq *rq, struct task_struct *p, int flags);

@@ -1972,6 +1973,7 @@ static inline int rt_mutex_getprio(struct task_struct *p)
# define rt_mutex_adjust_pi(p) do { } while (0)
#endif

+extern bool yield_to(struct task_struct *p, bool preempt);
extern void set_user_nice(struct task_struct *p, long nice);
extern int task_prio(const struct task_struct *p);
extern int task_nice(const struct task_struct *p);
diff --git a/kernel/sched.c b/kernel/sched.c
index ae5e1a1..2effcb7 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -1686,6 +1686,39 @@ static void double_rq_unlock(struct rq *rq1, struct rq *rq2)
__release(rq2->lock);
}

+#else /* CONFIG_SMP */
+
+/*
+ * double_rq_lock - safely lock two runqueues
+ *
+ * Note this does not disable interrupts like task_rq_lock,
+ * you need to do so manually before calling.
+ */
+static void double_rq_lock(struct rq *rq1, struct rq *rq2)
+ __acquires(rq1->lock)
+ __acquires(rq2->lock)
+{
+ BUG_ON(!irqs_disabled());
+ BUG_ON(rq1 != rq2);
+ raw_spin_lock(&rq1->lock);
+ __acquire(rq2->lock); /* Fake it out ;) */
+}
+
+/*
+ * double_rq_unlock - safely unlock two runqueues
+ *
+ * Note this does not restore interrupts like task_rq_unlock,
+ * you need to do so manually after calling.
+ */
+static void double_rq_unlock(struct rq *rq1, struct rq *rq2)
+ __releases(rq1->lock)
+ __releases(rq2->lock)
+{
+ BUG_ON(rq1 != rq2);
+ raw_spin_unlock(&rq1->lock);
+ __release(rq2->lock);
+}
+
#endif

static void calc_load_account_idle(struct rq *this_rq);
@@ -5448,6 +5481,58 @@ void __sched yield(void)
}
EXPORT_SYMBOL(yield);

+/**
+ * yield_to - yield the current processor to another thread in
+ * your thread group, or accelerate that thread toward the
+ * processor it's on.
+ *
+ * It's the caller's job to ensure that the target task struct
+ * can't go away on us before we can do any checks.
+ *
+ * Returns true if we indeed boosted the target task.
+ */
+bool __sched yield_to(struct task_struct *p, bool preempt)
+{
+ struct task_struct *curr = current;
+ struct rq *rq, *p_rq;
+ unsigned long flags;
+ bool yielded = 0;
+
+ local_irq_save(flags);
+ rq = this_rq();
+
+again:
+ p_rq = task_rq(p);
+ double_rq_lock(rq, p_rq);
+ while (task_rq(p) != p_rq) {
+ double_rq_unlock(rq, p_rq);
+ goto again;
+ }
+
+ if (!curr->sched_class->yield_to_task)
+ goto out;
+
+ if (curr->sched_class != p->sched_class)
+ goto out;
+
+ if (task_running(p_rq, p) || p->state)
+ goto out;
+
+ yielded = curr->sched_class->yield_to_task(rq, p, preempt);
+ if (yielded)
+ schedstat_inc(rq, yld_count);
+
+out:
+ double_rq_unlock(rq, p_rq);
+ local_irq_restore(flags);
+
+ if (yielded)
+ schedule();
+
+ return yielded;
+}
+EXPORT_SYMBOL_GPL(yield_to);
+
/*
* This task is about to go to sleep on IO. Increment rq->nr_iowait so
* that process accounting knows that this is a task in IO wait state.
diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index c0fbeb9..0270246 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -1975,6 +1975,25 @@ static void yield_task_fair(struct rq *rq)
set_skip_buddy(se);
}

+static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preempt)
+{
+ struct sched_entity *se = &p->se;
+
+ if (!se->on_rq)
+ return false;
+
+ /* Tell the scheduler that we'd really like pse to run next. */
+ set_next_buddy(se);
+
+ /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
+ if (preempt)
+ resched_task(rq->curr);
+
+ yield_task_fair(rq);
+
+ return true;
+}
+
#ifdef CONFIG_SMP
/**************************************************
* Fair scheduling class load-balancing methods:
@@ -4243,6 +4262,7 @@ static const struct sched_class fair_sched_class = {
.enqueue_task = enqueue_task_fair,
.dequeue_task = dequeue_task_fair,
.yield_task = yield_task_fair,
+ .yield_to_task = yield_to_task_fair,

.check_preempt_curr = check_preempt_wakeup,

2011-02-07 09:08:27

by Avi Kivity

[permalink] [raw]
Subject: Re: [PATCH -v8a 0/7] directed yield for Pause Loop Exiting

On 02/01/2011 04:44 PM, Rik van Riel wrote:
> When running SMP virtual machines, it is possible for one VCPU to be
> spinning on a spinlock, while the VCPU that holds the spinlock is not
> currently running, because the host scheduler preempted it to run
> something else.
>
> Both Intel and AMD CPUs have a feature that detects when a virtual
> CPU is spinning on a lock and will trap to the host.
>
> The current KVM code sleeps for a bit whenever that happens, which
> results in eg. a 64 VCPU Windows guest taking forever and a bit to
> boot up. This is because the VCPU holding the lock is actually
> running and not sleeping, so the pause is counter-productive.
>
> In other workloads a pause can also be counter-productive, with
> spinlock detection resulting in one guest giving up its CPU time
> to the others. Instead of spinning, it ends up simply not running
> much at all.
>
> This patch series aims to fix that, by having a VCPU that spins
> give the remainder of its timeslice to another VCPU in the same
> guest before yielding the CPU - one that is runnable but got
> preempted, hopefully the lock holder.
>
> v8:
> - some more changes and cleanups suggested by Peter
> v7:
> - move the vcpu to pid mapping to inside the vcpu->mutex
> - rename ->yield to ->skip
> - merge patch 5 into patch 4
> v6:
> - implement yield_task_fair in a way that works with task groups,
> this allows me to actually get a performance improvement!
> - fix another race Avi pointed out, the code should be good now
> v5:
> - fix the race condition Avi pointed out, by tracking vcpu->pid
> - also allows us to yield to vcpu tasks that got preempted while in qemu
> userspace
> v4:
> - change to newer version of Mike Galbraith's yield_to implementation
> - chainsaw out some code from Mike that looked like a great idea, but
> turned out to give weird interactions in practice
> v3:
> - more cleanups
> - change to Mike Galbraith's yield_to implementation
> - yield to spinning VCPUs, this seems to work better in some
> situations and has little downside potential
> v2:
> - make lots of cleanups and improvements suggested
> - do not implement timeslice scheduling or fairness stuff
> yet, since it is not entirely clear how to do that right
> (suggestions welcome)
>
>
> Benchmark results:
>
> Two 4-CPU KVM guests are pinned to the same 4 physical CPUs.
>
> One guest runs the AMQP performance test, the other guest runs
> 0, 2 or 4 infinite loops, for CPU overcommit factors of 0, 1.5
> and 4.
>
> The AMQP perftest is run 30 times, with message payloads of 8 and 16 bytes.
>
> size8 no overcommit 1.5x overcommit 2x overcommit
>
> no PLE 223801 135137 104951
> PLE 224135 141105 118744
>
> size16 no overcommit 1.5x overcommit 2x overcommit
>
> no PLE 222424 126175 105299
> PLE 222534 138082 132945
>
> Note: this is with the KVM guests NOT running inside cgroups. There
> seems to be a CPU load balancing issue with cgroup fair group scheduling,
> which often results in one guest getting only 80% CPU time and the other
> guest 320%. That will have to be fixed to get meaningful results with
> cgroups.
>
> CPU time division between the AMQP guest and the infinite loop guest
> were not exactly fair, but the guests got close to the same amount
> of CPU time in each test run.
>
> There is a substantial amount of randomness in CPU time division between
> guests, but the performance improvement is consistent between multiple
> runs.
>

I've merged tip's sched/core, which includes yield_to(), and applied the
final three patches. Thanks.

--
error compiling committee.c: too many arguments to function

2011-02-26 00:43:22

by Venkatesh Pallipadi

[permalink] [raw]
Subject: Re: [tip:sched/core] sched: Add yield_to(task, preempt) functionality

On Thu, Feb 3, 2011 at 6:12 AM, tip-bot for Mike Galbraith
<[email protected]> wrote:
> Commit-ID: ?d95f412200652694e63e64bfd49f0ae274a54479
> Gitweb: ? ? http://git.kernel.org/tip/d95f412200652694e63e64bfd49f0ae274a54479
> Author: ? ? Mike Galbraith <[email protected]>
> AuthorDate: Tue, 1 Feb 2011 09:50:51 -0500
> Committer: ?Ingo Molnar <[email protected]>
> CommitDate: Thu, 3 Feb 2011 14:20:33 +0100
>
> sched: Add yield_to(task, preempt) functionality

I was looking at this patch, while trying to figure out how best to
use next buddy to solve another unrelated to this cgroup context
switching problem. While going through this change I see something
that I don't really understand (inlined below). Not sure whether what
I am looking at is a bug or I am missing something very obvious.

Thanks in advance for clarification.

>
> Currently only implemented for fair class tasks.
>
> Add a yield_to_task method() to the fair scheduling class. allowing the
> caller of yield_to() to accelerate another thread in it's thread group,
> task group.
>

<snip>

>
> ?static void calc_load_account_idle(struct rq *this_rq);
> @@ -5448,6 +5481,58 @@ void __sched yield(void)
> ?}
> ?EXPORT_SYMBOL(yield);
>
> +/**
> + * yield_to - yield the current processor to another thread in
> + * your thread group, or accelerate that thread toward the
> + * processor it's on.
> + *
> + * It's the caller's job to ensure that the target task struct
> + * can't go away on us before we can do any checks.
> + *
> + * Returns true if we indeed boosted the target task.
> + */
> +bool __sched yield_to(struct task_struct *p, bool preempt)
> +{
> + ? ? ? struct task_struct *curr = current;
> + ? ? ? struct rq *rq, *p_rq;
> + ? ? ? unsigned long flags;
> + ? ? ? bool yielded = 0;
> +
> + ? ? ? local_irq_save(flags);
> + ? ? ? rq = this_rq();
> +
> +again:
> + ? ? ? p_rq = task_rq(p);
> + ? ? ? double_rq_lock(rq, p_rq);
> + ? ? ? while (task_rq(p) != p_rq) {
> + ? ? ? ? ? ? ? double_rq_unlock(rq, p_rq);
> + ? ? ? ? ? ? ? goto again;
> + ? ? ? }
> +
> + ? ? ? if (!curr->sched_class->yield_to_task)
> + ? ? ? ? ? ? ? goto out;
> +
> + ? ? ? if (curr->sched_class != p->sched_class)
> + ? ? ? ? ? ? ? goto out;
> +
> + ? ? ? if (task_running(p_rq, p) || p->state)
> + ? ? ? ? ? ? ? goto out;
> +
> + ? ? ? yielded = curr->sched_class->yield_to_task(rq, p, preempt);
> + ? ? ? if (yielded)
> + ? ? ? ? ? ? ? schedstat_inc(rq, yld_count);
> +
> +out:
> + ? ? ? double_rq_unlock(rq, p_rq);
> + ? ? ? local_irq_restore(flags);
> +
> + ? ? ? if (yielded)
> + ? ? ? ? ? ? ? schedule();
> +
> + ? ? ? return yielded;
> +}
> +EXPORT_SYMBOL_GPL(yield_to);
> +
> ?/*
> ?* This task is about to go to sleep on IO. Increment rq->nr_iowait so
> ?* that process accounting knows that this is a task in IO wait state.
> diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
> index c0fbeb9..0270246 100644
> --- a/kernel/sched_fair.c
> +++ b/kernel/sched_fair.c
> @@ -1975,6 +1975,25 @@ static void yield_task_fair(struct rq *rq)
> ? ? ? ?set_skip_buddy(se);
> ?}
>
> +static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preempt)
> +{
> + ? ? ? struct sched_entity *se = &p->se;
> +
> + ? ? ? if (!se->on_rq)
> + ? ? ? ? ? ? ? return false;
> +
> + ? ? ? /* Tell the scheduler that we'd really like pse to run next. */
> + ? ? ? set_next_buddy(se);

The below comment says about rescheduling p's CPU. But the rq variable
we have here is the curr_rq and not p_rq. So, should this be done in
yield_to() with p_rq. I did try to see the discussion on other
versions of this patch. v3 and before had -
"resched_task(task_of(p_cfs_rq->curr));" which seems to be correct...

Also, we already have a test of (task_running(p_rq, p) || p->state) in
yield_to. Wondering why we need the additional (!se->on_rq) above?

> +
> + ? ? ? /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
> + ? ? ? if (preempt)
> + ? ? ? ? ? ? ? resched_task(rq->curr);
> +
> + ? ? ? yield_task_fair(rq);
> +
> + ? ? ? return true;
> +}

Thanks,
Venki

> +
> ?#ifdef CONFIG_SMP
> ?/**************************************************
> ?* Fair scheduling class load-balancing methods:
> @@ -4243,6 +4262,7 @@ static const struct sched_class fair_sched_class = {
> ? ? ? ?.enqueue_task ? ? ? ? ? = enqueue_task_fair,
> ? ? ? ?.dequeue_task ? ? ? ? ? = dequeue_task_fair,
> ? ? ? ?.yield_task ? ? ? ? ? ? = yield_task_fair,
> + ? ? ? .yield_to_task ? ? ? ? ?= yield_to_task_fair,
>
> ? ? ? ?.check_preempt_curr ? ? = check_preempt_wakeup,
>
> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to [email protected]
> More majordomo info at ?http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at ?http://www.tux.org/lkml/
>

2011-02-26 05:45:01

by Rik van Riel

[permalink] [raw]
Subject: Re: [tip:sched/core] sched: Add yield_to(task, preempt) functionality

On 02/25/2011 07:43 PM, Venkatesh Pallipadi wrote:
> On Thu, Feb 3, 2011 at 6:12 AM, tip-bot for Mike Galbraith
> <[email protected]> wrote:
>> Commit-ID: d95f412200652694e63e64bfd49f0ae274a54479
>> Gitweb: http://git.kernel.org/tip/d95f412200652694e63e64bfd49f0ae274a54479
>> Author: Mike Galbraith<[email protected]>
>> AuthorDate: Tue, 1 Feb 2011 09:50:51 -0500
>> Committer: Ingo Molnar<[email protected]>
>> CommitDate: Thu, 3 Feb 2011 14:20:33 +0100
>>
>> sched: Add yield_to(task, preempt) functionality
>
> I was looking at this patch, while trying to figure out how best to
> use next buddy to solve another unrelated to this cgroup context
> switching problem. While going through this change I see something
> that I don't really understand (inlined below). Not sure whether what
> I am looking at is a bug or I am missing something very obvious.
>
> Thanks in advance for clarification.
>
>>
>> Currently only implemented for fair class tasks.
>>
>> Add a yield_to_task method() to the fair scheduling class. allowing the
>> caller of yield_to() to accelerate another thread in it's thread group,
>> task group.
>>
>
> <snip>
>
>>
>> static void calc_load_account_idle(struct rq *this_rq);
>> @@ -5448,6 +5481,58 @@ void __sched yield(void)
>> }
>> EXPORT_SYMBOL(yield);
>>
>> +/**
>> + * yield_to - yield the current processor to another thread in
>> + * your thread group, or accelerate that thread toward the
>> + * processor it's on.
>> + *
>> + * It's the caller's job to ensure that the target task struct
>> + * can't go away on us before we can do any checks.
>> + *
>> + * Returns true if we indeed boosted the target task.
>> + */
>> +bool __sched yield_to(struct task_struct *p, bool preempt)
>> +{
>> + struct task_struct *curr = current;
>> + struct rq *rq, *p_rq;
>> + unsigned long flags;
>> + bool yielded = 0;
>> +
>> + local_irq_save(flags);
>> + rq = this_rq();
>> +
>> +again:
>> + p_rq = task_rq(p);
>> + double_rq_lock(rq, p_rq);
>> + while (task_rq(p) != p_rq) {
>> + double_rq_unlock(rq, p_rq);
>> + goto again;
>> + }
>> +
>> + if (!curr->sched_class->yield_to_task)
>> + goto out;
>> +
>> + if (curr->sched_class != p->sched_class)
>> + goto out;
>> +
>> + if (task_running(p_rq, p) || p->state)
>> + goto out;
>> +
>> + yielded = curr->sched_class->yield_to_task(rq, p, preempt);
>> + if (yielded)
>> + schedstat_inc(rq, yld_count);
>> +
>> +out:
>> + double_rq_unlock(rq, p_rq);
>> + local_irq_restore(flags);
>> +
>> + if (yielded)
>> + schedule();
>> +
>> + return yielded;
>> +}
>> +EXPORT_SYMBOL_GPL(yield_to);
>> +
>> /*
>> * This task is about to go to sleep on IO. Increment rq->nr_iowait so
>> * that process accounting knows that this is a task in IO wait state.
>> diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
>> index c0fbeb9..0270246 100644
>> --- a/kernel/sched_fair.c
>> +++ b/kernel/sched_fair.c
>> @@ -1975,6 +1975,25 @@ static void yield_task_fair(struct rq *rq)
>> set_skip_buddy(se);
>> }
>>
>> +static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preempt)
>> +{
>> + struct sched_entity *se =&p->se;
>> +
>> + if (!se->on_rq)
>> + return false;
>> +
>> + /* Tell the scheduler that we'd really like pse to run next. */
>> + set_next_buddy(se);
>
> The below comment says about rescheduling p's CPU. But the rq variable
> we have here is the curr_rq and not p_rq. So, should this be done in
> yield_to() with p_rq. I did try to see the discussion on other
> versions of this patch. v3 and before had -
> "resched_task(task_of(p_cfs_rq->curr));" which seems to be correct...

You are correct. We are calling resched_task on the wrong task,
we should call it on p's runqueue's current task...

>> +
>> + /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
>> + if (preempt)
>> + resched_task(rq->curr);



--
All rights reversed

2011-02-28 09:26:37

by Mike Galbraith

[permalink] [raw]
Subject: Re: [tip:sched/core] sched: Add yield_to(task, preempt) functionality

On Sat, 2011-02-26 at 00:44 -0500, Rik van Riel wrote:
> On 02/25/2011 07:43 PM, Venkatesh Pallipadi wrote:

> > The below comment says about rescheduling p's CPU. But the rq variable
> > we have here is the curr_rq and not p_rq. So, should this be done in
> > yield_to() with p_rq. I did try to see the discussion on other
> > versions of this patch. v3 and before had -
> > "resched_task(task_of(p_cfs_rq->curr));" which seems to be correct...
>
> You are correct. We are calling resched_task on the wrong task,
> we should call it on p's runqueue's current task...

Are you going to send a fix?

> >> +
> >> + /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
> >> + if (preempt)
> >> + resched_task(rq->curr);

2011-03-02 00:30:19

by Venkatesh Pallipadi

[permalink] [raw]
Subject: [PATCH] sched: resched proper CPU on yield_to

yield_to_task_fair() has code to resched the CPU of yielding task when the
intention is to resched the CPU of the task that is being yielded to.

Change here fixes the problem and also makes the resched conditional on
rq != p_rq.

I don't have a setup to test this code path. So, this is only compile tested
right now.

Signed-off-by: Venkatesh Pallipadi <[email protected]>
---
kernel/sched.c | 9 ++++++++-
kernel/sched_fair.c | 4 ----
2 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/kernel/sched.c b/kernel/sched.c
index 5fb5aa2..afdbd09 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -5495,8 +5495,15 @@ again:
goto out;

yielded = curr->sched_class->yield_to_task(rq, p, preempt);
- if (yielded)
+ if (yielded) {
schedstat_inc(rq, yld_count);
+ /*
+ * Make p's CPU reschedule; pick_next_entity takes care of
+ * fairness.
+ */
+ if (preempt && rq != p_rq)
+ resched_task(p_rq->curr);
+ }

out:
double_rq_unlock(rq, p_rq);
diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index 36e8f02..1b1c780 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -2001,10 +2001,6 @@ static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preemp
/* Tell the scheduler that we'd really like pse to run next. */
set_next_buddy(se);

- /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
- if (preempt)
- resched_task(rq->curr);
-
yield_task_fair(rq);

return true;
--
1.7.3.1

2011-03-02 03:33:40

by Rik van Riel

[permalink] [raw]
Subject: Re: [PATCH] sched: resched proper CPU on yield_to

On 03/01/2011 07:28 PM, Venkatesh Pallipadi wrote:
> yield_to_task_fair() has code to resched the CPU of yielding task when the
> intention is to resched the CPU of the task that is being yielded to.
>
> Change here fixes the problem and also makes the resched conditional on
> rq != p_rq.

That would result in not rescheduling when current and p are
on the same runqueue, in effect making yield_to a noop for the
easiest case...

When rq != p_rq, we need to ensure both get rescheduled.

We want to have current not run right now (because it is waiting
on a resource that's not available), and we do want p to run.

I'm about to fall over, so I'll go to sleep now.

I can send a patch tomorrow morning, unless you beat me to it :)

--
All rights reversed

2011-03-02 03:37:16

by Venkatesh Pallipadi

[permalink] [raw]
Subject: Re: [PATCH] sched: resched proper CPU on yield_to

On Tue, Mar 1, 2011 at 7:33 PM, Rik van Riel <[email protected]> wrote:
> On 03/01/2011 07:28 PM, Venkatesh Pallipadi wrote:
>>
>> yield_to_task_fair() has code to resched the CPU of yielding task when the
>> intention is to resched the CPU of the task that is being yielded to.
>>
>> Change here fixes the problem and also makes the resched conditional on
>> rq != p_rq.
>
> That would result in not rescheduling when current and p are
> on the same runqueue, in effect making yield_to a noop for the
> easiest case...
>
> When rq != p_rq, we need to ensure both get rescheduled.
>

Yes. There is a schedule() right after this change which should take
care or resched on current CPU. Thats the reason I thought of skipping
resched on current CPU. No?

Thanks,
Venki

> We want to have current not run right now (because it is waiting
> on a resource that's not available), and we do want p to run.
>
> I'm about to fall over, so I'll go to sleep now.
>
> I can send a patch tomorrow morning, unless you beat me to it :)
>
> --
> All rights reversed
>

2011-03-02 03:52:35

by Rik van Riel

[permalink] [raw]
Subject: Re: [PATCH] sched: resched proper CPU on yield_to

On 03/01/2011 07:28 PM, Venkatesh Pallipadi wrote:
> yield_to_task_fair() has code to resched the CPU of yielding task when the
> intention is to resched the CPU of the task that is being yielded to.
>
> Change here fixes the problem and also makes the resched conditional on
> rq != p_rq.
>
> I don't have a setup to test this code path. So, this is only compile tested
> right now.
>
> Signed-off-by: Venkatesh Pallipadi<[email protected]>

Never mind what I wrote before. Your patch is correct.
(and I am way tired)

Reviewed-by: Rik van Riel <[email protected]>

--
All rights reversed

2011-03-04 11:50:39

by Venkatesh Pallipadi

[permalink] [raw]
Subject: [tip:sched/core] sched: Resched proper CPU on yield_to()

Commit-ID: 6d1cafd8b56ea726c10a5a104de57cc3ed8fa953
Gitweb: http://git.kernel.org/tip/6d1cafd8b56ea726c10a5a104de57cc3ed8fa953
Author: Venkatesh Pallipadi <[email protected]>
AuthorDate: Tue, 1 Mar 2011 16:28:21 -0800
Committer: Ingo Molnar <[email protected]>
CommitDate: Fri, 4 Mar 2011 11:14:31 +0100

sched: Resched proper CPU on yield_to()

yield_to_task_fair() has code to resched the CPU of yielding task when the
intention is to resched the CPU of the task that is being yielded to.

Change here fixes the problem and also makes the resched conditional on
rq != p_rq.

Signed-off-by: Venkatesh Pallipadi <[email protected]>
Reviewed-by: Rik van Riel <[email protected]>
Signed-off-by: Peter Zijlstra <[email protected]>
LKML-Reference: <[email protected]>
Signed-off-by: Ingo Molnar <[email protected]>
---
kernel/sched.c | 9 ++++++++-
kernel/sched_fair.c | 4 ----
2 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/kernel/sched.c b/kernel/sched.c
index f303070..61452e8 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -5522,8 +5522,15 @@ again:
goto out;

yielded = curr->sched_class->yield_to_task(rq, p, preempt);
- if (yielded)
+ if (yielded) {
schedstat_inc(rq, yld_count);
+ /*
+ * Make p's CPU reschedule; pick_next_entity takes care of
+ * fairness.
+ */
+ if (preempt && rq != p_rq)
+ resched_task(p_rq->curr);
+ }

out:
double_rq_unlock(rq, p_rq);
diff --git a/kernel/sched_fair.c b/kernel/sched_fair.c
index 1438e13..3f7ec9e 100644
--- a/kernel/sched_fair.c
+++ b/kernel/sched_fair.c
@@ -1987,10 +1987,6 @@ static bool yield_to_task_fair(struct rq *rq, struct task_struct *p, bool preemp
/* Tell the scheduler that we'd really like pse to run next. */
set_next_buddy(se);

- /* Make p's CPU reschedule; pick_next_entity takes care of fairness. */
- if (preempt)
- resched_task(rq->curr);
-
yield_task_fair(rq);

return true;