Name: Split Scheduler Structure into Per-CPU and Per-Runqueue Parts Author: Rusty Russell Status: Experimental D: The scheduler code talks alot about "balancing CPU load". In fact, D: it wants to balance runqueue load: the difference is important if D: two CPUs share a single runqueue. D: D: The first step, then, is to split the "struct runqueue" into D: CPU-specific parts (eg. current task) and runqueue-specific parts. D: Since this change effects the "knobs" of the scheduling algorithm D: in strange ways, I chose to have only one CPU rebalance each D: runqueue (unless idle). diff -urpN --exclude TAGS -X /home/rusty/devel/kernel/kernel-patches/current-dontdiff --minimal .12361-linux-2.6.0-test11-bk8/kernel/sched.c .12361-linux-2.6.0-test11-bk8.updated/kernel/sched.c --- .12361-linux-2.6.0-test11-bk8/kernel/sched.c 2003-12-12 13:37:44.000000000 +1100 +++ .12361-linux-2.6.0-test11-bk8.updated/kernel/sched.c 2003-12-15 12:59:57.000000000 +1100 @@ -153,8 +153,8 @@ #define LOW_CREDIT(p) \ ((p)->interactive_credit < -CREDIT_LIMIT) -#define TASK_PREEMPTS_CURR(p, rq) \ - ((p)->prio < (rq)->curr->prio) +#define TASK_PREEMPTS_CURR(p) \ + ((p)->prio < cpu_curr(task_cpu(p))->prio) /* * BASE_TIMESLICE scales user-nice values [ -20 ... 19 ] @@ -190,7 +190,7 @@ struct prio_array { }; /* - * This is the main, per-CPU runqueue data structure. + * This is the main, (usually) per-CPU runqueue data structure. * * Locking rule: those places that want to lock multiple runqueues * (such as the load balancing or the thread migration code), lock @@ -198,36 +198,48 @@ struct prio_array { */ struct runqueue { spinlock_t lock; - unsigned long nr_running, nr_switches, expired_timestamp, + unsigned long nr_running, expired_timestamp, nr_uninterruptible; - task_t *curr, *idle; - struct mm_struct *prev_mm; prio_array_t *active, *expired, arrays[2]; int prev_cpu_load[NR_CPUS]; #ifdef CONFIG_NUMA atomic_t *node_nr_running; int prev_node_load[MAX_NUMNODES]; #endif +}; + +/* Per-cpu structure: by default rq points to same CPU's struct runqueue. */ +struct schedinfo { + /* rq->lock protects this structure, too */ + struct runqueue *rq; + + task_t *curr, *idle; + struct mm_struct *prev_mm; + task_t *migration_thread; struct list_head migration_queue; + unsigned long nr_switches; atomic_t nr_iowait; }; static DEFINE_PER_CPU(struct runqueue, runqueues); +static DEFINE_PER_CPU(struct schedinfo, schedinfo); -#define cpu_rq(cpu) (&per_cpu(runqueues, (cpu))) +#define cpu_schedinfo(cpu) (&per_cpu(schedinfo, (cpu))) +#define cpu_rq(cpu) (cpu_schedinfo(cpu)->rq) #define this_rq() (&__get_cpu_var(runqueues)) -#define task_rq(p) cpu_rq(task_cpu(p)) -#define cpu_curr(cpu) (cpu_rq(cpu)->curr) +#define task_schedinfo(p) cpu_schedinfo(task_cpu(p)) +#define this_schedinfo() (&__get_cpu_var(schedinfo)) +#define cpu_curr(cpu) (cpu_schedinfo(cpu)->curr) /* * Default context-switch locking: */ #ifndef prepare_arch_switch -# define prepare_arch_switch(rq, next) do { } while(0) -# define finish_arch_switch(rq, next) spin_unlock_irq(&(rq)->lock) -# define task_running(rq, p) ((rq)->curr == (p)) +# define prepare_arch_switch(si, next) do { } while(0) +# define finish_arch_switch(si, next) spin_unlock_irq(&(si)->rq->lock) +# define task_running(si, p) ((si)->curr == (p)) #endif #ifdef CONFIG_NUMA @@ -276,28 +288,28 @@ __init void node_nr_running_init(void) #endif /* CONFIG_NUMA */ /* - * task_rq_lock - lock the runqueue a given task resides on and disable - * interrupts. Note the ordering: we can safely lookup the task_rq without + * task_si_lock - lock the runqueue a given task resides on and disable + * interrupts. Note the ordering: we can safely lookup the task_cpu without * explicitly disabling preemption. */ -static inline runqueue_t *task_rq_lock(task_t *p, unsigned long *flags) +static inline struct schedinfo *task_si_lock(task_t *p, unsigned long *flags) { - struct runqueue *rq; + struct schedinfo *si; repeat_lock_task: local_irq_save(*flags); - rq = task_rq(p); - spin_lock(&rq->lock); - if (unlikely(rq != task_rq(p))) { - spin_unlock_irqrestore(&rq->lock, *flags); + si = task_schedinfo(p); + spin_lock(&si->rq->lock); + if (unlikely(si != task_schedinfo(p))) { + spin_unlock_irqrestore(&si->rq->lock, *flags); goto repeat_lock_task; } - return rq; + return si; } -static inline void task_rq_unlock(runqueue_t *rq, unsigned long *flags) +static inline void task_si_unlock(struct schedinfo *si, unsigned long *flags) { - spin_unlock_irqrestore(&rq->lock, *flags); + spin_unlock_irqrestore(&si->rq->lock, *flags); } /* @@ -553,12 +565,12 @@ inline int task_curr(task_t *p) void wait_task_inactive(task_t * p) { unsigned long flags; - runqueue_t *rq; + struct schedinfo *si; repeat: preempt_disable(); - rq = task_rq(p); - if (unlikely(task_running(rq, p))) { + si = task_schedinfo(p); + if (unlikely(task_running(si, p))) { cpu_relax(); /* * enable/disable preemption just to make this @@ -568,13 +580,13 @@ repeat: preempt_enable(); goto repeat; } - rq = task_rq_lock(p, &flags); - if (unlikely(task_running(rq, p))) { - task_rq_unlock(rq, &flags); + si = task_si_lock(p, &flags); + if (unlikely(task_running(si, p))) { + task_si_unlock(si, &flags); preempt_enable(); goto repeat; } - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); preempt_enable(); } @@ -619,10 +631,10 @@ static int try_to_wake_up(task_t * p, un unsigned long flags; int success = 0; long old_state; - runqueue_t *rq; + struct schedinfo *si; repeat_lock_task: - rq = task_rq_lock(p, &flags); + si = task_si_lock(p, &flags); old_state = p->state; if (old_state & state) { if (!p->array) { @@ -630,16 +642,16 @@ repeat_lock_task: * Fast-migrate the task if it's not running or runnable * currently. Do not violate hard affinity. */ - if (unlikely(sync && !task_running(rq, p) && + if (unlikely(sync && !task_running(si, p) && (task_cpu(p) != smp_processor_id()) && cpu_isset(smp_processor_id(), p->cpus_allowed))) { set_task_cpu(p, smp_processor_id()); - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); goto repeat_lock_task; } if (old_state == TASK_UNINTERRUPTIBLE){ - rq->nr_uninterruptible--; + si->rq->nr_uninterruptible--; /* * Tasks on involuntary sleep don't earn * sleep_avg beyond just interactive state. @@ -647,17 +659,17 @@ repeat_lock_task: p->activated = -1; } if (sync && (task_cpu(p) == smp_processor_id())) - __activate_task(p, rq); + __activate_task(p, si->rq); else { - activate_task(p, rq); - if (TASK_PREEMPTS_CURR(p, rq)) - resched_task(rq->curr); + activate_task(p, si->rq); + if (TASK_PREEMPTS_CURR(p)) + resched_task(cpu_curr(task_cpu(p))); } success = 1; } p->state = TASK_RUNNING; } - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); return success; } @@ -682,7 +694,8 @@ int wake_up_state(task_t *p, unsigned in void wake_up_forked_process(task_t * p) { unsigned long flags; - runqueue_t *rq = task_rq_lock(current, &flags); + struct schedinfo *si = task_si_lock(current, &flags); + p->state = TASK_RUNNING; /* @@ -702,15 +715,15 @@ void wake_up_forked_process(task_t * p) set_task_cpu(p, smp_processor_id()); if (unlikely(!current->array)) - __activate_task(p, rq); + __activate_task(p, si->rq); else { p->prio = current->prio; list_add_tail(&p->run_list, ¤t->run_list); p->array = current->array; p->array->nr_active++; - nr_running_inc(rq); + nr_running_inc(si->rq); } - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); } /* @@ -758,11 +771,11 @@ void sched_exit(task_t * p) */ static inline void finish_task_switch(task_t *prev) { - runqueue_t *rq = this_rq(); - struct mm_struct *mm = rq->prev_mm; + struct schedinfo *si = this_schedinfo(); + struct mm_struct *mm = si->prev_mm; unsigned long prev_task_flags; - rq->prev_mm = NULL; + si->prev_mm = NULL; /* * A task struct has one reference for the use as "current". @@ -776,7 +789,7 @@ static inline void finish_task_switch(ta * Manfred Spraul */ prev_task_flags = prev->flags; - finish_arch_switch(rq, prev); + finish_arch_switch(si, prev); if (mm) mmdrop(mm); if (unlikely(prev_task_flags & PF_DEAD)) @@ -799,7 +812,8 @@ asmlinkage void schedule_tail(task_t *pr * context_switch - switch to the new MM and the new * thread's register state. */ -static inline task_t * context_switch(runqueue_t *rq, task_t *prev, task_t *next) +static inline task_t * context_switch(struct schedinfo *si, + task_t *prev, task_t *next) { struct mm_struct *mm = next->mm; struct mm_struct *oldmm = prev->active_mm; @@ -813,8 +827,8 @@ static inline task_t * context_switch(ru if (unlikely(!prev->mm)) { prev->active_mm = NULL; - WARN_ON(rq->prev_mm); - rq->prev_mm = oldmm; + WARN_ON(si->prev_mm); + si->prev_mm = oldmm; } /* Here we just switch the register state and the stack. */ @@ -859,7 +873,7 @@ unsigned long nr_context_switches(void) for (i = 0; i < NR_CPUS; i++) { if (!cpu_online(i)) continue; - sum += cpu_rq(i)->nr_switches; + sum += cpu_schedinfo(i)->nr_switches; } return sum; } @@ -871,7 +885,7 @@ unsigned long nr_iowait(void) for (i = 0; i < NR_CPUS; ++i) { if (!cpu_online(i)) continue; - sum += atomic_read(&cpu_rq(i)->nr_iowait); + sum += atomic_read(&cpu_schedinfo(i)->nr_iowait); } return sum; } @@ -879,7 +893,7 @@ unsigned long nr_iowait(void) /* * double_rq_lock - safely lock two runqueues * - * Note this does not disable interrupts like task_rq_lock, + * Note this does not disable interrupts like task_si_lock, * you need to do so manually before calling. */ static inline void double_rq_lock(runqueue_t *rq1, runqueue_t *rq2) @@ -900,7 +914,7 @@ static inline void double_rq_lock(runque /* * double_rq_unlock - safely unlock two runqueues * - * Note this does not restore interrupts like task_rq_unlock, + * Note this does not restore interrupts like task_si_unlock, * you need to do so manually after calling. */ static inline void double_rq_unlock(runqueue_t *rq1, runqueue_t *rq2) @@ -912,7 +926,7 @@ static inline void double_rq_unlock(runq #ifdef CONFIG_NUMA /* - * If dest_cpu is allowed for this process, migrate the task to it. + * If dest_rq is allowed for this process, migrate the task to it. * This is accomplished by forcing the cpu_allowed mask to only * allow dest_cpu, which will force the cpu onto dest_cpu. Then * the cpu_allowed mask is restored. @@ -932,17 +946,19 @@ static void sched_migrate_task(task_t *p } /* - * Find the least loaded CPU. Slightly favor the current CPU by - * setting its runqueue length as the minimum to start. + * Find the least loaded runqueue this task can run on. Slightly + * favor the current runqueue by setting its runqueue length as the + * minimum to start. */ -static int sched_best_cpu(struct task_struct *p) +static struct runqueue *sched_best_runqueue(struct task_struct *p) { - int i, minload, load, best_cpu, node = 0; + int i, minload, load, node = 0; + struct runqueue *best_rq; cpumask_t cpumask; - best_cpu = task_cpu(p); - if (cpu_rq(best_cpu)->nr_running <= 2) - return best_cpu; + best_rq = task_schedinfo(p)->rq; + if (best_rq->nr_running <= 2) + return best_rq; minload = 10000000; for_each_node_with_cpus(i) { @@ -962,24 +978,24 @@ static int sched_best_cpu(struct task_st minload = 10000000; cpumask = node_to_cpumask(node); for (i = 0; i < NR_CPUS; ++i) { - if (!cpu_isset(i, cpumask)) + if (!cpu_isset(i, cpumask) || !cpu_isset(i, p->cpus_allowed)) continue; if (cpu_rq(i)->nr_running < minload) { - best_cpu = i; - minload = cpu_rq(i)->nr_running; + best_rq = cpu_rq(i); + minload = best_rq->nr_running; } } - return best_cpu; + return best_rq; } void sched_balance_exec(void) { - int new_cpu; + struct runqueue *rq; if (numnodes > 1) { - new_cpu = sched_best_cpu(current); - if (new_cpu != smp_processor_id()) - sched_migrate_task(current, new_cpu); + rq = sched_best_runqueue(current); + if (rq != this_rq()) + sched_migrate_task(current, rq); } } @@ -1139,7 +1155,7 @@ static inline void pull_task(runqueue_t * Note that idle threads have a prio of MAX_PRIO, for this test * to be always true for them. */ - if (TASK_PREEMPTS_CURR(p, this_rq)) + if (TASK_PREEMPTS_CURR(p)) set_need_resched(); } @@ -1153,13 +1169,13 @@ static inline void pull_task(runqueue_t */ static inline int -can_migrate_task(task_t *tsk, runqueue_t *rq, int this_cpu, int idle) +can_migrate_task(task_t *tsk, int this_cpu, int idle) { unsigned long delta = sched_clock() - tsk->timestamp; if (!idle && (delta <= JIFFIES_TO_NS(cache_decay_ticks))) return 0; - if (task_running(rq, tsk)) + if (task_running(task_schedinfo(tsk), tsk)) return 0; if (!cpu_isset(this_cpu, tsk->cpus_allowed)) return 0; @@ -1233,7 +1249,7 @@ skip_queue: curr = curr->prev; - if (!can_migrate_task(tmp, busiest, this_cpu, idle)) { + if (!can_migrate_task(tmp, this_cpu, idle)) { if (curr != head) goto skip_queue; idx++; @@ -1358,7 +1374,8 @@ void scheduler_tick(int user_ticks, int { int cpu = smp_processor_id(); struct cpu_usage_stat *cpustat = &kstat_this_cpu.cpustat; - runqueue_t *rq = this_rq(); + struct schedinfo *si = this_schedinfo(); + runqueue_t *rq = si->rq; task_t *p = current; if (rcu_pending(cpu)) @@ -1373,8 +1390,8 @@ void scheduler_tick(int user_ticks, int sys_ticks = 0; } - if (p == rq->idle) { - if (atomic_read(&rq->nr_iowait) > 0) + if (p == si->idle) { + if (atomic_read(&si->nr_iowait) > 0) cpustat->iowait += sys_ticks; else cpustat->idle += sys_ticks; @@ -1471,6 +1488,7 @@ void scheduling_functions_start_here(voi asmlinkage void schedule(void) { task_t *prev, *next; + struct schedinfo *si; runqueue_t *rq; prio_array_t *array; struct list_head *queue; @@ -1493,7 +1511,8 @@ asmlinkage void schedule(void) need_resched: preempt_disable(); prev = current; - rq = this_rq(); + si = this_schedinfo(); + rq = si->rq; release_kernel_lock(prev); now = sched_clock(); @@ -1539,7 +1558,7 @@ pick_next_task: if (rq->nr_running) goto pick_next_task; #endif - next = rq->idle; + next = si->idle; rq->expired_timestamp = 0; goto switch_tasks; } @@ -1586,11 +1605,11 @@ switch_tasks: if (likely(prev != next)) { next->timestamp = now; - rq->nr_switches++; - rq->curr = next; + si->nr_switches++; + si->curr = next; - prepare_arch_switch(rq, next); - prev = context_switch(rq, prev, next); + prepare_arch_switch(si, next); + prev = context_switch(si, prev, next); barrier(); finish_task_switch(prev); @@ -1846,7 +1865,7 @@ void set_user_nice(task_t *p, long nice) { unsigned long flags; prio_array_t *array; - runqueue_t *rq; + struct schedinfo *si; int old_prio, new_prio, delta; if (TASK_NICE(p) == nice || nice < -20 || nice > 19) @@ -1855,7 +1874,7 @@ void set_user_nice(task_t *p, long nice) * We have to be careful, if called from sys_setpriority(), * the task might be in the middle of scheduling on another CPU. */ - rq = task_rq_lock(p, &flags); + si = task_si_lock(p, &flags); /* * The RT priorities are set via setscheduler(), but we still * allow the 'normal' nice value to be set - but as expected @@ -1882,11 +1901,11 @@ void set_user_nice(task_t *p, long nice) * If the task increased its priority or is running and * lowered its priority, then reschedule its CPU: */ - if (delta < 0 || (delta > 0 && task_running(rq, p))) - resched_task(rq->curr); + if (delta < 0 || (delta > 0 && task_running(si, p))) + resched_task(si->curr); } out_unlock: - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); } EXPORT_SYMBOL(set_user_nice); @@ -1965,7 +1984,7 @@ EXPORT_SYMBOL(task_nice); */ int idle_cpu(int cpu) { - return cpu_curr(cpu) == cpu_rq(cpu)->idle; + return cpu_curr(cpu) == cpu_schedinfo(cpu)->idle; } EXPORT_SYMBOL_GPL(idle_cpu); @@ -1989,7 +2008,7 @@ static int setscheduler(pid_t pid, int p int oldprio; prio_array_t *array; unsigned long flags; - runqueue_t *rq; + struct schedinfo *si; task_t *p; if (!param || pid < 0) @@ -2014,7 +2033,7 @@ static int setscheduler(pid_t pid, int p * To be able to change p->policy safely, the apropriate * runqueue lock must be held. */ - rq = task_rq_lock(p, &flags); + si = task_si_lock(p, &flags); if (policy < 0) policy = p->policy; @@ -2049,7 +2068,7 @@ static int setscheduler(pid_t pid, int p array = p->array; if (array) - deactivate_task(p, task_rq(p)); + deactivate_task(p, si->rq); retval = 0; p->policy = policy; p->rt_priority = lp.sched_priority; @@ -2059,21 +2078,21 @@ static int setscheduler(pid_t pid, int p else p->prio = p->static_prio; if (array) { - __activate_task(p, task_rq(p)); + __activate_task(p, si->rq); /* * Reschedule if we are currently running on this runqueue and * our priority decreased, or if we are not currently running on * this runqueue and our priority is higher than the current's */ - if (rq->curr == p) { + if (si->curr == p) { if (p->prio > oldprio) - resched_task(rq->curr); - } else if (p->prio < rq->curr->prio) - resched_task(rq->curr); + resched_task(p); + } else if (p->prio < si->curr->prio) + resched_task(si->curr); } out_unlock: - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); out_unlock_tasklist: read_unlock_irq(&tasklist_lock); @@ -2314,7 +2333,7 @@ void yield(void) EXPORT_SYMBOL(yield); /* - * This task is about to go to sleep on IO. Increment rq->nr_iowait so + * This task is about to go to sleep on IO. Increment nr_iowait so * that process accounting knows that this is a task in IO wait state. * * But don't do that if it is a deliberate, throttling IO wait (this task @@ -2322,23 +2341,23 @@ EXPORT_SYMBOL(yield); */ void io_schedule(void) { - struct runqueue *rq = this_rq(); + struct schedinfo *si = &__get_cpu_var(schedinfo); - atomic_inc(&rq->nr_iowait); + atomic_inc(&si->nr_iowait); schedule(); - atomic_dec(&rq->nr_iowait); + atomic_dec(&si->nr_iowait); } EXPORT_SYMBOL(io_schedule); long io_schedule_timeout(long timeout) { - struct runqueue *rq = this_rq(); + struct schedinfo *si = &__get_cpu_var(schedinfo); long ret; - atomic_inc(&rq->nr_iowait); + atomic_inc(&si->nr_iowait); ret = schedule_timeout(timeout); - atomic_dec(&rq->nr_iowait); + atomic_dec(&si->nr_iowait); return ret; } @@ -2522,19 +2541,20 @@ void show_state(void) void __init init_idle(task_t *idle, int cpu) { - runqueue_t *idle_rq = cpu_rq(cpu), *rq = cpu_rq(task_cpu(idle)); + struct schedinfo *idle_si = cpu_schedinfo(cpu); + runqueue_t *rq = cpu_rq(task_cpu(idle)); unsigned long flags; local_irq_save(flags); - double_rq_lock(idle_rq, rq); + double_rq_lock(idle_si->rq, rq); - idle_rq->curr = idle_rq->idle = idle; + idle_si->curr = idle_si->idle = idle; deactivate_task(idle, rq); idle->array = NULL; idle->prio = MAX_PRIO; idle->state = TASK_RUNNING; set_task_cpu(idle, cpu); - double_rq_unlock(idle_rq, rq); + double_rq_unlock(idle_si->rq, rq); set_tsk_need_resched(idle); local_irq_restore(flags); @@ -2582,36 +2602,36 @@ int set_cpus_allowed(task_t *p, cpumask_ { unsigned long flags; migration_req_t req; - runqueue_t *rq; + struct schedinfo *si; if (any_online_cpu(new_mask) == NR_CPUS) return -EINVAL; - rq = task_rq_lock(p, &flags); + si = task_si_lock(p, &flags); p->cpus_allowed = new_mask; /* * Can the task run on the task's current CPU? If not then * migrate the thread off to a proper CPU. */ if (cpu_isset(task_cpu(p), new_mask)) { - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); return 0; } /* * If the task is not on a runqueue (and not running), then * it is sufficient to simply update the task's cpu field. */ - if (!p->array && !task_running(rq, p)) { + if (!p->array && !task_running(si, p)) { set_task_cpu(p, any_online_cpu(p->cpus_allowed)); - task_rq_unlock(rq, &flags); + task_si_unlock(si, &flags); return 0; } init_completion(&req.done); req.task = p; - list_add(&req.list, &rq->migration_queue); - task_rq_unlock(rq, &flags); + list_add(&req.list, &si->migration_queue); + task_si_unlock(si, &flags); - wake_up_process(rq->migration_thread); + wake_up_process(si->migration_thread); wait_for_completion(&req.done); return 0; @@ -2622,25 +2642,25 @@ EXPORT_SYMBOL_GPL(set_cpus_allowed); /* Move (not current) task off this cpu, onto dest cpu. */ static void move_task_away(struct task_struct *p, int dest_cpu) { - runqueue_t *rq_dest; + struct schedinfo *si_dest; unsigned long flags; - rq_dest = cpu_rq(dest_cpu); + si_dest = cpu_schedinfo(dest_cpu); local_irq_save(flags); - double_rq_lock(this_rq(), rq_dest); + double_rq_lock(this_rq(), si_dest->rq); if (task_cpu(p) != smp_processor_id()) goto out; /* Already moved */ set_task_cpu(p, dest_cpu); if (p->array) { deactivate_task(p, this_rq()); - activate_task(p, rq_dest); - if (p->prio < rq_dest->curr->prio) - resched_task(rq_dest->curr); + activate_task(p, si_dest->rq); + if (p->prio < si_dest->curr->prio) + resched_task(si_dest->curr); } out: - double_rq_unlock(this_rq(), rq_dest); + double_rq_unlock(this_rq(), si_dest->rq); local_irq_restore(flags); } @@ -2661,7 +2681,7 @@ static int migration_thread(void * data) struct sched_param __user param = { .sched_priority = MAX_RT_PRIO-1 }; migration_startup_t *startup = data; int cpu = startup->cpu; - runqueue_t *rq; + struct schedinfo *si; int ret; startup->task = current; @@ -2676,8 +2696,8 @@ static int migration_thread(void * data) ret = setscheduler(0, SCHED_FIFO, ¶m); - rq = this_rq(); - rq->migration_thread = current; + si = &__get_cpu_var(schedinfo); + si->migration_thread = current; for (;;) { struct list_head *head; @@ -2686,17 +2706,17 @@ static int migration_thread(void * data) if (current->flags & PF_FREEZE) refrigerator(PF_IOTHREAD); - spin_lock_irq(&rq->lock); - head = &rq->migration_queue; + spin_lock_irq(&si->rq->lock); + head = &si->migration_queue; current->state = TASK_INTERRUPTIBLE; if (list_empty(head)) { - spin_unlock_irq(&rq->lock); + spin_unlock_irq(&si->rq->lock); schedule(); continue; } req = list_entry(head->next, migration_req_t, list); list_del_init(head->next); - spin_unlock_irq(&rq->lock); + spin_unlock_irq(&si->rq->lock); move_task_away(req->task, any_online_cpu(req->task->cpus_allowed)); @@ -2733,7 +2753,7 @@ static int migration_call(struct notifie wake_up_process(startup.task); - while (!cpu_rq(cpu)->migration_thread) + while (!cpu_schedinfo(cpu)->migration_thread) yield(); break; @@ -2803,22 +2823,26 @@ __init static void init_kstat(void) { void __init sched_init(void) { - runqueue_t *rq; + struct schedinfo *si; int i, j, k; /* Init the kstat counters */ init_kstat(); for (i = 0; i < NR_CPUS; i++) { prio_array_t *array; + runqueue_t *rq; rq = cpu_rq(i); rq->active = rq->arrays; rq->expired = rq->arrays + 1; spin_lock_init(&rq->lock); - INIT_LIST_HEAD(&rq->migration_queue); - atomic_set(&rq->nr_iowait, 0); nr_running_init(rq); + si = cpu_schedinfo(i); + si->rq = rq; + INIT_LIST_HEAD(&si->migration_queue); + atomic_set(&si->nr_iowait, 0); + for (j = 0; j < 2; j++) { array = rq->arrays + j; for (k = 0; k < MAX_PRIO; k++) { @@ -2833,9 +2857,9 @@ void __init sched_init(void) * We have to do a little magic to get the first * thread right in SMP mode. */ - rq = this_rq(); - rq->curr = current; - rq->idle = current; + si = this_schedinfo(); + si->curr = current; + si->idle = current; set_task_cpu(current, smp_processor_id()); wake_up_forked_process(current);