aboutsummaryrefslogtreecommitdiff
path: root/kernel/sched.c
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/sched.c')
-rw-r--r--kernel/sched.c846
1 files changed, 590 insertions, 256 deletions
diff --git a/kernel/sched.c b/kernel/sched.c
index 4e2f6033565..99e6d850eca 100644
--- a/kernel/sched.c
+++ b/kernel/sched.c
@@ -70,10 +70,13 @@
#include <linux/bootmem.h>
#include <linux/debugfs.h>
#include <linux/ctype.h>
+#include <linux/ftrace.h>
#include <asm/tlb.h>
#include <asm/irq_regs.h>
+#include "sched_cpupri.h"
+
/*
* Convert user-nice values [ -20 ... 0 ... 19 ]
* to static priority [ MAX_RT_PRIO..MAX_PRIO-1 ],
@@ -289,15 +292,15 @@ struct task_group root_task_group;
static DEFINE_PER_CPU(struct sched_entity, init_sched_entity);
/* Default task group's cfs_rq on each cpu */
static DEFINE_PER_CPU(struct cfs_rq, init_cfs_rq) ____cacheline_aligned_in_smp;
-#endif
+#endif /* CONFIG_FAIR_GROUP_SCHED */
#ifdef CONFIG_RT_GROUP_SCHED
static DEFINE_PER_CPU(struct sched_rt_entity, init_sched_rt_entity);
static DEFINE_PER_CPU(struct rt_rq, init_rt_rq) ____cacheline_aligned_in_smp;
-#endif
-#else
+#endif /* CONFIG_RT_GROUP_SCHED */
+#else /* !CONFIG_FAIR_GROUP_SCHED */
#define root_task_group init_task_group
-#endif
+#endif /* CONFIG_FAIR_GROUP_SCHED */
/* task_group_lock serializes add/remove of task groups and also changes to
* a task group's cpu shares.
@@ -307,9 +310,9 @@ static DEFINE_SPINLOCK(task_group_lock);
#ifdef CONFIG_FAIR_GROUP_SCHED
#ifdef CONFIG_USER_SCHED
# define INIT_TASK_GROUP_LOAD (2*NICE_0_LOAD)
-#else
+#else /* !CONFIG_USER_SCHED */
# define INIT_TASK_GROUP_LOAD NICE_0_LOAD
-#endif
+#endif /* CONFIG_USER_SCHED */
/*
* A weight of 0 or 1 can cause arithmetics problems.
@@ -363,6 +366,10 @@ static inline void set_task_rq(struct task_struct *p, unsigned int cpu)
#else
static inline void set_task_rq(struct task_struct *p, unsigned int cpu) { }
+static inline struct task_group *task_group(struct task_struct *p)
+{
+ return NULL;
+}
#endif /* CONFIG_GROUP_SCHED */
@@ -373,6 +380,7 @@ struct cfs_rq {
u64 exec_clock;
u64 min_vruntime;
+ u64 pair_start;
struct rb_root tasks_timeline;
struct rb_node *rb_leftmost;
@@ -401,6 +409,31 @@ struct cfs_rq {
*/
struct list_head leaf_cfs_rq_list;
struct task_group *tg; /* group that "owns" this runqueue */
+
+#ifdef CONFIG_SMP
+ /*
+ * the part of load.weight contributed by tasks
+ */
+ unsigned long task_weight;
+
+ /*
+ * h_load = weight * f(tg)
+ *
+ * Where f(tg) is the recursive weight fraction assigned to
+ * this group.
+ */
+ unsigned long h_load;
+
+ /*
+ * this cpu's part of tg->shares
+ */
+ unsigned long shares;
+
+ /*
+ * load.weight at the time we set shares
+ */
+ unsigned long rq_weight;
+#endif
#endif
};
@@ -452,6 +485,9 @@ struct root_domain {
*/
cpumask_t rto_mask;
atomic_t rto_count;
+#ifdef CONFIG_SMP
+ struct cpupri cpupri;
+#endif
};
/*
@@ -526,6 +562,9 @@ struct rq {
int push_cpu;
/* cpu of this runqueue: */
int cpu;
+ int online;
+
+ unsigned long avg_load_per_task;
struct task_struct *migration_thread;
struct list_head migration_queue;
@@ -607,6 +646,24 @@ static inline void update_rq_clock(struct rq *rq)
# define const_debug static const
#endif
+/**
+ * runqueue_is_locked
+ *
+ * Returns true if the current cpu runqueue is locked.
+ * This interface allows printk to be called with the runqueue lock
+ * held and know whether or not it is OK to wake up the klogd.
+ */
+int runqueue_is_locked(void)
+{
+ int cpu = get_cpu();
+ struct rq *rq = cpu_rq(cpu);
+ int ret;
+
+ ret = spin_is_locked(&rq->lock);
+ put_cpu();
+ return ret;
+}
+
/*
* Debugging: various feature bits
*/
@@ -749,6 +806,12 @@ late_initcall(sched_init_debug);
const_debug unsigned int sysctl_sched_nr_migrate = 32;
/*
+ * ratelimit for updating the group shares.
+ * default: 0.5ms
+ */
+const_debug unsigned int sysctl_sched_shares_ratelimit = 500000;
+
+/*
* period over which we measure -rt task cpu usage in us.
* default: 1s
*/
@@ -775,82 +838,6 @@ static inline u64 global_rt_runtime(void)
return (u64)sysctl_sched_rt_runtime * NSEC_PER_USEC;
}
-unsigned long long time_sync_thresh = 100000;
-
-static DEFINE_PER_CPU(unsigned long long, time_offset);
-static DEFINE_PER_CPU(unsigned long long, prev_cpu_time);
-
-/*
- * Global lock which we take every now and then to synchronize
- * the CPUs time. This method is not warp-safe, but it's good
- * enough to synchronize slowly diverging time sources and thus
- * it's good enough for tracing:
- */
-static DEFINE_SPINLOCK(time_sync_lock);
-static unsigned long long prev_global_time;
-
-static unsigned long long __sync_cpu_clock(unsigned long long time, int cpu)
-{
- /*
- * We want this inlined, to not get tracer function calls
- * in this critical section:
- */
- spin_acquire(&time_sync_lock.dep_map, 0, 0, _THIS_IP_);
- __raw_spin_lock(&time_sync_lock.raw_lock);
-
- if (time < prev_global_time) {
- per_cpu(time_offset, cpu) += prev_global_time - time;
- time = prev_global_time;
- } else {
- prev_global_time = time;
- }
-
- __raw_spin_unlock(&time_sync_lock.raw_lock);
- spin_release(&time_sync_lock.dep_map, 1, _THIS_IP_);
-
- return time;
-}
-
-static unsigned long long __cpu_clock(int cpu)
-{
- unsigned long long now;
-
- /*
- * Only call sched_clock() if the scheduler has already been
- * initialized (some code might call cpu_clock() very early):
- */
- if (unlikely(!scheduler_running))
- return 0;
-
- now = sched_clock_cpu(cpu);
-
- return now;
-}
-
-/*
- * For kernel-internal use: high-speed (but slightly incorrect) per-cpu
- * clock constructed from sched_clock():
- */
-unsigned long long cpu_clock(int cpu)
-{
- unsigned long long prev_cpu_time, time, delta_time;
- unsigned long flags;
-
- local_irq_save(flags);
- prev_cpu_time = per_cpu(prev_cpu_time, cpu);
- time = __cpu_clock(cpu) + per_cpu(time_offset, cpu);
- delta_time = time-prev_cpu_time;
-
- if (unlikely(delta_time > time_sync_thresh)) {
- time = __sync_cpu_clock(time, cpu);
- per_cpu(prev_cpu_time, cpu) = time;
- }
- local_irq_restore(flags);
-
- return time;
-}
-EXPORT_SYMBOL_GPL(cpu_clock);
-
#ifndef prepare_arch_switch
# define prepare_arch_switch(next) do { } while (0)
#endif
@@ -1313,15 +1300,15 @@ void wake_up_idle_cpu(int cpu)
if (!tsk_is_polling(rq->idle))
smp_send_reschedule(cpu);
}
-#endif
+#endif /* CONFIG_NO_HZ */
-#else
+#else /* !CONFIG_SMP */
static void __resched_task(struct task_struct *p, int tif_bit)
{
assert_spin_locked(&task_rq(p)->lock);
set_tsk_thread_flag(p, tif_bit);
}
-#endif
+#endif /* CONFIG_SMP */
#if BITS_PER_LONG == 32
# define WMULT_CONST (~0UL)
@@ -1336,6 +1323,9 @@ static void __resched_task(struct task_struct *p, int tif_bit)
*/
#define SRR(x, y) (((x) + (1UL << ((y) - 1))) >> (y))
+/*
+ * delta *= weight / lw
+ */
static unsigned long
calc_delta_mine(unsigned long delta_exec, unsigned long weight,
struct load_weight *lw)
@@ -1363,12 +1353,6 @@ calc_delta_mine(unsigned long delta_exec, unsigned long weight,
return (unsigned long)min(tmp, (u64)(unsigned long)LONG_MAX);
}
-static inline unsigned long
-calc_delta_fair(unsigned long delta_exec, struct load_weight *lw)
-{
- return calc_delta_mine(delta_exec, NICE_0_LOAD, lw);
-}
-
static inline void update_load_add(struct load_weight *lw, unsigned long inc)
{
lw->weight += inc;
@@ -1479,17 +1463,211 @@ static inline void dec_cpu_load(struct rq *rq, unsigned long load)
#ifdef CONFIG_SMP
static unsigned long source_load(int cpu, int type);
static unsigned long target_load(int cpu, int type);
-static unsigned long cpu_avg_load_per_task(int cpu);
static int task_hot(struct task_struct *p, u64 now, struct sched_domain *sd);
-#else /* CONFIG_SMP */
+
+static unsigned long cpu_avg_load_per_task(int cpu)
+{
+ struct rq *rq = cpu_rq(cpu);
+
+ if (rq->nr_running)
+ rq->avg_load_per_task = rq->load.weight / rq->nr_running;
+
+ return rq->avg_load_per_task;
+}
#ifdef CONFIG_FAIR_GROUP_SCHED
-static void cfs_rq_set_shares(struct cfs_rq *cfs_rq, unsigned long shares)
+
+typedef void (*tg_visitor)(struct task_group *, int, struct sched_domain *);
+
+/*
+ * Iterate the full tree, calling @down when first entering a node and @up when
+ * leaving it for the final time.
+ */
+static void
+walk_tg_tree(tg_visitor down, tg_visitor up, int cpu, struct sched_domain *sd)
+{
+ struct task_group *parent, *child;
+
+ rcu_read_lock();
+ parent = &root_task_group;
+down:
+ (*down)(parent, cpu, sd);
+ list_for_each_entry_rcu(child, &parent->children, siblings) {
+ parent = child;
+ goto down;
+
+up:
+ continue;
+ }
+ (*up)(parent, cpu, sd);
+
+ child = parent;
+ parent = parent->parent;
+ if (parent)
+ goto up;
+ rcu_read_unlock();
+}
+
+static void __set_se_shares(struct sched_entity *se, unsigned long shares);
+
+/*
+ * Calculate and set the cpu's group shares.
+ */
+static void
+__update_group_shares_cpu(struct task_group *tg, int cpu,
+ unsigned long sd_shares, unsigned long sd_rq_weight)
+{
+ int boost = 0;
+ unsigned long shares;
+ unsigned long rq_weight;
+
+ if (!tg->se[cpu])
+ return;
+
+ rq_weight = tg->cfs_rq[cpu]->load.weight;
+
+ /*
+ * If there are currently no tasks on the cpu pretend there is one of
+ * average load so that when a new task gets to run here it will not
+ * get delayed by group starvation.
+ */
+ if (!rq_weight) {
+ boost = 1;
+ rq_weight = NICE_0_LOAD;
+ }
+
+ if (unlikely(rq_weight > sd_rq_weight))
+ rq_weight = sd_rq_weight;
+
+ /*
+ * \Sum shares * rq_weight
+ * shares = -----------------------
+ * \Sum rq_weight
+ *
+ */
+ shares = (sd_shares * rq_weight) / (sd_rq_weight + 1);
+
+ /*
+ * record the actual number of shares, not the boosted amount.
+ */
+ tg->cfs_rq[cpu]->shares = boost ? 0 : shares;
+ tg->cfs_rq[cpu]->rq_weight = rq_weight;
+
+ if (shares < MIN_SHARES)
+ shares = MIN_SHARES;
+ else if (shares > MAX_SHARES)
+ shares = MAX_SHARES;
+
+ __set_se_shares(tg->se[cpu], shares);
+}
+
+/*
+ * Re-compute the task group their per cpu shares over the given domain.
+ * This needs to be done in a bottom-up fashion because the rq weight of a
+ * parent group depends on the shares of its child groups.
+ */
+static void
+tg_shares_up(struct task_group *tg, int cpu, struct sched_domain *sd)
+{
+ unsigned long rq_weight = 0;
+ unsigned long shares = 0;
+ int i;
+
+ for_each_cpu_mask(i, sd->span) {
+ rq_weight += tg->cfs_rq[i]->load.weight;
+ shares += tg->cfs_rq[i]->shares;
+ }
+
+ if ((!shares && rq_weight) || shares > tg->shares)
+ shares = tg->shares;
+
+ if (!sd->parent || !(sd->parent->flags & SD_LOAD_BALANCE))
+ shares = tg->shares;
+
+ if (!rq_weight)
+ rq_weight = cpus_weight(sd->span) * NICE_0_LOAD;
+
+ for_each_cpu_mask(i, sd->span) {
+ struct rq *rq = cpu_rq(i);
+ unsigned long flags;
+
+ spin_lock_irqsave(&rq->lock, flags);
+ __update_group_shares_cpu(tg, i, shares, rq_weight);
+ spin_unlock_irqrestore(&rq->lock, flags);
+ }
+}
+
+/*
+ * Compute the cpu's hierarchical load factor for each task group.
+ * This needs to be done in a top-down fashion because the load of a child
+ * group is a fraction of its parents load.
+ */
+static void
+tg_load_down(struct task_group *tg, int cpu, struct sched_domain *sd)
+{
+ unsigned long load;
+
+ if (!tg->parent) {
+ load = cpu_rq(cpu)->load.weight;
+ } else {
+ load = tg->parent->cfs_rq[cpu]->h_load;
+ load *= tg->cfs_rq[cpu]->shares;
+ load /= tg->parent->cfs_rq[cpu]->load.weight + 1;
+ }
+
+ tg->cfs_rq[cpu]->h_load = load;
+}
+
+static void
+tg_nop(struct task_group *tg, int cpu, struct sched_domain *sd)
+{
+}
+
+static void update_shares(struct sched_domain *sd)
+{
+ u64 now = cpu_clock(raw_smp_processor_id());
+ s64 elapsed = now - sd->last_update;
+
+ if (elapsed >= (s64)(u64)sysctl_sched_shares_ratelimit) {
+ sd->last_update = now;
+ walk_tg_tree(tg_nop, tg_shares_up, 0, sd);
+ }
+}
+
+static void update_shares_locked(struct rq *rq, struct sched_domain *sd)
+{
+ spin_unlock(&rq->lock);
+ update_shares(sd);
+ spin_lock(&rq->lock);
+}
+
+static void update_h_load(int cpu)
{
+ walk_tg_tree(tg_load_down, tg_nop, cpu, NULL);
}
+
+#else
+
+static inline void update_shares(struct sched_domain *sd)
+{
+}
+
+static inline void update_shares_locked(struct rq *rq, struct sched_domain *sd)
+{
+}
+
#endif
-#endif /* CONFIG_SMP */
+#endif
+
+#ifdef CONFIG_FAIR_GROUP_SCHED
+static void cfs_rq_set_shares(struct cfs_rq *cfs_rq, unsigned long shares)
+{
+#ifdef CONFIG_SMP
+ cfs_rq->shares = shares;
+#endif
+}
+#endif
#include "sched_stats.h"
#include "sched_idletask.c"
@@ -1500,27 +1678,17 @@ static void cfs_rq_set_shares(struct cfs_rq *cfs_rq, unsigned long shares)
#endif
#define sched_class_highest (&rt_sched_class)
+#define for_each_class(class) \
+ for (class = sched_class_highest; class; class = class->next)
-static inline void inc_load(struct rq *rq, const struct task_struct *p)
-{
- update_load_add(&rq->load, p->se.load.weight);
-}
-
-static inline void dec_load(struct rq *rq, const struct task_struct *p)
-{
- update_load_sub(&rq->load, p->se.load.weight);
-}
-
-static void inc_nr_running(struct task_struct *p, struct rq *rq)
+static void inc_nr_running(struct rq *rq)
{
rq->nr_running++;
- inc_load(rq, p);
}
-static void dec_nr_running(struct task_struct *p, struct rq *rq)
+static void dec_nr_running(struct rq *rq)
{
rq->nr_running--;
- dec_load(rq, p);
}
static void set_load_weight(struct task_struct *p)
@@ -1544,6 +1712,12 @@ static void set_load_weight(struct task_struct *p)
p->se.load.inv_weight = prio_to_wmult[p->static_prio - MAX_RT_PRIO];
}
+static void update_avg(u64 *avg, u64 sample)
+{
+ s64 diff = sample - *avg;
+ *avg += diff >> 3;
+}
+
static void enqueue_task(struct rq *rq, struct task_struct *p, int wakeup)
{
sched_info_queued(p);
@@ -1553,6 +1727,13 @@ static void enqueue_task(struct rq *rq, struct task_struct *p, int wakeup)
static void dequeue_task(struct rq *rq, struct task_struct *p, int sleep)
{
+ if (sleep && p->se.last_wakeup) {
+ update_avg(&p->se.avg_overlap,
+ p->se.sum_exec_runtime - p->se.last_wakeup);
+ p->se.last_wakeup = 0;
+ }
+
+ sched_info_dequeued(p);
p->sched_class->dequeue_task(rq, p, sleep);
p->se.on_rq = 0;
}
@@ -1612,7 +1793,7 @@ static void activate_task(struct rq *rq, struct task_struct *p, int wakeup)
rq->nr_uninterruptible--;
enqueue_task(rq, p, wakeup);
- inc_nr_running(p, rq);
+ inc_nr_running(rq);
}
/*
@@ -1624,7 +1805,7 @@ static void deactivate_task(struct rq *rq, struct task_struct *p, int sleep)
rq->nr_uninterruptible++;
dequeue_task(rq, p, sleep);
- dec_nr_running(p, rq);
+ dec_nr_running(rq);
}
/**
@@ -1636,12 +1817,6 @@ inline int task_curr(const struct task_struct *p)
return cpu_curr(task_cpu(p)) == p;
}
-/* Used instead of source_load when we know the type == 0 */
-unsigned long weighted_cpuload(const int cpu)
-{
- return cpu_rq(cpu)->load.weight;
-}
-
static inline void __set_task_cpu(struct task_struct *p, unsigned int cpu)
{
set_task_rq(p, cpu);
@@ -1670,6 +1845,12 @@ static inline void check_class_changed(struct rq *rq, struct task_struct *p,
#ifdef CONFIG_SMP
+/* Used instead of source_load when we know the type == 0 */
+static unsigned long weighted_cpuload(const int cpu)
+{
+ return cpu_rq(cpu)->load.weight;
+}
+
/*
* Is this task likely cache-hot:
*/
@@ -1880,7 +2061,7 @@ static unsigned long source_load(int cpu, int type)
struct rq *rq = cpu_rq(cpu);
unsigned long total = weighted_cpuload(cpu);
- if (type == 0)
+ if (type == 0 || !sched_feat(LB_BIAS))
return total;
return min(rq->cpu_load[type-1], total);
@@ -1895,25 +2076,13 @@ static unsigned long target_load(int cpu, int type)
struct rq *rq = cpu_rq(cpu);
unsigned long total = weighted_cpuload(cpu);
- if (type == 0)
+ if (type == 0 || !sched_feat(LB_BIAS))
return total;
return max(rq->cpu_load[type-1], total);
}
/*
- * Return the average load per task on the cpu's run queue
- */
-static unsigned long cpu_avg_load_per_task(int cpu)
-{
- struct rq *rq = cpu_rq(cpu);
- unsigned long total = weighted_cpuload(cpu);
- unsigned long n = rq->nr_running;
-
- return n ? total / n : SCHED_LOAD_SCALE;
-}
-
-/*
* find_idlest_group finds and returns the least busy CPU group within the
* domain.
*/
@@ -2019,6 +2188,9 @@ static int sched_balance_self(int cpu, int flag)
sd = tmp;
}
+ if (sd)
+ update_shares(sd);
+
while (sd) {
cpumask_t span, tmpmask;
struct sched_group *group;
@@ -2085,6 +2257,22 @@ static int try_to_wake_up(struct task_struct *p, unsigned int state, int sync)
if (!sched_feat(SYNC_WAKEUPS))
sync = 0;
+#ifdef CONFIG_SMP
+ if (sched_feat(LB_WAKEUP_UPDATE)) {
+ struct sched_domain *sd;
+
+ this_cpu = raw_smp_processor_id();
+ cpu = task_cpu(p);
+
+ for_each_domain(this_cpu, sd) {
+ if (cpu_isset(cpu, sd->span)) {
+ update_shares(sd);
+ break;
+ }
+ }
+ }
+#endif
+
smp_wmb();
rq = task_rq_lock(p, &flags);
old_state = p->state;
@@ -2131,7 +2319,7 @@ static int try_to_wake_up(struct task_struct *p, unsigned int state, int sync)
}
}
}
-#endif
+#endif /* CONFIG_SCHEDSTATS */
out_activate:
#endif /* CONFIG_SMP */
@@ -2149,6 +2337,9 @@ out_activate:
success = 1;
out_running:
+ trace_mark(kernel_sched_wakeup,
+ "pid %d state %ld ## rq %p task %p rq->curr %p",
+ p->pid, p->state, rq, p, rq->curr);
check_preempt_curr(rq, p);
p->state = TASK_RUNNING;
@@ -2157,6 +2348,8 @@ out_running:
p->sched_class->task_wake_up(rq, p);
#endif
out:
+ current->se.last_wakeup = current->se.sum_exec_runtime;
+
task_rq_unlock(rq, &flags);
return success;
@@ -2277,8 +2470,11 @@ void wake_up_new_task(struct task_struct *p, unsigned long clone_flags)
* management (if any):
*/
p->sched_class->task_new(rq, p);
- inc_nr_running(p, rq);
+ inc_nr_running(rq);
}
+ trace_mark(kernel_sched_wakeup_new,
+ "pid %d state %ld ## rq %p task %p rq->curr %p",
+ p->pid, p->state, rq, p, rq->curr);
check_preempt_curr(rq, p);
#ifdef CONFIG_SMP
if (p->sched_class->task_wake_up)
@@ -2331,7 +2527,7 @@ fire_sched_out_preempt_notifiers(struct task_struct *curr,
notifier->ops->sched_out(notifier, next);
}
-#else
+#else /* !CONFIG_PREEMPT_NOTIFIERS */
static void fire_sched_in_preempt_notifiers(struct task_struct *curr)
{
@@ -2343,7 +2539,7 @@ fire_sched_out_preempt_notifiers(struct task_struct *curr,
{
}
-#endif
+#endif /* CONFIG_PREEMPT_NOTIFIERS */
/**
* prepare_task_switch - prepare to switch tasks
@@ -2451,6 +2647,11 @@ context_switch(struct rq *rq, struct task_struct *prev,
struct mm_struct *mm, *oldmm;
prepare_task_switch(rq, prev, next);
+ trace_mark(kernel_sched_schedule,
+ "prev_pid %d next_pid %d prev_state %ld "
+ "## rq %p prev %p next %p",
+ prev->pid, next->pid, prev->state,
+ rq, prev, next);
mm = next->mm;
oldmm = prev->active_mm;
/*
@@ -2785,7 +2986,7 @@ balance_tasks(struct rq *this_rq, int this_cpu, struct rq *busiest,
enum cpu_idle_type idle, int *all_pinned,
int *this_best_prio, struct rq_iterator *iterator)
{
- int loops = 0, pulled = 0, pinned = 0, skip_for_load;
+ int loops = 0, pulled = 0, pinned = 0;
struct task_struct *p;
long rem_load_move = max_load_move;
@@ -2801,14 +3002,8 @@ balance_tasks(struct rq *this_rq, int this_cpu, struct rq *busiest,
next:
if (!p || loops++ > sysctl_sched_nr_migrate)
goto out;
- /*
- * To help distribute high priority tasks across CPUs we don't
- * skip a task if it will be the highest priority task (i.e. smallest
- * prio value) on its new queue regardless of its load weight
- */
- skip_for_load = (p->se.load.weight >> 1) > rem_load_move +
- SCHED_LOAD_SCALE_FUZZ;
- if ((skip_for_load && p->prio >= *this_best_prio) ||
+
+ if ((p->se.load.weight >> 1) > rem_load_move ||
!can_migrate_task(p, busiest, this_cpu, sd, idle, &pinned)) {
p = iterator->next(iterator->arg);
goto next;
@@ -2863,6 +3058,10 @@ static int move_tasks(struct rq *this_rq, int this_cpu, struct rq *busiest,
max_load_move - total_load_moved,
sd, idle, all_pinned, &this_best_prio);
class = class->next;
+
+ if (idle == CPU_NEWLY_IDLE && this_rq->nr_running)
+ break;
+
} while (class && max_load_move > total_load_moved);
return total_load_moved > 0;
@@ -2939,6 +3138,7 @@ find_busiest_group(struct sched_domain *sd, int this_cpu,
max_load = this_load = total_load = total_pwr = 0;
busiest_load_per_task = busiest_nr_running = 0;
this_load_per_task = this_nr_running = 0;
+
if (idle == CPU_NOT_IDLE)
load_idx = sd->busy_idx;
else if (idle == CPU_NEWLY_IDLE)
@@ -2953,6 +3153,8 @@ find_busiest_group(struct sched_domain *sd, int this_cpu,
int __group_imb = 0;
unsigned int balance_cpu = -1, first_idle_cpu = 0;
unsigned long sum_nr_running, sum_weighted_load;
+ unsigned long sum_avg_load_per_task;
+ unsigned long avg_load_per_task;
local_group = cpu_isset(this_cpu, group->cpumask);
@@ -2961,6 +3163,8 @@ find_busiest_group(struct sched_domain *sd, int this_cpu,
/* Tally up the load of all CPUs in the group */
sum_weighted_load = sum_nr_running = avg_load = 0;
+ sum_avg_load_per_task = avg_load_per_task = 0;
+
max_cpu_load = 0;
min_cpu_load = ~0UL;
@@ -2994,6 +3198,8 @@ find_busiest_group(struct sched_domain *sd, int this_cpu,
avg_load += load;
sum_nr_running += rq->nr_running;
sum_weighted_load += weighted_cpuload(i);
+
+ sum_avg_load_per_task += cpu_avg_load_per_task(i);
}
/*
@@ -3015,7 +3221,20 @@ find_busiest_group(struct sched_domain *sd, int this_cpu,
avg_load = sg_div_cpu_power(group,
avg_load * SCHED_LOAD_SCALE);
- if ((max_cpu_load - min_cpu_load) > SCHED_LOAD_SCALE)
+
+ /*
+ * Consider the group unbalanced when the imbalance is larger
+ * than the average weight of two tasks.
+ *
+ * APZ: with cgroup the avg task weight can vary wildly and
+ * might not be a suitable number - should we keep a
+ * normalized nr_running number somewhere that negates
+ * the hierarchy?
+ */
+ avg_load_per_task = sg_div_cpu_power(group,
+ sum_avg_load_per_task * SCHED_LOAD_SCALE);
+
+ if ((max_cpu_load - min_cpu_load) > 2*avg_load_per_task)
__group_imb = 1;
group_capacity = group->__cpu_power / SCHED_LOAD_SCALE;
@@ -3156,9 +3375,9 @@ small_imbalance:
if (busiest_load_per_task > this_load_per_task)
imbn = 1;
} else
- this_load_per_task = SCHED_LOAD_SCALE;
+ this_load_per_task = cpu_avg_load_per_task(this_cpu);
- if (max_load - this_load + SCHED_LOAD_SCALE_FUZZ >=
+ if (max_load - this_load + 2*busiest_load_per_task >=
busiest_load_per_task * imbn) {
*imbalance = busiest_load_per_task;
return busiest;
@@ -3284,6 +3503,7 @@ static int load_balance(int this_cpu, struct rq *this_rq,
schedstat_inc(sd, lb_count[idle]);
redo:
+ update_shares(sd);
group = find_busiest_group(sd, this_cpu, &imbalance, idle, &sd_idle,
cpus, balance);
@@ -3386,8 +3606,9 @@ redo:
if (!ld_moved && !sd_idle && sd->flags & SD_SHARE_CPUPOWER &&
!test_sd_parent(sd, SD_POWERSAVINGS_BALANCE))
- return -1;
- return ld_moved;
+ ld_moved = -1;
+
+ goto out;
out_balanced:
schedstat_inc(sd, lb_balanced[idle]);
@@ -3402,8 +3623,13 @@ out_one_pinned:
if (!sd_idle && sd->flags & SD_SHARE_CPUPOWER &&
!test_sd_parent(sd, SD_POWERSAVINGS_BALANCE))
- return -1;
- return 0;
+ ld_moved = -1;
+ else
+ ld_moved = 0;
+out:
+ if (ld_moved)
+ update_shares(sd);
+ return ld_moved;
}
/*
@@ -3438,6 +3664,7 @@ load_balance_newidle(int this_cpu, struct rq *this_rq, struct sched_domain *sd,
schedstat_inc(sd, lb_count[CPU_NEWLY_IDLE]);
redo:
+ update_shares_locked(this_rq, sd);
group = find_busiest_group(sd, this_cpu, &imbalance, CPU_NEWLY_IDLE,
&sd_idle, cpus, NULL);
if (!group) {
@@ -3481,6 +3708,7 @@ redo:
} else
sd->nr_balance_failed = 0;
+ update_shares_locked(this_rq, sd);
return ld_moved;
out_balanced:
@@ -3672,6 +3900,7 @@ static void rebalance_domains(int cpu, enum cpu_idle_type idle)
/* Earliest time when we have to do rebalance again */
unsigned long next_balance = jiffies + 60*HZ;
int update_next_balance = 0;
+ int need_serialize;
cpumask_t tmp;
for_each_domain(cpu, sd) {
@@ -3689,8 +3918,9 @@ static void rebalance_domains(int cpu, enum cpu_idle_type idle)
if (interval > HZ*NR_CPUS/10)
interval = HZ*NR_CPUS/10;
+ need_serialize = sd->flags & SD_SERIALIZE;
- if (sd->flags & SD_SERIALIZE) {
+ if (need_serialize) {
if (!spin_trylock(&balancing))
goto out;
}
@@ -3706,7 +3936,7 @@ static void rebalance_domains(int cpu, enum cpu_idle_type idle)
}
sd->last_balance = jiffies;
}
- if (sd->flags & SD_SERIALIZE)
+ if (need_serialize)
spin_unlock(&balancing);
out:
if (time_after(next_balance, sd->last_balance + interval)) {
@@ -4021,26 +4251,44 @@ void scheduler_tick(void)
#endif
}
-#if defined(CONFIG_PREEMPT) && defined(CONFIG_DEBUG_PREEMPT)
+#if defined(CONFIG_PREEMPT) && (defined(CONFIG_DEBUG_PREEMPT) || \
+ defined(CONFIG_PREEMPT_TRACER))
+
+static inline unsigned long get_parent_ip(unsigned long addr)
+{
+ if (in_lock_functions(addr)) {
+ addr = CALLER_ADDR2;
+ if (in_lock_functions(addr))
+ addr = CALLER_ADDR3;
+ }
+ return addr;
+}
void __kprobes add_preempt_count(int val)
{
+#ifdef CONFIG_DEBUG_PREEMPT
/*
* Underflow?
*/
if (DEBUG_LOCKS_WARN_ON((preempt_count() < 0)))
return;
+#endif
preempt_count() += val;
+#ifdef CONFIG_DEBUG_PREEMPT
/*
* Spinlock count overflowing soon?
*/
DEBUG_LOCKS_WARN_ON((preempt_count() & PREEMPT_MASK) >=
PREEMPT_MASK - 10);
+#endif
+ if (preempt_count() == val)
+ trace_preempt_off(CALLER_ADDR0, get_parent_ip(CALLER_ADDR1));
}
EXPORT_SYMBOL(add_preempt_count);
void __kprobes sub_preempt_count(int val)
{
+#ifdef CONFIG_DEBUG_PREEMPT
/*
* Underflow?
*/
@@ -4052,7 +4300,10 @@ void __kprobes sub_preempt_count(int val)
if (DEBUG_LOCKS_WARN_ON((val < PREEMPT_MASK) &&
!(preempt_count() & PREEMPT_MASK)))
return;
+#endif
+ if (preempt_count() == val)
+ trace_preempt_on(CALLER_ADDR0, get_parent_ip(CALLER_ADDR1));
preempt_count() -= val;
}
EXPORT_SYMBOL(sub_preempt_count);
@@ -4070,6 +4321,7 @@ static noinline void __schedule_bug(struct task_struct *prev)
prev->comm, prev->pid, preempt_count());
debug_show_held_locks(prev);
+ print_modules();
if (irqs_disabled())
print_irqtrace_events(prev);
@@ -4143,7 +4395,7 @@ asmlinkage void __sched schedule(void)
struct task_struct *prev, *next;
unsigned long *switch_count;
struct rq *rq;
- int cpu;
+ int cpu, hrtick = sched_feat(HRTICK);
need_resched:
preempt_disable();
@@ -4158,7 +4410,8 @@ need_resched_nonpreemptible:
schedule_debug(prev);
- hrtick_clear(rq);
+ if (hrtick)
+ hrtick_clear(rq);
/*
* Do the rq-clock update outside the rq lock:
@@ -4204,7 +4457,8 @@ need_resched_nonpreemptible:
} else
spin_unlock_irq(&rq->lock);
- hrtick_set(rq);
+ if (hrtick)
+ hrtick_set(rq);
if (unlikely(reacquire_kernel_lock(current) < 0))
goto need_resched_nonpreemptible;
@@ -4586,10 +4840,8 @@ void set_user_nice(struct task_struct *p, long nice)
goto out_unlock;
}
on_rq = p->se.on_rq;
- if (on_rq) {
+ if (on_rq)
dequeue_task(rq, p, 0);
- dec_load(rq, p);
- }
p->static_prio = NICE_TO_PRIO(nice);
set_load_weight(p);
@@ -4599,7 +4851,6 @@ void set_user_nice(struct task_struct *p, long nice)
if (on_rq) {
enqueue_task(rq, p, 0);
- inc_load(rq, p);
/*
* If the task increased its priority or is running and
* lowered its priority, then reschedule its CPU:
@@ -4744,16 +4995,8 @@ __setscheduler(struct rq *rq, struct task_struct *p, int policy, int prio)
set_load_weight(p);
}
-/**
- * sched_setscheduler - change the scheduling policy and/or RT priority of a thread.
- * @p: the task in question.
- * @policy: new policy.
- * @param: structure containing the new RT priority.
- *
- * NOTE that the task may be already dead.
- */
-int sched_setscheduler(struct task_struct *p, int policy,
- struct sched_param *param)
+static int __sched_setscheduler(struct task_struct *p, int policy,
+ struct sched_param *param, bool user)
{
int retval, oldprio, oldpolicy = -1, on_rq, running;
unsigned long flags;
@@ -4785,7 +5028,7 @@ recheck:
/*
* Allow unprivileged RT tasks to decrease priority:
*/
- if (!capable(CAP_SYS_NICE)) {
+ if (user && !capable(CAP_SYS_NICE)) {
if (rt_policy(policy)) {
unsigned long rlim_rtprio;
@@ -4821,7 +5064,8 @@ recheck:
* Do not allow realtime tasks into groups that have no runtime
* assigned.
*/
- if (rt_policy(policy) && task_group(p)->rt_bandwidth.rt_runtime == 0)
+ if (user
+ && rt_policy(policy) && task_group(p)->rt_bandwidth.rt_runtime == 0)
return -EPERM;
#endif
@@ -4870,8 +5114,39 @@ recheck:
return 0;
}
+
+/**
+ * sched_setscheduler - change the scheduling policy and/or RT priority of a thread.
+ * @p: the task in question.
+ * @policy: new policy.
+ * @param: structure containing the new RT priority.
+ *
+ * NOTE that the task may be already dead.
+ */
+int sched_setscheduler(struct task_struct *p, int policy,
+ struct sched_param *param)
+{
+ return __sched_setscheduler(p, policy, param, true);
+}
EXPORT_SYMBOL_GPL(sched_setscheduler);
+/**
+ * sched_setscheduler_nocheck - change the scheduling policy and/or RT priority of a thread from kernelspace.
+ * @p: the task in question.
+ * @policy: new policy.
+ * @param: structure containing the new RT priority.
+ *
+ * Just like sched_setscheduler, only don't bother checking if the
+ * current context has permission. For example, this is needed in
+ * stop_machine(): we create temporary high priority worker threads,
+ * but our caller might not have that capability.
+ */
+int sched_setscheduler_nocheck(struct task_struct *p, int policy,
+ struct sched_param *param)
+{
+ return __sched_setscheduler(p, policy, param, false);
+}
+
static int
do_sched_setscheduler(pid_t pid, int policy, struct sched_param __user *param)
{
@@ -5070,24 +5345,6 @@ asmlinkage long sys_sched_setaffinity(pid_t pid, unsigned int len,
return sched_setaffinity(pid, &new_mask);
}
-/*
- * Represents all cpu's present in the system
- * In systems capable of hotplug, this map could dynamically grow
- * as new cpu's are detected in the system via any platform specific
- * method, such as ACPI for e.g.
- */
-
-cpumask_t cpu_present_map __read_mostly;
-EXPORT_SYMBOL(cpu_present_map);
-
-#ifndef CONFIG_SMP
-cpumask_t cpu_online_map __read_mostly = CPU_MASK_ALL;
-EXPORT_SYMBOL(cpu_online_map);
-
-cpumask_t cpu_possible_map __read_mostly = CPU_MASK_ALL;
-EXPORT_SYMBOL(cpu_possible_map);
-#endif
-
long sched_getaffinity(pid_t pid, cpumask_t *mask)
{
struct task_struct *p;
@@ -5384,7 +5641,7 @@ out_unlock:
return retval;
}
-static const char stat_nam[] = "RSDTtZX";
+static const char stat_nam[] = TASK_STATE_TO_CHAR_STR;
void sched_show_task(struct task_struct *p)
{
@@ -5571,6 +5828,12 @@ int set_cpus_allowed_ptr(struct task_struct *p, const cpumask_t *new_mask)
goto out;
}
+ if (unlikely((p->flags & PF_THREAD_BOUND) && p != current &&
+ !cpus_equal(p->cpus_allowed, *new_mask))) {
+ ret = -EINVAL;
+ goto out;
+ }
+
if (p->sched_class->set_cpus_allowed)
p->sched_class->set_cpus_allowed(p, new_mask);
else {
@@ -6060,6 +6323,36 @@ static void unregister_sched_domain_sysctl(void)
}
#endif
+static void set_rq_online(struct rq *rq)
+{
+ if (!rq->online) {
+ const struct sched_class *class;
+
+ cpu_set(rq->cpu, rq->rd->online);
+ rq->online = 1;
+
+ for_each_class(class) {
+ if (class->rq_online)
+ class->rq_online(rq);
+ }
+ }
+}
+
+static void set_rq_offline(struct rq *rq)
+{
+ if (rq->online) {
+ const struct sched_class *class;
+
+ for_each_class(class) {
+ if (class->rq_offline)
+ class->rq_offline(rq);
+ }
+
+ cpu_clear(rq->cpu, rq->rd->online);
+ rq->online = 0;
+ }
+}
+
/*
* migration_call - callback that gets triggered when a CPU is added.
* Here we can start up the necessary migration thread for the new CPU.
@@ -6097,7 +6390,8 @@ migration_call(struct notifier_block *nfb, unsigned long action, void *hcpu)
spin_lock_irqsave(&rq->lock, flags);
if (rq->rd) {
BUG_ON(!cpu_isset(cpu, rq->rd->span));
- cpu_set(cpu, rq->rd->online);
+
+ set_rq_online(rq);
}
spin_unlock_irqrestore(&rq->lock, flags);
break;
@@ -6158,7 +6452,7 @@ migration_call(struct notifier_block *nfb, unsigned long action, void *hcpu)
spin_lock_irqsave(&rq->lock, flags);
if (rq->rd) {
BUG_ON(!cpu_isset(cpu, rq->rd->span));
- cpu_clear(cpu, rq->rd->online);
+ set_rq_offline(rq);
}
spin_unlock_irqrestore(&rq->lock, flags);
break;
@@ -6192,6 +6486,28 @@ void __init migration_init(void)
#ifdef CONFIG_SCHED_DEBUG
+static inline const char *sd_level_to_string(enum sched_domain_level lvl)
+{
+ switch (lvl) {
+ case SD_LV_NONE:
+ return "NONE";
+ case SD_LV_SIBLING:
+ return "SIBLING";
+ case SD_LV_MC:
+ return "MC";
+ case SD_LV_CPU:
+ return "CPU";
+ case SD_LV_NODE:
+ return "NODE";
+ case SD_LV_ALLNODES:
+ return "ALLNODES";
+ case SD_LV_MAX:
+ return "MAX";
+
+ }
+ return "MAX";
+}
+
static int sched_domain_debug_one(struct sched_domain *sd, int cpu, int level,
cpumask_t *groupmask)
{
@@ -6211,7 +6527,8 @@ static int sched_domain_debug_one(struct sched_domain *sd, int cpu, int level,
return -1;
}
- printk(KERN_CONT "span %s\n", str);
+ printk(KERN_CONT "span %s level %s\n",
+ str, sd_level_to_string(sd->level));
if (!cpu_isset(cpu, sd->span)) {
printk(KERN_ERR "ERROR: domain->span does not contain "
@@ -6295,9 +6612,9 @@ static void sched_domain_debug(struct sched_domain *sd, int cpu)
}
kfree(groupmask);
}
-#else
+#else /* !CONFIG_SCHED_DEBUG */
# define sched_domain_debug(sd, cpu) do { } while (0)
-#endif
+#endif /* CONFIG_SCHED_DEBUG */
static int sd_degenerate(struct sched_domain *sd)
{
@@ -6357,20 +6674,16 @@ sd_parent_degenerate(struct sched_domain *sd, struct sched_domain *parent)
static void rq_attach_root(struct rq *rq, struct root_domain *rd)
{
unsigned long flags;
- const struct sched_class *class;
spin_lock_irqsave(&rq->lock, flags);
if (rq->rd) {
struct root_domain *old_rd = rq->rd;
- for (class = sched_class_highest; class; class = class->next) {
- if (class->leave_domain)
- class->leave_domain(rq);
- }
+ if (cpu_isset(rq->cpu, old_rd->online))
+ set_rq_offline(rq);
cpu_clear(rq->cpu, old_rd->span);
- cpu_clear(rq->cpu, old_rd->online);
if (atomic_dec_and_test(&old_rd->refcount))
kfree(old_rd);
@@ -6381,12 +6694,7 @@ static void rq_attach_root(struct rq *rq, struct root_domain *rd)
cpu_set(rq->cpu, rd->span);
if (cpu_isset(rq->cpu, cpu_online_map))
- cpu_set(rq->cpu, rd->online);
-
- for (class = sched_class_highest; class; class = class->next) {
- if (class->join_domain)
- class->join_domain(rq);
- }
+ set_rq_online(rq);
spin_unlock_irqrestore(&rq->lock, flags);
}
@@ -6397,6 +6705,8 @@ static void init_rootdomain(struct root_domain *rd)
cpus_clear(rd->span);
cpus_clear(rd->online);
+
+ cpupri_init(&rd->cpupri);
}
static void init_defrootdomain(void)
@@ -6539,9 +6849,9 @@ static int find_next_best_node(int node, nodemask_t *used_nodes)
min_val = INT_MAX;
- for (i = 0; i < MAX_NUMNODES; i++) {
+ for (i = 0; i < nr_node_ids; i++) {
/* Start at @node */
- n = (node + i) % MAX_NUMNODES;
+ n = (node + i) % nr_node_ids;
if (!nr_cpus_node(n))
continue;
@@ -6591,7 +6901,7 @@ static void sched_domain_node_span(int node, cpumask_t *span)
cpus_or(*span, *span, *nodemask);
}
}
-#endif
+#endif /* CONFIG_NUMA */
int sched_smt_power_savings = 0, sched_mc_power_savings = 0;
@@ -6610,7 +6920,7 @@ cpu_to_cpu_group(int cpu, const cpumask_t *cpu_map, struct sched_group **sg,
*sg = &per_cpu(sched_group_cpus, cpu);
return cpu;
}
-#endif
+#endif /* CONFIG_SCHED_SMT */
/*
* multi-core sched-domains:
@@ -6618,7 +6928,7 @@ cpu_to_cpu_group(int cpu, const cpumask_t *cpu_map, struct sched_group **sg,
#ifdef CONFIG_SCHED_MC
static DEFINE_PER_CPU(struct sched_domain, core_domains);
static DEFINE_PER_CPU(struct sched_group, sched_group_core);
-#endif
+#endif /* CONFIG_SCHED_MC */
#if defined(CONFIG_SCHED_MC) && defined(CONFIG_SCHED_SMT)
static int
@@ -6720,7 +7030,7 @@ static void init_numa_sched_groups_power(struct sched_group *group_head)
sg = sg->next;
} while (sg != group_head);
}
-#endif
+#endif /* CONFIG_NUMA */
#ifdef CONFIG_NUMA
/* Free memory allocated for various sched_group structures */
@@ -6735,7 +7045,7 @@ static void free_sched_groups(const cpumask_t *cpu_map, cpumask_t *nodemask)
if (!sched_group_nodes)
continue;
- for (i = 0; i < MAX_NUMNODES; i++) {
+ for (i = 0; i < nr_node_ids; i++) {
struct sched_group *oldsg, *sg = sched_group_nodes[i];
*nodemask = node_to_cpumask(i);
@@ -6757,11 +7067,11 @@ next_sg:
sched_group_nodes_bycpu[cpu] = NULL;
}
}
-#else
+#else /* !CONFIG_NUMA */
static void free_sched_groups(const cpumask_t *cpu_map, cpumask_t *nodemask)
{
}
-#endif
+#endif /* CONFIG_NUMA */
/*
* Initialize sched groups cpu_power.
@@ -6928,7 +7238,7 @@ static int __build_sched_domains(const cpumask_t *cpu_map,
/*
* Allocate the per-node list of sched groups
*/
- sched_group_nodes = kcalloc(MAX_NUMNODES, sizeof(struct sched_group *),
+ sched_group_nodes = kcalloc(nr_node_ids, sizeof(struct sched_group *),
GFP_KERNEL);
if (!sched_group_nodes) {
printk(KERN_WARNING "Can not alloc sched group node list\n");
@@ -7067,7 +7377,7 @@ static int __build_sched_domains(const cpumask_t *cpu_map,
#endif
/* Set up physical groups */
- for (i = 0; i < MAX_NUMNODES; i++) {
+ for (i = 0; i < nr_node_ids; i++) {
SCHED_CPUMASK_VAR(nodemask, allmasks);
SCHED_CPUMASK_VAR(send_covered, allmasks);
@@ -7091,7 +7401,7 @@ static int __build_sched_domains(const cpumask_t *cpu_map,
send_covered, tmpmask);
}
- for (i = 0; i < MAX_NUMNODES; i++) {
+ for (i = 0; i < nr_node_ids; i++) {
/* Set up node groups */
struct sched_group *sg, *prev;
SCHED_CPUMASK_VAR(nodemask, allmasks);
@@ -7130,9 +7440,9 @@ static int __build_sched_domains(const cpumask_t *cpu_map,
cpus_or(*covered, *covered, *nodemask);
prev = sg;
- for (j = 0; j < MAX_NUMNODES; j++) {
+ for (j = 0; j < nr_node_ids; j++) {
SCHED_CPUMASK_VAR(notcovered, allmasks);
- int n = (i + j) % MAX_NUMNODES;
+ int n = (i + j) % nr_node_ids;
node_to_cpumask_ptr(pnodemask, n);
cpus_complement(*notcovered, *covered);
@@ -7185,7 +7495,7 @@ static int __build_sched_domains(const cpumask_t *cpu_map,
}
#ifdef CONFIG_NUMA
- for (i = 0; i < MAX_NUMNODES; i++)
+ for (i = 0; i < nr_node_ids; i++)
init_numa_sched_groups_power(sched_group_nodes[i]);
if (sd_allnodes) {
@@ -7470,7 +7780,7 @@ int sched_create_sysfs_power_savings_entries(struct sysdev_class *cls)
#endif
return err;
}
-#endif
+#endif /* CONFIG_SCHED_MC || CONFIG_SCHED_SMT */
/*
* Force a reinitialization of the sched domains hierarchy. The domains
@@ -7481,21 +7791,28 @@ int sched_create_sysfs_power_savings_entries(struct sysdev_class *cls)
static int update_sched_domains(struct notifier_block *nfb,
unsigned long action, void *hcpu)
{
+ int cpu = (int)(long)hcpu;
+
switch (action) {
- case CPU_UP_PREPARE:
- case CPU_UP_PREPARE_FROZEN:
case CPU_DOWN_PREPARE:
case CPU_DOWN_PREPARE_FROZEN:
+ disable_runtime(cpu_rq(cpu));
+ /* fall-through */
+ case CPU_UP_PREPARE:
+ case CPU_UP_PREPARE_FROZEN:
detach_destroy_domains(&cpu_online_map);
free_sched_domains();
return NOTIFY_OK;
- case CPU_UP_CANCELED:
- case CPU_UP_CANCELED_FROZEN:
+
case CPU_DOWN_FAILED:
case CPU_DOWN_FAILED_FROZEN:
case CPU_ONLINE:
case CPU_ONLINE_FROZEN:
+ enable_runtime(cpu_rq(cpu));
+ /* fall-through */
+ case CPU_UP_CANCELED:
+ case CPU_UP_CANCELED_FROZEN:
case CPU_DEAD:
case CPU_DEAD_FROZEN:
/*
@@ -7695,8 +8012,8 @@ void __init sched_init(void)
root_task_group.cfs_rq = (struct cfs_rq **)ptr;
ptr += nr_cpu_ids * sizeof(void **);
-#endif
-#endif
+#endif /* CONFIG_USER_SCHED */
+#endif /* CONFIG_FAIR_GROUP_SCHED */
#ifdef CONFIG_RT_GROUP_SCHED
init_task_group.rt_se = (struct sched_rt_entity **)ptr;
ptr += nr_cpu_ids * sizeof(void **);
@@ -7710,8 +8027,8 @@ void __init sched_init(void)
root_task_group.rt_rq = (struct rt_rq **)ptr;
ptr += nr_cpu_ids * sizeof(void **);
-#endif
-#endif
+#endif /* CONFIG_USER_SCHED */
+#endif /* CONFIG_RT_GROUP_SCHED */
}
#ifdef CONFIG_SMP
@@ -7727,8 +8044,8 @@ void __init sched_init(void)
#ifdef CONFIG_USER_SCHED
init_rt_bandwidth(&root_task_group.rt_bandwidth,
global_rt_period(), RUNTIME_INF);
-#endif
-#endif
+#endif /* CONFIG_USER_SCHED */
+#endif /* CONFIG_RT_GROUP_SCHED */
#ifdef CONFIG_GROUP_SCHED
list_add(&init_task_group.list, &task_groups);
@@ -7738,8 +8055,8 @@ void __init sched_init(void)
INIT_LIST_HEAD(&root_task_group.children);
init_task_group.parent = &root_task_group;
list_add(&init_task_group.siblings, &root_task_group.children);
-#endif
-#endif
+#endif /* CONFIG_USER_SCHED */
+#endif /* CONFIG_GROUP_SCHED */
for_each_possible_cpu(i) {
struct rq *rq;
@@ -7819,6 +8136,7 @@ void __init sched_init(void)
rq->next_balance = jiffies;
rq->push_cpu = 0;
rq->cpu = i;
+ rq->online = 0;
rq->migration_thread = NULL;
INIT_LIST_HEAD(&rq->migration_queue);
rq_attach_root(rq, &def_root_domain);
@@ -7834,7 +8152,7 @@ void __init sched_init(void)
#endif
#ifdef CONFIG_SMP
- open_softirq(SCHED_SOFTIRQ, run_rebalance_domains, NULL);
+ open_softirq(SCHED_SOFTIRQ, run_rebalance_domains);
#endif
#ifdef CONFIG_RT_MUTEXES
@@ -8058,7 +8376,7 @@ static inline void unregister_fair_sched_group(struct task_group *tg, int cpu)
{
list_del_rcu(&tg->cfs_rq[cpu]->leaf_cfs_rq_list);
}
-#else
+#else /* !CONFG_FAIR_GROUP_SCHED */
static inline void free_fair_sched_group(struct task_group *tg)
{
}
@@ -8076,7 +8394,7 @@ static inline void register_fair_sched_group(struct task_group *tg, int cpu)
static inline void unregister_fair_sched_group(struct task_group *tg, int cpu)
{
}
-#endif
+#endif /* CONFIG_FAIR_GROUP_SCHED */
#ifdef CONFIG_RT_GROUP_SCHED
static void free_rt_sched_group(struct task_group *tg)
@@ -8147,7 +8465,7 @@ static inline void unregister_rt_sched_group(struct task_group *tg, int cpu)
{
list_del_rcu(&tg->rt_rq[cpu]->leaf_rt_rq_list);
}
-#else
+#else /* !CONFIG_RT_GROUP_SCHED */
static inline void free_rt_sched_group(struct task_group *tg)
{
}
@@ -8165,7 +8483,7 @@ static inline void register_rt_sched_group(struct task_group *tg, int cpu)
static inline void unregister_rt_sched_group(struct task_group *tg, int cpu)
{
}
-#endif
+#endif /* CONFIG_RT_GROUP_SCHED */
#ifdef CONFIG_GROUP_SCHED
static void free_sched_group(struct task_group *tg)
@@ -8276,17 +8594,14 @@ void sched_move_task(struct task_struct *tsk)
task_rq_unlock(rq, &flags);
}
-#endif
+#endif /* CONFIG_GROUP_SCHED */
#ifdef CONFIG_FAIR_GROUP_SCHED
-static void set_se_shares(struct sched_entity *se, unsigned long shares)
+static void __set_se_shares(struct sched_entity *se, unsigned long shares)
{
struct cfs_rq *cfs_rq = se->cfs_rq;
- struct rq *rq = cfs_rq->rq;
int on_rq;
- spin_lock_irq(&rq->lock);
-
on_rq = se->on_rq;
if (on_rq)
dequeue_entity(cfs_rq, se, 0);
@@ -8296,8 +8611,17 @@ static void set_se_shares(struct sched_entity *se, unsigned long shares)
if (on_rq)
enqueue_entity(cfs_rq, se, 0);
+}
- spin_unlock_irq(&rq->lock);
+static void set_se_shares(struct sched_entity *se, unsigned long shares)
+{
+ struct cfs_rq *cfs_rq = se->cfs_rq;
+ struct rq *rq = cfs_rq->rq;
+ unsigned long flags;
+
+ spin_lock_irqsave(&rq->lock, flags);
+ __set_se_shares(se, shares);
+ spin_unlock_irqrestore(&rq->lock, flags);
}
static DEFINE_MUTEX(shares_mutex);
@@ -8336,8 +8660,13 @@ int sched_group_set_shares(struct task_group *tg, unsigned long shares)
* w/o tripping rebalance_share or load_balance_fair.
*/
tg->shares = shares;
- for_each_possible_cpu(i)
+ for_each_possible_cpu(i) {
+ /*
+ * force a rebalance
+ */
+ cfs_rq_set_shares(tg->cfs_rq[i], 0);
set_se_shares(tg->se[i], shares);
+ }
/*
* Enable load balance activity on this group, by inserting it back on
@@ -8376,7 +8705,7 @@ static unsigned long to_ratio(u64 period, u64 runtime)
#ifdef CONFIG_CGROUP_SCHED
static int __rt_schedulable(struct task_group *tg, u64 period, u64 runtime)
{
- struct task_group *tgi, *parent = tg ? tg->parent : NULL;
+ struct task_group *tgi, *parent = tg->parent;
unsigned long total = 0;
if (!parent) {
@@ -8400,7 +8729,7 @@ static int __rt_schedulable(struct task_group *tg, u64 period, u64 runtime)
}
rcu_read_unlock();
- return total + to_ratio(period, runtime) <
+ return total + to_ratio(period, runtime) <=
to_ratio(ktime_to_ns(parent->rt_bandwidth.rt_period),
parent->rt_bandwidth.rt_runtime);
}
@@ -8520,16 +8849,21 @@ long sched_group_rt_period(struct task_group *tg)
static int sched_rt_global_constraints(void)
{
+ struct task_group *tg = &root_task_group;
+ u64 rt_runtime, rt_period;
int ret = 0;
+ rt_period = ktime_to_ns(tg->rt_bandwidth.rt_period);
+ rt_runtime = tg->rt_bandwidth.rt_runtime;
+
mutex_lock(&rt_constraints_mutex);
- if (!__rt_schedulable(NULL, 1, 0))
+ if (!__rt_schedulable(tg, rt_period, rt_runtime))
ret = -EINVAL;
mutex_unlock(&rt_constraints_mutex);
return ret;
}
-#else
+#else /* !CONFIG_RT_GROUP_SCHED */
static int sched_rt_global_constraints(void)
{
unsigned long flags;
@@ -8547,7 +8881,7 @@ static int sched_rt_global_constraints(void)
return 0;
}
-#endif
+#endif /* CONFIG_RT_GROUP_SCHED */
int sched_rt_handler(struct ctl_table *table, int write,
struct file *filp, void __user *buffer, size_t *lenp,
@@ -8655,7 +8989,7 @@ static u64 cpu_shares_read_u64(struct cgroup *cgrp, struct cftype *cft)
return (u64) tg->shares;
}
-#endif
+#endif /* CONFIG_FAIR_GROUP_SCHED */
#ifdef CONFIG_RT_GROUP_SCHED
static int cpu_rt_runtime_write(struct cgroup *cgrp, struct cftype *cft,
@@ -8679,7 +9013,7 @@ static u64 cpu_rt_period_read_uint(struct cgroup *cgrp, struct cftype *cft)
{
return sched_group_rt_period(cgroup_tg(cgrp));
}
-#endif
+#endif /* CONFIG_RT_GROUP_SCHED */
static struct cftype cpu_files[] = {
#ifdef CONFIG_FAIR_GROUP_SCHED