patch-2.2.8 linux/kernel/sched.c

Next file: linux/kernel/signal.c
Previous file: linux/kernel/exit.c
Back to the patch index
Back to the overall index

diff -u --recursive --new-file v2.2.7/linux/kernel/sched.c linux/kernel/sched.c
@@ -96,13 +96,156 @@
  
 struct task_struct * task[NR_TASKS] = {&init_task, };
 
+/*
+ * 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 kernel_stat kstat = { 0 };
 
+#ifdef __SMP__
+
+#define idle_task(cpu) (task[cpu_number_map[(cpu)]])
+#define can_schedule(p)	(!(p)->has_cpu)
+
+#else
+
+#define idle_task(cpu) (&init_task)
+#define can_schedule(p) (1)
+
+#endif
+
 void scheduling_functions_start_here(void) { }
 
+/*
+ * This is the function that decides how desirable a 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
+ *	     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 * prev,
+				 struct task_struct * p, int this_cpu)
+{
+	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..
+	 */
+	weight = p->counter;
+	if (!weight)
+		goto out;
+			
 #ifdef __SMP__
-static void reschedule_idle_slow(struct task_struct * p)
+	/* Give a largish advantage to the same processor...   */
+	/* (this is equivalent to penalizing other processors) */
+	if (p->processor == this_cpu)
+		weight += PROC_CHANGE_PENALTY;
+#endif
+
+	/* .. and a slight advantage to the current MM */
+	if (p->mm == prev->mm)
+		weight += 1;
+	weight += p->priority;
+
+out:
+	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 * prev,
+					struct task_struct * p, int this_cpu)
+{
+	if (p->policy & SCHED_YIELD) {
+		p->policy &= ~SCHED_YIELD;
+		return 0;
+	}
+	return goodness(prev, p, this_cpu);
+}
+
+/*
+ * the 'goodness value' of replacing a process on a given CPU.
+ * positive value means 'replace', zero or negative means 'dont'.
+ */
+static inline int preemption_goodness (struct task_struct * prev,
+				struct task_struct * p, int cpu)
+{
+	return goodness(prev, p, cpu) - goodness(prev, prev, cpu);
+}
+
+/*
+ * If there is a dependency between p1 and p2,
+ * don't be too eager to go into the slow schedule.
+ * In particular, if p1 and p2 both want the kernel
+ * lock, there is no point in trying to make them
+ * extremely parallel..
+ *
+ * (No lock - lock_depth < 0)
+ *
+ * There are two additional metrics here:
+ *
+ * first, a 'cutoff' interval, currently 0-200 usecs on
+ * x86 CPUs, depending on the size of the 'SMP-local cache'.
+ * If the current process has longer average timeslices than
+ * this, then we utilize the idle CPU.
+ *
+ * second, if the wakeup comes from a process context,
+ * then the two processes are 'related'. (they form a
+ * 'gang')
+ *
+ * An idle CPU is almost always a bad thing, thus we skip
+ * the idle-CPU utilization only if both these conditions
+ * are true. (ie. a 'process-gang' rescheduling with rather
+ * high frequency should stay on the same CPU).
+ *
+ * [We can switch to something more finegrained in 2.3.]
+ *
+ * do not 'guess' if the to-be-scheduled task is RT.
+ */
+#define related(p1,p2) (((p1)->lock_depth >= 0) && (p2)->lock_depth >= 0) && \
+	(((p2)->policy == SCHED_OTHER) && ((p1)->avg_slice < cacheflush_time))
+
+static inline void reschedule_idle_slow(struct task_struct * p)
 {
+#ifdef __SMP__
 /*
  * (see reschedule_idle() for an explanation first ...)
  *
@@ -124,60 +267,71 @@
  * 2.3. Also we can try to use the avg_slice value to predict
  * 'likely reschedule' events even on other CPUs.]
  */
-	int best_cpu = p->processor, this_cpu = smp_processor_id();
-	struct task_struct **idle = task, *tsk, *target_tsk;
-	int i = smp_num_cpus;
+	int this_cpu = smp_processor_id(), target_cpu;
+	struct task_struct *tsk, *target_tsk;
+	int cpu, best_cpu, weight, best_weight, i;
+	unsigned long flags;
+
+	best_weight = 0; /* prevents negative weight */
+
+	spin_lock_irqsave(&runqueue_lock, flags);
+
+	/*
+	 * shortcut if the woken up task's last CPU is
+	 * idle now.
+	 */
+	best_cpu = p->processor;
+	target_tsk = idle_task(best_cpu);
+	if (cpu_curr(best_cpu) == target_tsk)
+		goto send_now;
 
 	target_tsk = NULL;
-	do {
-		tsk = *idle;
-		idle++;
-		if (tsk->has_cpu) {
-			if (tsk->processor == this_cpu)
-				continue;
+	for (i = 0; i < smp_num_cpus; i++) {
+		cpu = cpu_logical_map(i);
+		tsk = cpu_curr(cpu);
+		if (related(tsk, p))
+			goto out_no_target;
+		weight = preemption_goodness(tsk, p, cpu);
+		if (weight > best_weight) {
+			best_weight = weight;
 			target_tsk = tsk;
-			if (tsk->processor == best_cpu) {
-				/*
-				 * bingo, we couldnt get a better
-				 * CPU, activate it.
-				 */
-				goto send; /* this one helps GCC ... */
-			}
 		}
-	} while (--i > 0);
+	}
 
 	/*
-	 * found any idle CPU?
+	 * found any suitable CPU?
 	 */
-	if (target_tsk) {
-send:
-		target_tsk->need_resched = 1;
-		smp_send_reschedule(target_tsk->processor);
-		return;
-	}
+	if (!target_tsk)
+		goto out_no_target;
+		
+send_now:
+	target_cpu = target_tsk->processor;
+	target_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;
+out_no_target:
+	spin_unlock_irqrestore(&runqueue_lock, flags);
+	return;
+#else /* UP */
+	int this_cpu = smp_processor_id();
+	struct task_struct *tsk;
+
+	tsk = cpu_curr(this_cpu);
+	if (preemption_goodness(tsk, p, this_cpu) > 0)
+		tsk->need_resched = 1;
+#endif
 }
-#endif /* __SMP__ */
 
-/*
- * If there is a dependency between p1 and p2,
- * don't be too eager to go into the slow schedule.
- * In particular, if p1 and p2 both want the kernel
- * lock, there is no point in trying to make them
- * extremely parallel..
- *
- * (No lock - lock_depth < 0)
- */
-#define related(p1,p2) ((p1)->lock_depth >= 0 && (p2)->lock_depth >= 0)
-
-static inline void reschedule_idle(struct task_struct * p)
+static void reschedule_idle(struct task_struct * p)
 {
-
-	if (p->policy != SCHED_OTHER || p->counter > current->counter + 3) {
-		current->need_resched = 1;
-		return;
-	}
-
 #ifdef __SMP__
+	int cpu = smp_processor_id();
 	/*
 	 * ("wakeup()" should not be called before we've initialized
 	 * SMP completely.
@@ -187,35 +341,20 @@
 	 *
 	 * SMP rescheduling is done in 2 passes:
 	 *  - pass #1: faster: 'quick decisions'
-	 *  - pass #2: slower: 'lets try and find another CPU'
+	 *  - pass #2: slower: 'lets try and find a suitable CPU'
 	 */
 
 	/*
-	 * Pass #1
-	 *
-	 * There are two metrics here:
-	 *
-	 * first, a 'cutoff' interval, currently 0-200 usecs on
-	 * x86 CPUs, depending on the size of the 'SMP-local cache'.
-	 * If the current process has longer average timeslices than
-	 * this, then we utilize the idle CPU.
-	 *
-	 * second, if the wakeup comes from a process context,
-	 * then the two processes are 'related'. (they form a
-	 * 'gang')
-	 *
-	 * An idle CPU is almost always a bad thing, thus we skip
-	 * the idle-CPU utilization only if both these conditions
-	 * are true. (ie. a 'process-gang' rescheduling with rather
-	 * high frequency should stay on the same CPU).
-	 *
-	 * [We can switch to something more finegrained in 2.3.]
+	 * Pass #1. (subtle. We might be in the middle of __switch_to, so
+	 * to preserve scheduling atomicity we have to use cpu_curr)
 	 */
-	if ((current->avg_slice < cacheflush_time) && related(current, p))
+	if ((p->processor == cpu) && related(cpu_curr(cpu), p))
 		return;
-
-	reschedule_idle_slow(p);
 #endif /* __SMP__ */
+	/*
+	 * Pass #2
+	 */
+	reschedule_idle_slow(p);
 }
 
 /*
@@ -291,7 +430,6 @@
  * The run-queue lock locks the parts that actually access
  * and change the run-queues, and have to be interrupt-safe.
  */
-spinlock_t scheduler_lock = SPIN_LOCK_UNLOCKED;	/* should be acquired first */
 spinlock_t runqueue_lock = SPIN_LOCK_UNLOCKED;  /* second */
 rwlock_t tasklist_lock = RW_LOCK_UNLOCKED;	/* third */
 
@@ -307,12 +445,19 @@
 {
 	unsigned long flags;
 
+	/*
+	 * We want the common case fall through straight, thus the goto.
+	 */
 	spin_lock_irqsave(&runqueue_lock, flags);
 	p->state = TASK_RUNNING;
-	if (!p->next_run) {
-		add_to_runqueue(p);
-		reschedule_idle(p);
-	}
+	if (p->next_run)
+		goto out;
+	add_to_runqueue(p);
+	spin_unlock_irqrestore(&runqueue_lock, flags);
+
+	reschedule_idle(p);
+	return;
+out:
 	spin_unlock_irqrestore(&runqueue_lock, flags);
 }
 
@@ -324,63 +469,6 @@
 }
 
 /*
- * This is the function that decides how desirable a 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
- *	     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, struct task_struct * prev, int this_cpu)
-{
-	int policy = p->policy;
-	int weight;
-
-	if (policy & SCHED_YIELD) {
-		p->policy = policy & ~SCHED_YIELD;
-		return 0;
-	}
-
-	/*
-	 * Realtime process, select the first one on the
-	 * runqueue (taking priorities within processes
-	 * into account).
-	 */
-	if (policy != SCHED_OTHER)
-		return 1000 + p->rt_priority;
-
-	/*
-	 * 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..
-	 */
-	weight = p->counter;
-	if (weight) {
-			
-#ifdef __SMP__
-		/* Give a largish advantage to the same processor...   */
-		/* (this is equivalent to penalizing other processors) */
-		if (p->processor == this_cpu)
-			weight += PROC_CHANGE_PENALTY;
-#endif
-
-		/* .. and a slight advantage to the current thread */
-		if (p->mm == prev->mm)
-			weight += 1;
-		weight += p->priority;
-	}
-
-	return weight;
-}
-
-/*
  * Event timer code
  */
 #define TVN_BITS 6
@@ -513,18 +601,6 @@
 	return ret;
 }
 
-#ifdef __SMP__
-
-#define idle_task (task[cpu_number_map[this_cpu]])
-#define can_schedule(p)	(!(p)->has_cpu)
-
-#else
-
-#define idle_task (&init_task)
-#define can_schedule(p) (1)
-
-#endif
-
 signed long schedule_timeout(signed long timeout)
 {
 	struct timer_list timer;
@@ -577,60 +653,24 @@
 }
 
 /*
- * This one aligns per-CPU data on cacheline boundaries.
+ * schedule_tail() is getting called from the fork return path. This
+ * cleans up all remaining scheduler things, without impacting the
+ * common case.
  */
-static union {
-	struct schedule_data {
-		struct task_struct * prev;
-		long prevstate;
-		cycles_t last_schedule;
-	} schedule_data;
-	char __pad [SMP_CACHE_BYTES];
-} aligned_data [NR_CPUS] __cacheline_aligned = { {{&init_task,0}}};
-
-
-static inline void __schedule_tail (void)
+static inline void __schedule_tail (struct task_struct *prev)
 {
 #ifdef __SMP__
-	struct schedule_data * sched_data;
-
-	/*
-	 * We might have switched CPUs:
-	 */
-	sched_data = & aligned_data[smp_processor_id()].schedule_data;
-
-	/*
-	 * Subtle. In the rare event that we got a wakeup to 'prev' just
-	 * during the reschedule (this is possible, the scheduler is pretty
-	 * parallel), we should do another reschedule in the next task's
-	 * context. schedule() will do the right thing next time around.
-	 * this is equivalent to 'delaying' the wakeup until the reschedule
-	 * has finished.
-	 */
-	if (sched_data->prev->state != sched_data->prevstate)
-		current->need_resched = 1;
-
-	/*
-	 * Release the previous process ...
-	 *
-	 * We have dropped all locks, and we must make sure that we
-	 * only mark the previous process as no longer having a CPU
-	 * after all other state has been seen by other CPU's. Thus
-	 * the write memory barrier!
-	 */
+	if ((prev->state == TASK_RUNNING) &&
+			(prev != idle_task(smp_processor_id())))
+		reschedule_idle(prev);
 	wmb();
-	sched_data->prev->has_cpu = 0;
+	prev->has_cpu = 0;
 #endif /* __SMP__ */
 }
 
-/*
- * schedule_tail() is getting called from the fork return path. This
- * cleans up all remaining scheduler things, without impacting the
- * common case.
- */
-void schedule_tail (void)
+void schedule_tail (struct task_struct *prev)
 {
-	__schedule_tail();
+	__schedule_tail(prev);
 }
 
 /*
@@ -646,37 +686,38 @@
 asmlinkage void schedule(void)
 {
 	struct schedule_data * sched_data;
-	struct task_struct * prev, * next;
-	int this_cpu;
+	struct task_struct *prev, *next, *p;
+	int this_cpu, c;
 
-	run_task_queue(&tq_scheduler);
+	if (tq_scheduler)
+		goto handle_tq_scheduler;
+tq_scheduler_back:
 
 	prev = current;
 	this_cpu = prev->processor;
-	/*
-	 * 'sched_data' is protected by the fact that we can run
-	 * only one process per CPU.
-	 */
-	sched_data = & aligned_data[this_cpu].schedule_data;
 
 	if (in_interrupt())
 		goto scheduling_in_interrupt;
+
 	release_kernel_lock(prev, this_cpu);
 
 	/* Do "administrative" work here while we don't hold any locks */
-	if (bh_active & bh_mask)
-		do_bottom_half();
+	if (bh_mask & bh_active)
+		goto handle_bh;
+handle_bh_back:
+
+	/*
+	 * 'sched_data' is protected by the fact that we can run
+	 * only one process per CPU.
+	 */
+	sched_data = & aligned_data[this_cpu].schedule_data;
 
-	spin_lock(&scheduler_lock);
 	spin_lock_irq(&runqueue_lock);
 
 	/* move an exhausted RR process to be last.. */
-	prev->need_resched = 0;
-
-	if (!prev->counter && prev->policy == SCHED_RR) {
-		prev->counter = prev->priority;
-		move_last_runqueue(prev);
-	}
+	if (prev->policy == SCHED_RR)
+		goto move_rr_last;
+move_rr_back:
 
 	switch (prev->state) {
 		case TASK_INTERRUPTIBLE:
@@ -688,62 +729,72 @@
 			del_from_runqueue(prev);
 		case TASK_RUNNING:
 	}
+	prev->need_resched = 0;
 
-	sched_data->prevstate = prev->state;
+repeat_schedule:
 
-/* this is the scheduler proper: */
-	{
-		struct task_struct * p = init_task.next_run;
-		int c = -1000;
+	/*
+	 * this is the scheduler proper:
+	 */
 
-		/* Default process to select.. */
-		next = idle_task;
-		if (prev->state == TASK_RUNNING) {
-			c = goodness(prev, prev, this_cpu);
-			next = prev;
-		}
+	p = init_task.next_run;
+	/* Default process to select.. */
+	next = idle_task(this_cpu);
+	c = -1000;
+	if (prev->state == TASK_RUNNING)
+		goto still_running;
+still_running_back:
 
-		/*
-		 * This is subtle.
-		 * Note how we can enable interrupts here, even
-		 * though interrupts can add processes to the run-
-		 * queue. This is because any new processes will
-		 * be added to the front of the queue, so "p" above
-		 * is a safe starting point.
-		 * run-queue deletion and re-ordering is protected by
-		 * the scheduler lock
-		 */
-		spin_unlock_irq(&runqueue_lock);
+	/*
+	 * This is subtle.
+	 * Note how we can enable interrupts here, even
+	 * though interrupts can add processes to the run-
+	 * queue. This is because any new processes will
+	 * be added to the front of the queue, so "p" above
+	 * is a safe starting point.
+	 * run-queue deletion and re-ordering is protected by
+	 * the scheduler lock
+	 */
 /*
  * Note! there may appear new tasks on the run-queue during this, as
  * interrupts are enabled. However, they will be put on front of the
  * list, so our list starting at "p" is essentially fixed.
  */
-		while (p != &init_task) {
-			if (can_schedule(p)) {
-				int weight = goodness(p, prev, this_cpu);
-				if (weight > c)
-					c = weight, next = p;
-			}
-			p = p->next_run;
-		}
-
-		/* Do we need to re-calculate counters? */
-		if (!c) {
-			struct task_struct *p;
-			read_lock(&tasklist_lock);
-			for_each_task(p)
-				p->counter = (p->counter >> 1) + p->priority;
-			read_unlock(&tasklist_lock);
+	while (p != &init_task) {
+		if (can_schedule(p)) {
+			int weight = goodness(prev, p, this_cpu);
+			if (weight > c)
+				c = weight, next = p;
 		}
+		p = p->next_run;
 	}
 
+	/* Do we need to re-calculate counters? */
+	if (!c)
+		goto recalculate;
+	/*
+	 * from this point on nothing can prevent us from
+	 * switching to the next task, save this fact in
+	 * sched_data.
+	 */
+	sched_data->curr = next;
+#ifdef __SMP__
+ 	next->has_cpu = 1;
+	next->processor = this_cpu;
+#endif
+	spin_unlock_irq(&runqueue_lock);
+
+	if (prev == next)
+		goto same_process;
+
+#ifdef __SMP__
  	/*
  	 * maintain the per-process 'average timeslice' value.
  	 * (this has to be recalculated even if we reschedule to
- 	 * the same process) Currently this is only used on SMP:
+ 	 * the same process) Currently this is only used on SMP,
+	 * and it's approximate, so we do not have to maintain
+	 * it while holding the runqueue spinlock.
  	 */
-#ifdef __SMP__
 	{
 		cycles_t t, this_slice;
 
@@ -752,10 +803,11 @@
 		sched_data->last_schedule = t;
 
 		/*
-		 * Simple, exponentially fading average calculation:
+		 * Exponentially fading average calculation, with
+		 * some weight so it doesnt get fooled easily by
+		 * smaller irregularities.
 		 */
-		prev->avg_slice = this_slice + prev->avg_slice;
-		prev->avg_slice >>= 1;
+		prev->avg_slice = (this_slice*1 + prev->avg_slice*1)/2;
 	}
 
 	/*
@@ -763,27 +815,55 @@
 	 * thus we have to lock the previous process from getting
 	 * rescheduled during switch_to().
 	 */
- 	next->processor = this_cpu;
- 	next->has_cpu = 1;
-	spin_unlock(&scheduler_lock);
+
 #endif /* __SMP__ */
- 	if (prev != next) {
-#ifdef __SMP__
-		sched_data->prev = prev;
-#endif
-	 	kstat.context_swtch++;
-		get_mmu_context(next);
-		switch_to(prev,next);
 
-		__schedule_tail();
-	}
+	kstat.context_swtch++;
+	get_mmu_context(next);
+	switch_to(prev, next, prev);
+	__schedule_tail(prev);
+
+same_process:
   
 	reacquire_kernel_lock(current);
 	return;
 
+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, prev, this_cpu);
+	next = prev;
+	goto still_running_back;
+
+handle_bh:
+	do_bottom_half();
+	goto handle_bh_back;
+
+handle_tq_scheduler:
+	run_task_queue(&tq_scheduler);
+	goto tq_scheduler_back;
+
+move_rr_last:
+	if (!prev->counter) {
+		prev->counter = prev->priority;
+		move_last_runqueue(prev);
+	}
+	goto move_rr_back;
+
 scheduling_in_interrupt:
 	printk("Scheduling in interrupt\n");
 	*(int *)0 = 0;
+	return;
 }
 
 rwlock_t waitqueue_lock = RW_LOCK_UNLOCKED;
@@ -798,21 +878,42 @@
  */
 void __wake_up(struct wait_queue **q, unsigned int mode)
 {
-	struct wait_queue *next;
+	struct task_struct *p;
+	struct wait_queue *head, *next;
+
+        if (!q)
+		goto out;
+	/*
+	 * this is safe to be done before the check because it
+	 * means no deference, just pointer operations.
+	 */
+	head = WAIT_QUEUE_HEAD(q);
 
 	read_lock(&waitqueue_lock);
-	if (q && (next = *q)) {
-		struct wait_queue *head;
+	next = *q;
+	if (!next)
+		goto out_unlock;
 
-		head = WAIT_QUEUE_HEAD(q);
-		while (next != head) {
-			struct task_struct *p = next->task;
-			next = next->next;
-			if (p->state & mode)
+	while (next != head) {
+		p = next->task;
+		next = next->next;
+		if (p->state & mode) {
+			/*
+			 * We can drop the read-lock early if this
+			 * is the only/last process.
+			 */
+			if (next == head) {
+				read_unlock(&waitqueue_lock);
 				wake_up_process(p);
+				goto out;
+			}
+			wake_up_process(p);
 		}
 	}
+out_unlock:
 	read_unlock(&waitqueue_lock);
+out:
+	return;
 }
 
 /*
@@ -942,14 +1043,14 @@
 
 #define	SLEEP_ON_HEAD					\
 	wait.task = current;				\
-	write_lock_irqsave(&waitqueue_lock, flags);	\
+	write_lock_irqsave(&waitqueue_lock,flags);	\
 	__add_wait_queue(p, &wait);			\
 	write_unlock(&waitqueue_lock);
 
 #define	SLEEP_ON_TAIL						\
 	write_lock_irq(&waitqueue_lock);			\
 	__remove_wait_queue(p, &wait);				\
-	write_unlock_irqrestore(&waitqueue_lock, flags);
+	write_unlock_irqrestore(&waitqueue_lock,flags);
 
 void interruptible_sleep_on(struct wait_queue **p)
 {
@@ -1623,7 +1724,6 @@
 	/*
 	 * We play safe to avoid deadlocks.
 	 */
-	spin_lock(&scheduler_lock);
 	spin_lock_irq(&runqueue_lock);
 	read_lock(&tasklist_lock);
 
@@ -1671,7 +1771,6 @@
 out_unlock:
 	read_unlock(&tasklist_lock);
 	spin_unlock_irq(&runqueue_lock);
-	spin_unlock(&scheduler_lock);
 
 out_nounlock:
 	return retval;
@@ -1746,14 +1845,12 @@
 
 asmlinkage int sys_sched_yield(void)
 {
-	spin_lock(&scheduler_lock);
 	spin_lock_irq(&runqueue_lock);
 	if (current->policy == SCHED_OTHER)
 		current->policy |= SCHED_YIELD;
 	current->need_resched = 1;
 	move_last_runqueue(current);
 	spin_unlock_irq(&runqueue_lock);
-	spin_unlock(&scheduler_lock);
 	return 0;
 }
 
@@ -1930,11 +2027,22 @@
 	read_unlock(&tasklist_lock);
 }
 
+void __init init_idle(void)
+{
+	cycles_t t;
+	struct schedule_data * sched_data;
+	sched_data = &aligned_data[smp_processor_id()].schedule_data;
+
+	t = get_cycles();
+	sched_data->curr = current;
+	sched_data->last_schedule = t;
+}
+
 void __init sched_init(void)
 {
 	/*
-	 *	We have to do a little magic to get the first
-	 *	process right in SMP mode.
+	 * We have to do a little magic to get the first
+	 * process right in SMP mode.
 	 */
 	int cpu=hard_smp_processor_id();
 	int nr = NR_TASKS;

FUNET's LINUX-ADM group, linux-adm@nic.funet.fi
TCL-scripts by Sam Shen (who was at: slshen@lbl.gov)