lkml.org 
[lkml]   [2000]   [Feb]   [25]   [last100]   RSS Feed
Views: [wrap][no wrap]   [headers]  [forward] 
 
Messages in this thread
Patch in this message
/
Subject[PATCH] proposed scheduler enhancements and fixes
From
Date
The patch below makes the following scheduler changes (random order):

* adds a separate run queue for RT processes. This allows RT processes to be
scheduled without looking at other processes and makes goodness()
SCHED_OTHER-specific, i.e., faster;

* adds a new scheduling policy SCHED_IDLE. Processes of this type run on CPUs
that would otherwise be idle. Useful for apps like SETI@Home, code crackers,
etc. Implementation of this feature is extremely lightweight.
Among the scheduling functions only schedule() is SCHED_IDLE-aware
and the overhead for non-idle CPUs is at most 1 instruction per schedule()
invocation;

* implements process pinning, which allows a process to be pinned to a subset
of the CPUs. Again, implementation is very lightweight (would have been free
if only gcc had good register allocation). Some details about the
implementation. First, we hijack has_cpu and turn it into a bit vector
cpu_sched_vec. When this is ~0 the process may be scheduled on any CPU.
Other non-zero values pin the process to the CPUs that correspond to the 1s
in the value. When a process is executing we set cpu_sched_vec = 0 to make
the process unselectable by other CPUs (the equivalent of has_cpu = 1). The
other change we need is replacing

#define can_schedule(p) (!(p)->has_cpu)

with

#define can_schedule(p, cpumask) ((p)->cpu_sched_vec & (cpumask))

Each of these translate into one instruction, hence the low cost;

* implements lazy counter recalculation by recalculating only runnable
processes;

* fixes race that could trigger recalculation on multiple CPUs;

* fixes recalculation bug that was messing up SCHED_RR scheduling quanta;

* eliminates most of the situations that would permit high priority processes
to languish on the run queue while CPUs remained idle or continued running
low priority stuff. No more idle CPUs on MP systems while there are runnable
processes even if the processes reschedule at high frequency. The only
situation left where we cannot guarantee this behavior is due to the race
between __schedule_tail and wake_up_process;

* __schedule_tail and wake_up_process no longer reschedule the same process
twice;

* sched_yield() works;

* so does sched_rr_get_interval();

* the reschedule logic in setscheduler() was broken for MP --- fixed;

* various other fixes and cleanups that I don't remember right now.

All tests I've tried (lat_ctx, dbench, parallel make, etc.) show improvement
with this patch. However, I noticed that context switch times have gone down
the toilet in recent kernel versions. On my machine lat_ctx -s 0 2 used to be
a bit over 2. With 2.3.47 it has sky rocketed to 4.25 and with the patch it
reduces to 3.95. All I know at this point is that it happened sometime after
2.3.40.

Anyway, patch against 2.3.47 follows. Comments and suggestions welcome.

---
Dimitris Michailidis dimitris@engr.sgi.com

diff -ru linux-2.3.47.orig/arch/i386/kernel/process.c linux-2.3.47/arch/i386/kernel/process.c
--- linux-2.3.47.orig/arch/i386/kernel/process.c Tue Feb 22 16:26:42 2000
+++ linux-2.3.47/arch/i386/kernel/process.c Thu Feb 24 17:19:00 2000
@@ -87,6 +87,7 @@
void cpu_idle(void)
{
/* endless idle loop with no priority at all */
+ current->rt_priority = -2; /* must be before init_idle() */
init_idle();
current->priority = 0;
current->counter = -100;
diff -ru linux-2.3.47.orig/arch/i386/kernel/smpboot.c linux-2.3.47/arch/i386/kernel/smpboot.c
--- linux-2.3.47.orig/arch/i386/kernel/smpboot.c Tue Feb 1 15:03:14 2000
+++ linux-2.3.47/arch/i386/kernel/smpboot.c Thu Feb 24 16:45:44 2000
@@ -516,12 +516,11 @@
idle->processor = cpu;
x86_cpu_to_apicid[cpu] = apicid;
x86_apicid_to_cpu[apicid] = cpu;
- idle->has_cpu = 1; /* we schedule the first task manually */
+ idle->cpu_sched_vec = 0; /* we schedule the first task manually */
idle->thread.eip = (unsigned long) start_secondary;

del_from_runqueue(idle);
unhash_process(idle);
- init_tasks[cpu] = idle;

/* start_eip had better be page-aligned! */
start_eip = setup_trampoline();
diff -ru linux-2.3.47.orig/fs/proc/proc_misc.c linux-2.3.47/fs/proc/proc_misc.c
--- linux-2.3.47.orig/fs/proc/proc_misc.c Thu Feb 10 17:15:11 2000
+++ linux-2.3.47/fs/proc/proc_misc.c Thu Feb 24 16:44:04 2000
@@ -97,7 +97,7 @@
int len;

uptime = jiffies;
- idle = init_tasks[0]->times.tms_utime + init_tasks[0]->times.tms_stime;
+ idle = init_task.times.tms_utime + init_task.times.tms_stime;

/* The formula for the fraction parts really is ((t * 100) / HZ) % 100, but
that would overflow about every five days at HZ == 100.
diff -ru linux-2.3.47.orig/include/linux/prctl.h linux-2.3.47/include/linux/prctl.h
--- linux-2.3.47.orig/include/linux/prctl.h Tue Oct 12 12:12:58 1999
+++ linux-2.3.47/include/linux/prctl.h Wed Feb 23 20:29:31 2000
@@ -10,4 +10,8 @@
#define PR_GET_DUMPABLE 3
#define PR_SET_DUMPABLE 4

+/* Get/set current->cpu_run_on_vec */
+#define PR_SET_RUN_ON 5
+#define PR_GET_RUN_ON 6
+
#endif /* _LINUX_PRCTL_H */
diff -ru linux-2.3.47.orig/include/linux/sched.h linux-2.3.47/include/linux/sched.h
--- linux-2.3.47.orig/include/linux/sched.h Thu Feb 24 18:04:32 2000
+++ linux-2.3.47/include/linux/sched.h Thu Feb 24 16:57:58 2000
@@ -108,6 +108,7 @@
#define SCHED_OTHER 0
#define SCHED_FIFO 1
#define SCHED_RR 2
+#define SCHED_IDLE 4

/*
* This is an additional bit set when we want to
@@ -270,18 +271,21 @@
volatile long need_resched;

cycles_t avg_slice;
- int lock_depth; /* Lock depth. We can context switch in and out of holding a syscall kernel lock... */
+ unsigned long policy;
/* begin intel cache line */
long counter;
long priority;
- unsigned long policy;
+ int rt_priority;
/* memory management info */
struct mm_struct *mm, *active_mm;
- int has_cpu;
+ unsigned long cpu_sched_vec;
int processor;
struct list_head run_list;
+ struct list_head *runq_head;
+ int counter_recalc;
+ unsigned long cpu_run_on_vec;
+ int lock_depth; /* Lock depth. We can context switch in and out of holding a syscall kernel lock... */
struct task_struct *next_task, *prev_task;
- int last_processor;

/* task state */
struct linux_binfmt *binfmt;
@@ -291,6 +295,7 @@
unsigned long personality;
int dumpable:1;
int did_exec:1;
+ int swappable:1;
pid_t pid;
pid_t pgrp;
pid_t tty_old_pgrp;
@@ -310,7 +315,6 @@

wait_queue_head_t wait_chldexit; /* for wait4() */
struct semaphore *vfork_sem; /* for vfork() */
- unsigned long rt_priority;
unsigned long it_real_value, it_prof_value, it_virt_value;
unsigned long it_real_incr, it_prof_incr, it_virt_incr;
struct timer_list real_timer;
@@ -319,7 +323,6 @@
long per_cpu_utime[NR_CPUS], per_cpu_stime[NR_CPUS];
/* mm fault and swap info: this can arguably be seen as either mm-specific or thread-specific */
unsigned long min_flt, maj_flt, nswap, cmin_flt, cmaj_flt, cnswap;
- int swappable:1;
/* process credentials */
uid_t uid,euid,suid,fsuid;
gid_t gid,egid,sgid,fsgid;
@@ -403,6 +406,7 @@
mm: NULL, \
active_mm: &init_mm, \
run_list: LIST_HEAD_INIT(tsk.run_list), \
+ cpu_run_on_vec: ~0, \
next_task: &tsk, \
prev_task: &tsk, \
p_opptr: &tsk, \
@@ -441,7 +445,6 @@
extern union task_union init_task_union;

extern struct mm_struct init_mm;
-extern struct task_struct *init_tasks[NR_CPUS];

/* PID hashing. (shouldnt this be dynamic?) */
#define PIDHASH_SZ (4096 >> 2)
diff -ru linux-2.3.47.orig/kernel/exit.c linux-2.3.47/kernel/exit.c
--- linux-2.3.47.orig/kernel/exit.c Thu Feb 10 17:15:15 2000
+++ linux-2.3.47/kernel/exit.c Wed Feb 23 19:42:46 2000
@@ -26,17 +26,16 @@
{
if (p != current) {
#ifdef __SMP__
- int has_cpu;
-
/*
* Wait to make sure the process isn't on the
* runqueue (active on some other CPU still)
+ *
+ * The scheduler will notify us when it's done with this task
+ * by setting cpu_run_on_vec to 0;
*/
do {
- spin_lock_irq(&runqueue_lock);
- has_cpu = p->has_cpu;
- spin_unlock_irq(&runqueue_lock);
- } while (has_cpu);
+ rmb();
+ } while (p->cpu_run_on_vec);
#endif
free_uid(p);
unhash_process(p);
diff -ru linux-2.3.47.orig/kernel/fork.c linux-2.3.47/kernel/fork.c
--- linux-2.3.47.orig/kernel/fork.c Mon Feb 14 12:34:08 2000
+++ linux-2.3.47/kernel/fork.c Thu Feb 24 17:19:15 2000
@@ -671,7 +671,7 @@
#ifdef __SMP__
{
int i;
- p->has_cpu = 0;
+ p->cpu_sched_vec = p->cpu_run_on_vec;
p->processor = current->processor;
/* ?? should we just memset this ?? */
for(i = 0; i < smp_num_cpus; i++)
@@ -728,7 +728,6 @@
hash_pid(p);
nr_threads++;
write_unlock_irq(&tasklist_lock);
-
wake_up_process(p); /* do this last */
++total_forks;

diff -ru linux-2.3.47.orig/kernel/sched.c linux-2.3.47/kernel/sched.c
--- linux-2.3.47.orig/kernel/sched.c Tue Feb 22 16:26:44 2000
+++ linux-2.3.47/kernel/sched.c Fri Feb 25 15:36:10 2000
@@ -1,7 +1,7 @@
/*
* linux/kernel/sched.c
*
- * Kernel scheduler and related syscalls
+ * Kernel scheduler, scheduling primitives and related syscalls
*
* Copyright (C) 1991, 1992 Linus Torvalds
*
@@ -12,13 +12,6 @@
* 1998-12-28 Implemented better SMP scheduling by Ingo Molnar
*/

-/*
- * 'sched.c' is the main kernel file. It contains scheduling primitives
- * (sleep_on, wakeup, schedule etc) as well as a number of simple system
- * call functions (type getpid()), which just extract a field from
- * current-task
- */
-
#include <linux/mm.h>
#include <linux/init.h>
#include <linux/smp_lock.h>
@@ -28,109 +21,99 @@
#include <asm/uaccess.h>
#include <asm/mmu_context.h>

-
extern void timer_bh(void);
extern void tqueue_bh(void);
extern void immediate_bh(void);

-/*
- * scheduler variables
- */
-
unsigned securebits = SECUREBITS_DEFAULT; /* systemwide security settings */

-extern void mem_use(void);
-
/*
* Init task must be ok at boot for the ix86 as we will check its signals
* via the SMP irq return path.
*/

-struct task_struct * init_tasks[NR_CPUS] = {&init_task, };
+/* tasklist_lock protects the linked list of processes. */
+rwlock_t tasklist_lock = RW_LOCK_UNLOCKED;

/*
- * The tasklist_lock protects the linked list of processes.
- *
- * The scheduler lock is protecting against multiple entry
- * into the scheduling code, and doesn't need to worry
- * about interrupts (because interrupts cannot call the
- * scheduler).
- *
- * The run-queue lock locks the parts that actually access
- * and change the run-queues, and have to be interrupt-safe.
+ * The run-queue lock protects the run-queues.
+ * Interrupts should be disabled while holding this lock.
*/
-spinlock_t runqueue_lock = SPIN_LOCK_UNLOCKED; /* second */
-rwlock_t tasklist_lock = RW_LOCK_UNLOCKED; /* third */
+spinlock_t runqueue_lock = SPIN_LOCK_UNLOCKED;
+
+struct run_queue {
+ struct list_head rt_q; /* holds RT processes */
+ struct list_head so_q; /* holds SCHED_OTHER processes */
+ struct list_head si_q; /* holds SCHED_IDLE processes */
+ int recalc; /* # of counter recalculations on so_q */
+} ____cacheline_aligned;

-static LIST_HEAD(runqueue_head);
+static struct run_queue runq __cacheline_aligned;

/*
* We align per-CPU scheduling data on cacheline boundaries,
* to prevent cacheline ping-pong.
*/
-static union {
- struct schedule_data {
- struct task_struct * curr;
- cycles_t last_schedule;
- } schedule_data;
- char __pad [SMP_CACHE_BYTES];
-} aligned_data [NR_CPUS] __cacheline_aligned = { {{&init_task,0}}};
-
-#define cpu_curr(cpu) aligned_data[(cpu)].schedule_data.curr
+struct schedule_data {
+ struct task_struct *curr; /* (selected to become) current task */
+ struct task_struct *idler; /* idle task for this CPU */
+ struct task_struct *held; /* task we'll switch to when curr sleeps */
+ cycles_t last_schedule;
+ int prio; /* curr->rt_priority */
+ unsigned long mask; /* 1 << CPU# */
+} ____cacheline_aligned;
+
+static struct schedule_data schedule_data[NR_CPUS] __cacheline_aligned = {
+ {&init_task, &init_task, NULL, 0, 0, -2}
+};
+
+#define cpu_curr(cpu) (schedule_data[(cpu)].curr)
+#define cpu_prio(cpu) (schedule_data[(cpu)].prio)
+#define cpu_mask(cpu) (schedule_data[(cpu)].mask)

struct kernel_stat kstat = { 0 };

#ifdef __SMP__

-#define idle_task(cpu) (init_tasks[cpu_number_map(cpu)])
-#define can_schedule(p) (!(p)->has_cpu)
+#define idle_task(cpu) (schedule_data[(cpu)].idler)
+#define can_schedule(p, cpumask) ((p)->cpu_sched_vec & (cpumask))
+#define cpu_can_schedule(p, cpu) (can_schedule(p, cpu_mask(cpu)))

#else

#define idle_task(cpu) (&init_task)
-#define can_schedule(p) (1)
+#define can_schedule(p, cpu_mask) (1)

#endif

void scheduling_functions_start_here(void) { }

/*
- * This is the function that decides how desirable a process is..
+ * This is the function that decides how desirable a SCHED_OTHER process is.
* You can weigh different processes against each other depending
* on what CPU they've run on lately etc to try to handle cache
* and TLB miss penalties.
*
* Return values:
- * -1000: never select this
+ * -100: p is an idle task
* 0: out of time, recalculate counters (but it might still be
* selected)
* +ve: "goodness" value (the larger, the better)
- * +1000: realtime process, select this.
*/
-
-static inline int goodness(struct task_struct * p, int this_cpu, struct mm_struct *this_mm)
+static inline int goodness(struct task_struct *p, int this_cpu,
+ struct mm_struct *this_mm)
{
int weight;

/*
- * Realtime process, select the first one on the
- * runqueue (taking priorities within processes
- * into account).
- */
- if (p->policy != SCHED_OTHER) {
- weight = 1000 + p->rt_priority;
- goto out;
- }
-
- /*
* Give the process a first-approximation goodness value
* according to the number of clock-ticks it has left.
*
* Don't do any other calculations if the time slice is
- * over..
+ * over or if this is the idle task.
*/
weight = p->counter;
- if (!weight)
+ if (weight <= 0)
goto out;

#ifdef __SMP__
@@ -149,150 +132,207 @@
return weight;
}

-/*
- * subtle. We want to discard a yielded process only if it's being
- * considered for a reschedule. Wakeup-time 'queries' of the scheduling
- * state do not count. Another optimization we do: sched_yield()-ed
- * processes are runnable (and thus will be considered for scheduling)
- * right when they are calling schedule(). So the only place we need
- * to care about SCHED_YIELD is when we calculate the previous process'
- * goodness ...
- */
-static inline int prev_goodness(struct task_struct * p, int this_cpu, struct mm_struct *this_mm)
+/* Special case of goodness when p is known to be some CPU's current process */
+static inline int curr_goodness(struct task_struct *p)
{
- if (p->policy & SCHED_YIELD) {
- p->policy &= ~SCHED_YIELD;
- return 0;
+ int weight = p->counter;
+
+ if (weight > 0) {
+#ifdef __SMP__
+ weight += p->priority + (PROC_CHANGE_PENALTY + 1);
+#else
+ weight += p->priority + 1;
+#endif
}
- return goodness(p, this_cpu, this_mm);
+ return weight;
}

-/*
- * the 'goodness value' of replacing a process on a given CPU.
- * positive value means 'replace', zero or negative means 'dont'.
+/* An even more special version of the above when we know that p's scheduling
+ * quantum has not expired. This version also ignores active_mm affinity.
*/
-static inline int preemption_goodness(struct task_struct * prev, struct task_struct * p, int cpu)
+static inline int goodness_no_mm(struct task_struct *p)
{
- return goodness(p, cpu, prev->active_mm) - goodness(prev, cpu, prev->active_mm);
+#ifdef __SMP__
+ return p->counter + p->priority + PROC_CHANGE_PENALTY;
+#else
+ return p->counter + p->priority;
+#endif
}

+/* tune these */
+#define PREEMPTION_THRESHOLD 0
+#define INTERACTIVE_THRESHOLD cacheflush_time
+
+/* heuristic for determining interactive processes */
+#define is_interactive(p) ((p)->avg_slice < INTERACTIVE_THRESHOLD)
+
/*
* This is ugly, but reschedule_idle() is very timing-critical.
* We enter with the runqueue spinlock held, but we might end
* up unlocking it early, so the caller must not unlock the
* runqueue, it's always done by reschedule_idle().
+ *
+ * The MP version of this function is too large so we don't inline
+ * it, otherwise we get a ridiculous number of copies.
*/
-static inline void reschedule_idle(struct task_struct * p, unsigned long flags)
-{
#ifdef __SMP__
- int this_cpu = smp_processor_id(), target_cpu;
+static void reschedule_idle(struct task_struct *p, unsigned long flags)
+{
struct task_struct *tsk;
- int cpu, best_cpu, i;
+ int best_cpu, min_prio, this_prio;

- /*
- * shortcut if the woken up task's last CPU is
- * idle now.
- */
best_cpu = p->processor;
- tsk = idle_task(best_cpu);
- if (cpu_curr(best_cpu) == tsk)
- goto send_now;
-
- /*
- * We know that the preferred CPU has a cache-affine current
- * process, lets try to find a new idle CPU for the woken-up
- * process:
- */
- for (i = smp_num_cpus - 1; i >= 0; i--) {
- cpu = cpu_logical_map(i);
- if (cpu == best_cpu)
- continue;
- tsk = cpu_curr(cpu);
- /*
- * We use the last available idle CPU. This creates
- * a priority list between idle CPUs, but this is not
- * a problem.
- */
- if (tsk == idle_task(cpu))
- goto send_now;
- }
+ min_prio = cpu_can_schedule(p, best_cpu) ? cpu_prio(best_cpu) : 100;
+ this_prio = p->rt_priority;

- /*
- * No CPU is idle, but maybe this process has enough priority
- * to preempt it's preferred CPU.
+ /* Rule 1:
+ * A process does not lose its preferred CPU to lower priority tasks.
*/
- tsk = cpu_curr(best_cpu);
- if (preemption_goodness(tsk, p, best_cpu) > 0)
- goto send_now;
+ if (min_prio >= this_prio) {
+ /* Find a CPU with a lowest rt_priority task. */

- /*
- * We will get here often - or in the high CPU contention
- * case. No CPU is idle and this process is either lowprio or
- * the preferred CPU is highprio. Try to preemt some other CPU
- * only if it's RT or if it's iteractive and the preferred
- * cpu won't reschedule shortly.
+ int i = smp_num_cpus - 1;
+ do {
+ int cpu = cpu_logical_map(i);
+ if (cpu_can_schedule(p, cpu) && cpu_prio(cpu) < min_prio) {
+ min_prio = cpu_prio(cpu);
+ best_cpu = cpu;
+ }
+ } while (--i >= 0);
+ }
+
+ /* Rule 2:
+ * Preempt a CPU currently running a lower rt_priority task.
*/
- if ((p->avg_slice < cacheflush_time && cpu_curr(best_cpu)->avg_slice > cacheflush_time) ||
- p->policy != SCHED_OTHER)
- {
- for (i = smp_num_cpus - 1; i >= 0; i--) {
- cpu = cpu_logical_map(i);
- if (cpu == best_cpu)
- continue;
- tsk = cpu_curr(cpu);
- if (preemption_goodness(tsk, p, cpu) > 0)
+ if (min_prio < this_prio)
+ goto send_now_raise_prio;
+
+ /* The above two rules handle RT and SCHED_IDLE tasks completely.
+ * The only case left to consider is an interactive SCHED_OTHER task
+ * which may be able to preempt some other SCHED_OTHER task.
+ *
+ * For the calculations below we ignore active_mm affinity to make
+ * goodness independent of current tasks. The active_mm affinity
+ * bonus is small and if p needs it to win then the resulting
+ * preemption is not justified. If this bonus ever becomes more
+ * substantial we may need to revisit this approximation.
+ */
+ if ((this_prio | min_prio) == 0 && is_interactive(p) && p->counter) {
+ int this_goodness = goodness_no_mm(p);
+
+ /* First, check if p can preempt its preferred CPU. */
+ best_cpu = p->processor;
+ tsk = cpu_curr(best_cpu);
+ if (cpu_prio(best_cpu) == 0 && cpu_can_schedule(p, best_cpu) &&
+ this_goodness > curr_goodness(tsk) + PREEMPTION_THRESHOLD)
goto send_now;
+
+ /* We check other CPUs only if our preferred CPU is
+ * not expected to reschedule soon.
+ */
+ if (tsk->avg_slice > cacheflush_time) {
+ int i = smp_num_cpus - 1;
+
+ this_goodness -= PROC_CHANGE_PENALTY; /* drop CPU affinity */
+ do {
+ int cpu = cpu_logical_map(i);
+ if (cpu == best_cpu || cpu_prio(cpu) ||
+ !cpu_can_schedule(p, cpu))
+ continue;
+ tsk = cpu_curr(cpu);
+ if (this_goodness > curr_goodness(tsk) + PREEMPTION_THRESHOLD) {
+ best_cpu = cpu;
+ goto send_now;
+ }
+ } while (--i >= 0);
}
}

spin_unlock_irqrestore(&runqueue_lock, flags);
return;
-
+
+send_now_raise_prio:
+ tsk = cpu_curr(best_cpu);
+ cpu_prio(best_cpu) = this_prio;
send_now:
- target_cpu = tsk->processor;
tsk->need_resched = 1;
spin_unlock_irqrestore(&runqueue_lock, flags);
/*
* the APIC stuff can go outside of the lock because
* it uses no task information, only CPU#.
*/
- if (target_cpu != this_cpu)
- smp_send_reschedule(target_cpu);
- return;
+ if (best_cpu != smp_processor_id())
+ smp_send_reschedule(best_cpu);
+}
#else /* UP */
+static inline void reschedule_idle(struct task_struct *p, unsigned long flags)
+{
int this_cpu = smp_processor_id();
struct task_struct *tsk;

tsk = cpu_curr(this_cpu);
- if (preemption_goodness(tsk, p, this_cpu) > 0)
+ if (p->rt_priority > cpu_prio(this_cpu) ||
+ ((p->rt_priority | cpu_prio(this_cpu)) == 0 && p->counter &&
+ goodness_no_mm(p) > curr_goodness(tsk) + PREEMPTION_THRESHOLD))
tsk->need_resched = 1;
spin_unlock_irqrestore(&runqueue_lock, flags);
-#endif
}
+#endif

-/*
- * Careful!
- *
- * This has to add the process to the _beginning_ of the
- * run-queue, not the end. See the comment about "This is
- * subtle" in the scheduler proper..
- */
static inline void add_to_runqueue(struct task_struct * p)
{
- list_add(&p->run_list, &runqueue_head);
+ if (p->rt_priority != 0) /* != SCHED_OTHER */
+ list_add_tail(&p->run_list, p->runq_head);
+ else {
+ int diff;
+
+ list_add(&p->run_list, p->runq_head);
+
+ /* Apply any counter recalculations that occured while we were
+ * not on the run queue. If too many such recalculations have
+ * taken place we fast path the calculation by setting
+ * p->counter to its max for this process. Here, 'too many'
+ * is 6, the base 2 log of the max priority.
+ */
+ if ((diff = runq.recalc - p->counter_recalc) != 0) {
+ p->counter_recalc = runq.recalc;
+
+ if (diff > 5)
+ p->counter = 2 * p->priority - 1;
+ else
+ do {
+ p->counter = (p->counter >> 1) + p->priority;
+ } while (--diff);
+ }
+ }
nr_running++;
}

static inline void move_last_runqueue(struct task_struct * p)
{
list_del(&p->run_list);
- list_add_tail(&p->run_list, &runqueue_head);
+ list_add_tail(&p->run_list, p->runq_head);
}

static inline void move_first_runqueue(struct task_struct * p)
{
list_del(&p->run_list);
- list_add(&p->run_list, &runqueue_head);
+ list_add(&p->run_list, p->runq_head);
+}
+
+/*
+ * Move a process between run queues. If the process is runnable it will be
+ * moved to the end of its new queue. Called with the runqueue_lock held.
+ */
+static inline void migrate_to_runqueue(struct task_struct *p,
+ struct list_head *new_q)
+{
+ if (task_on_runqueue(p)) {
+ list_del(&p->run_list);
+ list_add_tail(&p->run_list, new_q);
+ }
+ p->runq_head = new_q;
+ p->counter_recalc = runq.recalc;
}

/*
@@ -408,15 +448,21 @@
static inline void __schedule_tail(struct task_struct *prev)
{
#ifdef __SMP__
- if ((prev->state == TASK_RUNNING) &&
- (prev != idle_task(smp_processor_id()))) {
- unsigned long flags;
-
- spin_lock_irqsave(&runqueue_lock, flags);
- reschedule_idle(prev, flags); // spin_unlocks runqueue
+ if (prev->rt_priority != -2) { /* nothing to do for the idle task */
+ if (prev->state == TASK_RUNNING) {
+ unsigned long flags;
+
+ spin_lock_irqsave(&runqueue_lock, flags);
+ prev->cpu_sched_vec = prev->cpu_run_on_vec;
+ reschedule_idle(prev, flags); // spin_unlocks runqueue
+ } else {
+ wmb();
+ if (prev->state == TASK_ZOMBIE) /* signal release() */
+ prev->cpu_run_on_vec = 0;
+ else
+ prev->cpu_sched_vec = prev->cpu_run_on_vec;
+ }
}
- wmb();
- prev->has_cpu = 0;
#endif /* __SMP__ */
}

@@ -425,22 +471,23 @@
__schedule_tail(prev);
}

+#define for_each_task_on_runq(p, queue) \
+ for ((p) = (queue).next; (p) != &(queue); (p) = (p)->next)
+
/*
* 'schedule()' is the scheduler function. It's a very simple and nice
* scheduler: it's not perfect, but certainly works for most things.
*
* The goto is "interesting".
- *
- * NOTE!! Task 0 is the 'idle' task, which gets called when no other
- * tasks can run. It can not be killed, and it cannot sleep. The 'state'
- * information in task[0] is never used.
*/
asmlinkage void schedule(void)
{
- struct schedule_data * sched_data;
+ struct schedule_data *sched_data;
struct task_struct *prev, *next, *p;
+ struct mm_struct *this_mm;
struct list_head *tmp;
int this_cpu, c;
+ unsigned long mask;

if (!current->active_mm) BUG();
if (tq_scheduler)
@@ -464,14 +511,20 @@
* 'sched_data' is protected by the fact that we can run
* only one process per CPU.
*/
- sched_data = & aligned_data[this_cpu].schedule_data;
+ sched_data = &schedule_data[this_cpu];
+ next = idle_task(this_cpu);
+ mask = cpu_mask(this_cpu);
+ c = -100;

+ this_mm = prev->active_mm;
spin_lock_irq(&runqueue_lock);

- /* move an exhausted RR process to be last.. */
- if (prev->policy == SCHED_RR)
- goto move_rr_last;
-move_rr_back:
+ /* Move a process that has voluntarily sched_yield()ed to the end of
+ * its run queue. Exhausted SCHED_RR processes are treated similarly.
+ */
+ if (prev->policy & (SCHED_YIELD | SCHED_RR))
+ goto move_last;
+move_back:

switch (prev->state & ~TASK_EXCLUSIVE) {
case TASK_INTERRUPTIBLE:
@@ -483,52 +536,75 @@
del_from_runqueue(prev);
case TASK_RUNNING:
}
- prev->need_resched = 0;

+#ifdef __SMP__
/*
- * this is the scheduler proper:
+ * Temporarily set cpu_sched_vec = cpu_run_on_vec so can_schedule() may
+ * return true for the current process in case it is still runnable.
+ * The runqueue lock protects us from other CPUs. Just remember to
+ * clear cpu_sched_vec before dropping the lock.
*/
+ prev->cpu_sched_vec = prev->cpu_run_on_vec;
+#endif
+
+ prev->need_resched = 0;
+
+ /* First look for a RT task */
+ for_each_task_on_runq(tmp, runq.rt_q) {
+ p = list_entry(tmp, struct task_struct, run_list);
+ if (can_schedule(p, mask) && p->rt_priority > next->rt_priority)
+ next = p;
+ }
+ if (next->rt_priority > 0)
+ goto selected_task;

-repeat_schedule:
+#if 0
/*
- * Default process to select..
- */
- next = idle_task(this_cpu);
- c = -1000;
- if (prev->state == TASK_RUNNING)
- goto still_running;
-still_running_back:
+ * Next look for a SCHED_OTHER task. The default process to select is
+ * the current process if it is still runnable, it is SCHED_OTHER and
+ * it didn't sched_yield(), otherwise the idle task.
+ */
+ if (prev->state == TASK_RUNNING && prev->policy == SCHED_OTHER &&
+ can_schedule(prev, mask))
+ next = prev, c = curr_goodness(prev);
+#endif

- tmp = runqueue_head.next;
- while (tmp != &runqueue_head) {
+ for_each_task_on_runq(tmp, runq.so_q) {
p = list_entry(tmp, struct task_struct, run_list);
- if (can_schedule(p)) {
- int weight = goodness(p, this_cpu, prev->active_mm);
+ if (can_schedule(p, mask)) {
+ int weight = goodness(p, this_cpu, this_mm);
if (weight > c)
c = weight, next = p;
}
- tmp = tmp->next;
}

- /* Do we need to re-calculate counters? */
- if (!c)
+ if (c == 0) /* we need to recalculate counters */
goto recalculate;
+ else if (c < 0)
+ goto pick_sched_idle;
+
+selected_task:
/*
* from this point on nothing can prevent us from
* switching to the next task, save this fact in
* sched_data.
*/
sched_data->curr = next;
+ sched_data->prio = next->rt_priority;
#ifdef __SMP__
- next->has_cpu = 1;
- next->processor = this_cpu;
+ next->cpu_sched_vec = prev->cpu_sched_vec = 0;
#endif
+ prev->policy &= ~SCHED_YIELD;
+ if (prev->policy & (SCHED_FIFO | SCHED_IDLE))
+ prev->counter = MAX_SCHEDULE_TIMEOUT; /* "infinite" quantum */
spin_unlock_irq(&runqueue_lock);

if (prev == next)
goto same_process;

#ifdef __SMP__
+ next->processor = this_cpu;
+
/*
* maintain the per-process 'average timeslice' value.
* (this has to be recalculated even if we reschedule to
@@ -545,18 +621,11 @@

/*
* Exponentially fading average calculation, with
- * some weight so it doesnt get fooled easily by
+ * some weight so it doesn't get fooled easily by
* smaller irregularities.
*/
prev->avg_slice = (this_slice*1 + prev->avg_slice*1)/2;
}
-
- /*
- * We drop the scheduler lock early (it's a global spinlock),
- * thus we have to lock the previous process from getting
- * rescheduled during switch_to().
- */
-
#endif /* __SMP__ */

kstat.context_swtch++;
@@ -572,20 +641,19 @@
prepare_to_switch();
{
struct mm_struct *mm = next->mm;
- struct mm_struct *oldmm = prev->active_mm;
if (!mm) {
if (next->active_mm) BUG();
- next->active_mm = oldmm;
- atomic_inc(&oldmm->mm_count);
- enter_lazy_tlb(oldmm, next, this_cpu);
+ next->active_mm = this_mm;
+ atomic_inc(&this_mm->mm_count);
+ enter_lazy_tlb(this_mm, next, this_cpu);
} else {
if (next->active_mm != mm) BUG();
- switch_mm(oldmm, mm, next, this_cpu);
+ switch_mm(this_mm, mm, next, this_cpu);
}

if (!prev->mm) {
prev->active_mm = NULL;
- mmdrop(oldmm);
+ mmdrop(this_mm);
}
}

@@ -600,22 +668,41 @@
reacquire_kernel_lock(current);
return;

+pick_sched_idle: /* look for a SCHED_IDLE task */
+ for_each_task_on_runq(tmp, runq.si_q) {
+ p = list_entry(tmp, struct task_struct, run_list);
+ if (can_schedule(p, mask)) {
+ next = p;
+ break;
+ }
+ }
+ goto selected_task;
+
recalculate:
- {
- struct task_struct *p;
- spin_unlock_irq(&runqueue_lock);
- read_lock(&tasklist_lock);
- for_each_task(p)
- p->counter = (p->counter >> 1) + p->priority;
- read_unlock(&tasklist_lock);
- spin_lock_irq(&runqueue_lock);
- }
- goto repeat_schedule;
-
-still_running:
- c = prev_goodness(prev, this_cpu, prev->active_mm);
- next = prev;
- goto still_running_back;
+ /* Recalculate counters and choose a process to schedule in one pass.
+ * At this point we are certain that we will find some process to
+ * schedule. Counter recalculation applies only to SCHED_OTHER tasks.
+ */
+ runq.recalc++;
+#if 0
+ /* Getting this right whether or not prev sched_yield()ed is tricky */
+ if (next == prev) {
+ next->counter = next->priority;
+ c = curr_goodness(next);
+ next->counter = 0;
+ }
+#endif
+ for_each_task_on_runq(tmp, runq.so_q) {
+ p = list_entry(tmp, struct task_struct, run_list);
+ p->counter = (p->counter >> 1) + p->priority;
+ p->counter_recalc++;
+ if (can_schedule(p, mask)) {
+ int weight = goodness(p, this_cpu, this_mm);
+ if (weight > c)
+ c = weight, next = p;
+ }
+ }
+ goto selected_task;

handle_softirq:
do_softirq();
@@ -625,12 +712,16 @@
run_task_queue(&tq_scheduler);
goto tq_scheduler_back;

-move_rr_last:
- if (!prev->counter) {
- prev->counter = prev->priority;
+move_last:
+ /* SCHED_YIELD processes are always moved to the end of the run queue.
+ * SCHED_RR processes are moved only if they've exhausted their slice.
+ */
+ if ((prev->policy & SCHED_YIELD) || prev->counter == 0) {
move_last_runqueue(prev);
+ if (prev->policy & SCHED_RR)
+ prev->counter = prev->priority; /* renew quantum */
}
- goto move_rr_back;
+ goto move_back;

scheduling_in_interrupt:
printk("Scheduling in interrupt\n");
@@ -830,12 +921,13 @@
return tsk;
}

-static int setscheduler(pid_t pid, int policy,
- struct sched_param *param)
+static int setscheduler(pid_t pid, int policy, struct sched_param *param)
{
struct sched_param lp;
struct task_struct *p;
- int retval;
+ struct list_head *rq;
+ int retval, sendIPI = 0;
+ unsigned long flags;

retval = -EINVAL;
if (!param || pid < 0)
@@ -845,59 +937,79 @@
if (copy_from_user(&lp, param, sizeof(struct sched_param)))
goto out_nounlock;

+ retval = -ESRCH;
/*
* We play safe to avoid deadlocks.
*/
+ __save_flags(flags);
spin_lock_irq(&runqueue_lock);
read_lock(&tasklist_lock);

p = find_process_by_pid(pid);
-
- retval = -ESRCH;
if (!p)
goto out_unlock;

+ retval = -EINVAL;
if (policy < 0)
- policy = p->policy;
- else {
- retval = -EINVAL;
- if (policy != SCHED_FIFO && policy != SCHED_RR &&
- policy != SCHED_OTHER)
- goto out_unlock;
- }
+ policy = p->policy & ~SCHED_YIELD;
+ else if (policy != SCHED_FIFO && policy != SCHED_RR &&
+ policy != SCHED_OTHER && policy != SCHED_IDLE)
+ goto out_unlock;

/*
* Valid priorities for SCHED_FIFO and SCHED_RR are 1..99, valid
- * priority for SCHED_OTHER is 0.
+ * priority for SCHED_OTHER and SCHED_IDLE is 0.
*/
- retval = -EINVAL;
if (lp.sched_priority < 0 || lp.sched_priority > 99)
goto out_unlock;
- if ((policy == SCHED_OTHER) != (lp.sched_priority == 0))
+ if ((policy == SCHED_OTHER || policy == SCHED_IDLE) != (lp.sched_priority == 0))
goto out_unlock;

retval = -EPERM;
- if ((policy == SCHED_FIFO || policy == SCHED_RR) &&
- !capable(CAP_SYS_NICE))
+ if (policy != SCHED_OTHER && !capable(CAP_SYS_NICE))
goto out_unlock;
if ((current->euid != p->euid) && (current->euid != p->uid) &&
!capable(CAP_SYS_NICE))
goto out_unlock;

retval = 0;
+ p->rt_priority = policy == SCHED_IDLE ? -1 : lp.sched_priority;
p->policy = policy;
- p->rt_priority = lp.sched_priority;
- if (task_on_runqueue(p))
- move_first_runqueue(p);

- current->need_resched = 1;
+ if (policy == SCHED_OTHER)
+ rq = &runq.so_q;
+ else if (policy == SCHED_IDLE)
+ rq = &runq.si_q;
+ else
+ rq = &runq.rt_q;
+ migrate_to_runqueue(p, rq);

-out_unlock:
read_unlock(&tasklist_lock);
+
+ /* Check if we need to reschedule the process */
+ if (p->state == TASK_RUNNING) {
+ if (p == cpu_curr(p->processor)) {
+ p->need_resched = 1;
+ sendIPI = p->processor != smp_processor_id();
+ } else {
+ reschedule_idle(p, flags); /* unlocks runqueue_lock */
+ goto out_nounlock;
+ }
+ }
spin_unlock_irq(&runqueue_lock);
+
+#ifdef __SMP__
+ if (sendIPI)
+ smp_send_reschedule(p->processor);
+#endif

out_nounlock:
return retval;
+
+out_unlock:
+ read_unlock(&tasklist_lock);
+ spin_unlock_irq(&runqueue_lock);
+ goto out_nounlock;
}

asmlinkage long sys_sched_setscheduler(pid_t pid, int policy,
@@ -920,16 +1032,11 @@
if (pid < 0)
goto out_nounlock;

- read_lock(&tasklist_lock);
-
retval = -ESRCH;
+ read_lock(&tasklist_lock);
p = find_process_by_pid(pid);
- if (!p)
- goto out_unlock;
-
- retval = p->policy;
-
-out_unlock:
+ if (p)
+ retval = p->policy & ~SCHED_YIELD;
read_unlock(&tasklist_lock);

out_nounlock:
@@ -946,35 +1053,31 @@
if (!param || pid < 0)
goto out_nounlock;

+ retval = -ESRCH;
read_lock(&tasklist_lock);
p = find_process_by_pid(pid);
- retval = -ESRCH;
- if (!p)
- goto out_unlock;
- lp.sched_priority = p->rt_priority;
+ if (p)
+ lp.sched_priority = p->rt_priority >= 0 ? p->rt_priority : 0;
read_unlock(&tasklist_lock);

/*
* This one might sleep, we cannot do it with a spinlock held ...
*/
- retval = copy_to_user(param, &lp, sizeof(*param)) ? -EFAULT : 0;
+ if (p)
+ retval = copy_to_user(param, &lp, sizeof(lp)) ? -EFAULT : 0;

out_nounlock:
return retval;
-
-out_unlock:
- read_unlock(&tasklist_lock);
- return retval;
}

asmlinkage long sys_sched_yield(void)
{
- spin_lock_irq(&runqueue_lock);
- if (current->policy == SCHED_OTHER)
- current->policy |= SCHED_YIELD;
+ /* Don't mess with the run queue here, let the scheduler do it.
+ * It already has to deal with exhausted SCHED_RR processes and it
+ * holds the runqueue_lock anyway.
+ */
+ current->policy |= SCHED_YIELD;
current->need_resched = 1;
- move_last_runqueue(current);
- spin_unlock_irq(&runqueue_lock);
return 0;
}

@@ -988,6 +1091,7 @@
ret = 99;
break;
case SCHED_OTHER:
+ case SCHED_IDLE:
ret = 0;
break;
}
@@ -1004,6 +1108,7 @@
ret = 1;
break;
case SCHED_OTHER:
+ case SCHED_IDLE:
ret = 0;
}
return ret;
@@ -1012,12 +1117,23 @@
asmlinkage long sys_sched_rr_get_interval(pid_t pid, struct timespec *interval)
{
struct timespec t;
+ struct task_struct *p;
+ int retval = -EINVAL;

- t.tv_sec = 0;
- t.tv_nsec = 150000;
- if (copy_to_user(interval, &t, sizeof(struct timespec)))
- return -EFAULT;
- return 0;
+ if (pid < 0)
+ goto out_nounlock;
+
+ retval = -ESRCH;
+ read_lock(&tasklist_lock);
+ p = find_process_by_pid(pid);
+ if (p)
+ jiffies_to_timespec(p->policy & SCHED_FIFO ? 0 : p->priority,
+ &t);
+ read_unlock(&tasklist_lock);
+ if (p)
+ retval = copy_to_user(interval, &t, sizeof(t)) ? -EFAULT : 0;
+out_nounlock:
+ return retval;
}

static void show_task(struct task_struct * p)
@@ -1145,8 +1261,7 @@

void __init init_idle(void)
{
- struct schedule_data * sched_data;
- sched_data = &aligned_data[smp_processor_id()].schedule_data;
+ struct schedule_data *sched_data = &schedule_data[smp_processor_id()];

if (current != &init_task && task_on_runqueue(current)) {
printk("UGH! (%d:%d) was on the runqueue, removing.\n",
@@ -1154,19 +1269,33 @@
del_from_runqueue(current);
}
sched_data->curr = current;
+ sched_data->idler = current;
+ sched_data->held = NULL;
sched_data->last_schedule = get_cycles();
+ sched_data->prio = current->rt_priority;
+ sched_data->mask = 1 << smp_processor_id();
}

void __init sched_init(void)
{
+ int cpu = smp_processor_id();
+ int nr;
+
/*
* We have to do a little magic to get the first
* process right in SMP mode.
*/
- int cpu = smp_processor_id();
- int nr;
-
init_task.processor = cpu;
+
+ /*
+ * init_task is never on a run queue, but its children inherit this.
+ */
+ init_task.runq_head = &runq.so_q;
+
+ INIT_LIST_HEAD(&runq.rt_q);
+ INIT_LIST_HEAD(&runq.so_q);
+ INIT_LIST_HEAD(&runq.si_q);
+ runq.recalc = 0;

for(nr = 0; nr < PIDHASH_SZ; nr++)
pidhash[nr] = NULL;
Only in linux-2.3.47/kernel: sched.s.1
diff -ru linux-2.3.47.orig/kernel/signal.c linux-2.3.47/kernel/signal.c
--- linux-2.3.47.orig/kernel/signal.c Sat Jan 29 15:16:12 2000
+++ linux-2.3.47/kernel/signal.c Wed Feb 23 20:02:26 2000
@@ -358,7 +358,7 @@
* since signal event passing goes through ->blocked.
*/
spin_lock(&runqueue_lock);
- if (t->has_cpu && t->processor != smp_processor_id())
+ if (!t->cpu_sched_vec && t->processor != smp_processor_id())
smp_send_reschedule(t->processor);
spin_unlock(&runqueue_lock);
#endif /* __SMP__ */
diff -ru linux-2.3.47.orig/kernel/sys.c linux-2.3.47/kernel/sys.c
--- linux-2.3.47.orig/kernel/sys.c Thu Feb 10 17:15:15 2000
+++ linux-2.3.47/kernel/sys.c Wed Feb 23 20:56:40 2000
@@ -1018,6 +1018,7 @@
{
int error = 0;
int sig;
+ unsigned long run_on_vec;

switch (option) {
case PR_SET_PDEATHSIG:
@@ -1041,6 +1042,19 @@
break;
}
current->dumpable = arg2;
+ break;
+ case PR_SET_RUN_ON:
+ run_on_vec = arg2;
+
+ /* A mask of 0 requests pinning to the current CPU */
+ if (run_on_vec == 0)
+ run_on_vec = 1 << smp_processor_id();
+ current->cpu_run_on_vec = run_on_vec;
+ if ((run_on_vec & (1 << smp_processor_id())) == 0)
+ current->need_resched = 1; /* we must move */
+ break;
+ case PR_GET_RUN_ON:
+ error = put_user(current->cpu_run_on_vec, (long *)arg2);
break;
default:
error = -EINVAL;

-
To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
the body of a message to majordomo@vger.rutgers.edu
Please read the FAQ at http://www.tux.org/lkml/

\
 
 \ /
  Last update: 2005-03-22 13:56    [W:0.092 / U:0.356 seconds]
©2003-2020 Jasper Spaans|hosted at Digital Ocean and TransIP|Read the blog|Advertise on this site