From: Nick Piggin <piggin@cyberone.com.au>


This is Nick's scheduler policy v10 after some things have been broken out
and commented.

This is a rewrite of the _policy_ of Ingo's O(1) scheduler. All the mechanism
is still Ingo's.

The patch removes the interactivity estimator. It introduces a priority
calculator which adapts more quickly to change running patterns. To compensate
these changes, it introduces a form of Linus' backboost.

It completely changes timeslice allocation. Previously a timeslice would be
allocated based solely on a process' nice level - ~200ms for -20, 10ms for 19.
Timeslices are now based only on priority (however nice level directly affects
priority).

You'll have to read task_timeslice to get a proper picture of how it works,
but here is an (inaccurate) examples:

Two high priority processes are running: they'll each get a 25ms timeslice.
Two low priority processes become runnable: they'll each get a 5ms timeslice.
High priority processes sleep: the low prio processes now get 100ms timeslices.




 include/linux/sched.h |    7 
 kernel/sched.c        |  469 ++++++++++++++++++++++++++++----------------------
 2 files changed, 269 insertions(+), 207 deletions(-)

diff -puN include/linux/sched.h~np-sched-04-sched-policy-10b include/linux/sched.h
--- 25/include/linux/sched.h~np-sched-04-sched-policy-10b	2003-09-01 02:06:56.000000000 -0700
+++ 25-akpm/include/linux/sched.h	2003-09-01 02:06:59.000000000 -0700
@@ -342,12 +342,17 @@ struct task_struct {
 	struct list_head run_list;
 	prio_array_t *array;
 
+	/* Scheduler variables follow. kernel/sched.c */
+	unsigned long array_sequence;
+	unsigned long timestamp;
+
+	unsigned long total_time, sleep_time;
 	unsigned long sleep_avg;
-	unsigned long last_run;
 
 	unsigned long policy;
 	cpumask_t cpus_allowed;
 	unsigned int time_slice, first_time_slice;
+	unsigned int used_slice;
 
 	struct list_head tasks;
 	struct list_head ptrace_children;
diff -puN kernel/sched.c~np-sched-04-sched-policy-10b kernel/sched.c
--- 25/kernel/sched.c~np-sched-04-sched-policy-10b	2003-09-01 02:06:56.000000000 -0700
+++ 25-akpm/kernel/sched.c	2003-09-01 02:06:56.000000000 -0700
@@ -60,79 +60,50 @@
 #define MAX_USER_PRIO		(USER_PRIO(MAX_PRIO))
 
 /*
- * These are the 'tuning knobs' of the scheduler:
- *
- * Minimum timeslice is 10 msecs, default timeslice is 100 msecs,
- * maximum timeslice is 200 msecs. Timeslices get refilled after
- * they expire.
- */
-#define MIN_TIMESLICE		( 10 * HZ / 1000)
-#define MAX_TIMESLICE		(200 * HZ / 1000)
-#define CHILD_PENALTY		50
-#define PARENT_PENALTY		100
-#define EXIT_WEIGHT		3
-#define PRIO_BONUS_RATIO	25
-#define INTERACTIVE_DELTA	2
-#define MAX_SLEEP_AVG		(10*HZ)
-#define STARVATION_LIMIT	(10*HZ)
-#define NODE_THRESHOLD		125
-
-/*
- * If a task is 'interactive' then we reinsert it in the active
- * array after it has expired its current timeslice. (it will not
- * continue to run immediately, it will still roundrobin with
- * other interactive tasks.)
- *
- * This part scales the interactivity limit depending on niceness.
- *
- * We scale it linearly, offset by the INTERACTIVE_DELTA delta.
- * Here are a few examples of different nice levels:
- *
- *  TASK_INTERACTIVE(-20): [1,1,1,1,1,1,1,1,1,0,0]
- *  TASK_INTERACTIVE(-10): [1,1,1,1,1,1,1,0,0,0,0]
- *  TASK_INTERACTIVE(  0): [1,1,1,1,0,0,0,0,0,0,0]
- *  TASK_INTERACTIVE( 10): [1,1,0,0,0,0,0,0,0,0,0]
- *  TASK_INTERACTIVE( 19): [0,0,0,0,0,0,0,0,0,0,0]
- *
- * (the X axis represents the possible -5 ... 0 ... +5 dynamic
- *  priority range a task can explore, a value of '1' means the
- *  task is rated interactive.)
- *
- * Ie. nice +19 tasks can never get 'interactive' enough to be
- * reinserted into the active array. And only heavily CPU-hog nice -20
- * tasks will be expired. Default nice 0 tasks are somewhere between,
- * it takes some effort for them to get interactive, but it's not
- * too hard.
+ * MIN_TIMESLICE is the timeslice that a minimum priority process gets if there
+ * is a maximum priority process runnable. MAX_TIMESLICE is derived from the
+ * formula in task_timeslice. It cannot be changed here. It is the timesilce
+ * that the maximum priority process will get. Larger timeslices are attainable
+ * by low priority processes however.
  */
+#define MIN_TIMESLICE		((1 * HZ / 1000) ? 1 * HZ / 1000 : 1)
+#define MAX_TIMESLICE		(40 * MIN_TIMESLICE) /* do not change */
 
-#define SCALE(v1,v1_max,v2_max) \
-	(v1) * (v2_max) / (v1_max)
+/* Maximum amount of history that will be used to calculate priority */
+#define MAX_SLEEP		(HZ)
 
-#define DELTA(p) \
-	(SCALE(TASK_NICE(p), 40, MAX_USER_PRIO*PRIO_BONUS_RATIO/100) + \
-		INTERACTIVE_DELTA)
+/*
+ * Maximum affect that 1 block of activity (run/sleep/etc) can have. This is
+ * will moderate dicard freak events (eg. SIGSTOP)
+ */
+#define MAX_SLEEP_AFFECT	(MAX_SLEEP/2)
 
-#define TASK_INTERACTIVE(p) \
-	((p)->prio <= (p)->static_prio - DELTA(p))
+/*
+ * The amount of history can be decreased (on fork for example). This puts a
+ * lower bound on it.
+ */
+#define MIN_HISTORY		(HZ/20)
 
 /*
- * BASE_TIMESLICE scales user-nice values [ -20 ... 19 ]
- * to time slice values.
- *
- * The higher a thread's priority, the bigger timeslices
- * it gets during one round of execution. But even the lowest
- * priority thread gets MIN_TIMESLICE worth of execution time.
- *
- * task_timeslice() is the interface that is used by the scheduler.
+ * SLEEP_FACTOR is a fixed point factor used to scale history tracking things.
+ * In particular: total_time, sleep_time, sleep_avg.
  */
+#define SLEEP_FACTOR		(1024)
 
-#define BASE_TIMESLICE(p) (MIN_TIMESLICE + \
-	((MAX_TIMESLICE - MIN_TIMESLICE) * (MAX_PRIO-1-(p)->static_prio)/(MAX_USER_PRIO - 1)))
+#define NODE_THRESHOLD		125
 
-static inline unsigned int task_timeslice(task_t *p)
-{
-	return BASE_TIMESLICE(p);
-}
+/*
+ * The scheduler classifies a process as performing one of the following
+ * activities
+ */
+#define STIME_SLEEP		1	/* Sleeping */
+#define STIME_RUN		2	/* Using CPU */
+#define STIME_WAIT		3	/* Waiting to be run */
+
+#define TASK_PREEMPTS_CURR(p, rq)			\
+	( (p)->prio < (rq)->curr->prio			\
+	 	|| ((p)->prio == (rq)->curr->prio	\
+			 && (p)->static_prio < (rq)->curr->static_prio) )
 
 /*
  * These are the runqueue data structures:
@@ -157,7 +128,8 @@ struct prio_array {
  */
 struct runqueue {
 	spinlock_t lock;
-	unsigned long nr_running, nr_switches, expired_timestamp,
+	unsigned long array_sequence;
+	unsigned long nr_running, nr_switches,
 			nr_uninterruptible;
 	task_t *curr, *idle;
 	struct mm_struct *prev_mm;
@@ -298,34 +270,94 @@ static inline void enqueue_task(struct t
 }
 
 /*
- * effective_prio - return the priority that is based on the static
- * priority but is modified by bonuses/penalties.
- *
- * We scale the actual sleep average [0 .... MAX_SLEEP_AVG]
- * into the -5 ... 0 ... +5 bonus/penalty range.
- *
- * We use 25% of the full 0...39 priority range so that:
- *
- * 1) nice +19 interactive tasks do not preempt nice 0 CPU hogs.
- * 2) nice -20 CPU hogs do not get preempted by nice 0 tasks.
+ * add_task_time updates a task @p after @time of doing the specified @type
+ * of activity. See STIME_*. This is used for priority calculation.
+ */
+static void add_task_time(task_t *p, unsigned long time, unsigned long type)
+{
+	unsigned long r;
+
+	if (time == 0)
+		return;
+
+	if (time > MAX_SLEEP_AFFECT)
+		time = MAX_SLEEP_AFFECT;
+
+	r = MAX_SLEEP - time;
+	p->total_time = (r*p->total_time + MAX_SLEEP/2) / MAX_SLEEP;
+	p->sleep_time = (r*p->sleep_time + MAX_SLEEP/2) / MAX_SLEEP;
+
+	if (type != STIME_WAIT) {
+		p->total_time += SLEEP_FACTOR * time;
+		if (type == STIME_SLEEP)
+			p->sleep_time += SLEEP_FACTOR * time;
+
+		p->sleep_avg = (SLEEP_FACTOR * p->sleep_time) / p->total_time;
+	}
+
+	if (p->total_time < SLEEP_FACTOR * MIN_HISTORY) {
+		p->total_time = SLEEP_FACTOR * MIN_HISTORY;
+		p->sleep_time = p->total_time * p->sleep_avg / SLEEP_FACTOR;
+	}
+}
+
+/*
+ * The higher a thread's priority, the bigger timeslices
+ * it gets during one round of execution. But even the lowest
+ * priority thread gets MIN_TIMESLICE worth of execution time.
  *
- * Both properties are important to certain workloads.
+ * Timeslices are scaled, so if only low priority processes are running,
+ * they will all get long timeslices.
+ */
+static unsigned int task_timeslice(task_t *p, runqueue_t *rq)
+{
+	int idx, delta;
+	unsigned int base, timeslice;
+
+	if (rt_task(p))
+		return MAX_TIMESLICE;
+
+	idx = min(find_next_bit(rq->active->bitmap, MAX_PRIO, MAX_RT_PRIO),
+			find_next_bit(rq->expired->bitmap, MAX_PRIO, MAX_RT_PRIO));
+	idx = min(idx, p->prio);
+	delta = p->prio - idx;
+
+	/*
+	 * This is a bit subtle. The first line establishes a timeslice based
+	 * on how far this task is from being the highest priority runnable.
+	 * The second line scales this result so low priority tasks will get
+	 * big timeslices if higher priority ones are not running.
+	 */
+	base = MIN_TIMESLICE * (MAX_USER_PRIO + 1) / (delta + 2);
+	timeslice = base * (USER_PRIO(idx) + 8) / 8;
+
+	if (timeslice <= 0)
+		timeslice = 1;
+
+	return timeslice;
+}
+
+/*
+ * task_priority: calculates a task's priority based on previous running
+ * history (see add_task_time). The priority is just a simple linear function
+ * based on sleep_avg and static_prio.
  */
-static int effective_prio(task_t *p)
+static unsigned long task_priority(task_t *p)
 {
 	int bonus, prio;
 
 	if (rt_task(p))
 		return p->prio;
 
-	bonus = MAX_USER_PRIO*PRIO_BONUS_RATIO*p->sleep_avg/MAX_SLEEP_AVG/100 -
-			MAX_USER_PRIO*PRIO_BONUS_RATIO/100/2;
+	bonus = (MAX_USER_PRIO * p->sleep_avg) / SLEEP_FACTOR / 2;
+	prio = USER_PRIO(p->static_prio) / 2 + (MAX_USER_PRIO / 2);
 
-	prio = p->static_prio - bonus;
+	prio = MAX_RT_PRIO + prio - bonus;
 	if (prio < MAX_RT_PRIO)
 		prio = MAX_RT_PRIO;
 	if (prio > MAX_PRIO-1)
 		prio = MAX_PRIO-1;
+
 	return prio;
 }
 
@@ -346,34 +378,37 @@ static inline void __activate_task(task_
  */
 static inline void activate_task(task_t *p, runqueue_t *rq)
 {
-	long sleep_time = jiffies - p->last_run - 1;
+	unsigned long sleep = jiffies - p->timestamp;
+	p->timestamp = jiffies;
 
-	if (sleep_time > 0) {
-		int sleep_avg;
+	if (sleep > MAX_SLEEP)
+		sleep = MAX_SLEEP;
 
-		/*
-		 * This code gives a bonus to interactive tasks.
-		 *
-		 * The boost works by updating the 'average sleep time'
-		 * value here, based on ->last_run. The more time a task
-		 * spends sleeping, the higher the average gets - and the
-		 * higher the priority boost gets as well.
-		 */
-		sleep_avg = p->sleep_avg + sleep_time;
+	/*
+	 * If woken by a userspace task, it is assumed the waker has  done
+	 * some work for us, so share some priority with it
+	 */
+	if (!in_interrupt() && current->mm) {
+		unsigned long boost = sleep / 2;
+		add_task_time(current, boost, STIME_SLEEP);
+		add_task_time(p, sleep - boost, STIME_SLEEP);
+	} else {
+		add_task_time(p, sleep, STIME_SLEEP);
 
-		/*
-		 * 'Overflow' bonus ticks go to the waker as well, so the
-		 * ticks are not lost. This has the effect of further
-		 * boosting tasks that are related to maximum-interactive
-		 * tasks.
-		 */
-		if (sleep_avg > MAX_SLEEP_AVG)
-			sleep_avg = MAX_SLEEP_AVG;
-		if (p->sleep_avg != sleep_avg) {
-			p->sleep_avg = sleep_avg;
-			p->prio = effective_prio(p);
-		}
+		if (in_interrupt())
+			add_task_time(p, sleep / 2, STIME_SLEEP);
 	}
+
+	p->prio = task_priority(p);
+
+	/*
+	 * If we have slept through an active/expired array switch, restart
+	 * our timeslice too.
+	 */
+	if (rq->array_sequence != p->array_sequence) {
+		p->used_slice = 0;
+	}
+
 	__activate_task(p, rq);
 }
 
@@ -382,6 +417,7 @@ static inline void activate_task(task_t 
  */
 static inline void deactivate_task(struct task_struct *p, runqueue_t *rq)
 {
+	p->array_sequence = rq->array_sequence;
 	nr_running_dec(rq);
 	if (p->state == TASK_UNINTERRUPTIBLE)
 		rq->nr_uninterruptible++;
@@ -425,7 +461,7 @@ static inline void resched_task(task_t *
  * be called with interrupts off, or it may introduce deadlock with
  * smp_call_function() if an IPI is sent by the same process we are
  * waiting to become inactive.
- */
+ n*/
 void wait_task_inactive(task_t * p)
 {
 	unsigned long flags;
@@ -496,11 +532,9 @@ repeat_lock_task:
 			}
 			if (old_state == TASK_UNINTERRUPTIBLE)
 				rq->nr_uninterruptible--;
-			if (sync)
-				__activate_task(p, rq);
-			else {
-				activate_task(p, rq);
-				if (p->prio < rq->curr->prio)
+			activate_task(p, rq);
+			if (!sync) {
+				if (TASK_PREEMPTS_CURR(p, rq))
 					resched_task(rq->curr);
 			}
 			success = 1;
@@ -538,27 +572,39 @@ int wake_up_state(task_t *p, unsigned in
  */
 void sched_fork(task_t *p)
 {
+	unsigned long ts;
+	unsigned long flags;
+	runqueue_t *rq;
+
 	/*
 	 * Share the timeslice between parent and child, thus the
 	 * total amount of pending timeslices in the system doesn't change,
 	 * resulting in more scheduling fairness.
 	 */
 	local_irq_disable();
-        p->time_slice = (current->time_slice + 1) >> 1;
+	p->timestamp = jiffies;
+	rq = task_rq_lock(current, &flags);
+	ts = task_timeslice(current, rq);
+	task_rq_unlock(rq, &flags);
+
+	/*
+	 * Share half our timeslice with the child.
+	 */
+	p->used_slice = current->used_slice + (ts - current->used_slice) / 2;
+	current->used_slice += (ts - current->used_slice + 1) / 2;
+
 	/*
 	 * The remainder of the first timeslice might be recovered by
 	 * the parent if the child exits early enough.
 	 */
 	p->first_time_slice = 1;
-	current->time_slice >>= 1;
-	p->last_run = jiffies;
-	if (!current->time_slice) {
+	if (current->used_slice >= ts) {
 		/*
 	 	 * This case is rare, it happens when the parent has only
 	 	 * a single jiffy left from its timeslice. Taking the
 		 * runqueue lock is not a problem.
 		 */
-		current->time_slice = 1;
+		current->used_slice = ts - 1;
 		preempt_disable();
 		scheduler_tick(0, 0);
 		local_irq_enable();
@@ -573,31 +619,43 @@ void sched_fork(task_t *p)
  * This function will do some initial scheduler statistics housekeeping
  * that must be done for every newly created process.
  */
-void wake_up_forked_process(task_t * p)
+void wake_up_forked_process(task_t *p)
 {
 	unsigned long flags;
 	runqueue_t *rq = task_rq_lock(current, &flags);
 
 	p->state = TASK_RUNNING;
-	/*
-	 * We decrease the sleep average of forking parents
-	 * and children as well, to keep max-interactive tasks
-	 * from forking tasks that are max-interactive.
-	 */
-	current->sleep_avg = current->sleep_avg * PARENT_PENALTY / 100;
-	p->sleep_avg = p->sleep_avg * CHILD_PENALTY / 100;
-	p->prio = effective_prio(p);
+
 	set_task_cpu(p, smp_processor_id());
 
-	if (unlikely(!current->array))
-		__activate_task(p, rq);
-	else {
-		p->prio = current->prio;
-		list_add_tail(&p->run_list, &current->run_list);
-		p->array = current->array;
-		p->array->nr_active++;
-		nr_running_inc(rq);
+	/*
+	 * Get only 1/10th of the parents history, but at a much higher
+	 * priority. Limited by MIN_HISTORY.
+	 */
+	p->total_time = current->total_time / 10;
+	p->sleep_time = current->sleep_time / 10
+			+ (current->total_time - current->sleep_time) / 20;
+	if (p->total_time != 0)
+		p->sleep_avg = (SLEEP_FACTOR * p->sleep_time) / p->total_time;
+	else
+		p->sleep_avg = SLEEP_FACTOR / 2;
+
+	if (p->total_time < SLEEP_FACTOR * MIN_HISTORY) {
+		p->total_time = SLEEP_FACTOR * MIN_HISTORY;
+		p->sleep_time = p->total_time * p->sleep_avg / SLEEP_FACTOR;
 	}
+
+	/*
+	 * Lose 1/4 sleep_time for forking.
+	 */
+	current->sleep_time = 3 * current->sleep_time / 4;
+	if (current->total_time != 0)
+		current->sleep_avg = (SLEEP_FACTOR * current->sleep_time)
+						/ current->total_time;
+
+	p->prio = task_priority(p);
+	__activate_task(p, rq);
+
 	task_rq_unlock(rq, &flags);
 }
 
@@ -615,19 +673,24 @@ void sched_exit(task_t * p)
 	unsigned long flags;
 
 	local_irq_save(flags);
+
+	/* Regain the unused timeslice given to @p by its parent */
 	if (p->first_time_slice) {
-		p->parent->time_slice += p->time_slice;
-		if (unlikely(p->parent->time_slice > MAX_TIMESLICE))
-			p->parent->time_slice = MAX_TIMESLICE;
+		unsigned long flags;
+		runqueue_t *rq;
+		rq = task_rq_lock(p, &flags);
+		p->parent->used_slice -= task_timeslice(p, rq) - p->used_slice;
+		task_rq_unlock(rq, &flags);
+	}
+
+	/* Apply some penalty to @p's parent if @p used a lot of CPU */
+	if (p->sleep_avg < p->parent->sleep_avg) {
+		add_task_time(p->parent,
+				(p->parent->sleep_avg - p->sleep_avg)/2,
+				STIME_RUN);
 	}
+
 	local_irq_restore(flags);
-	/*
-	 * If the child was a (relative-) CPU hog then decrease
-	 * the sleep_avg of the parent as well.
-	 */
-	if (p->sleep_avg < p->parent->sleep_avg)
-		p->parent->sleep_avg = (p->parent->sleep_avg * EXIT_WEIGHT +
-			p->sleep_avg) / (EXIT_WEIGHT + 1);
 }
 
 /**
@@ -1029,13 +1092,8 @@ 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 (p->prio < this_rq->curr->prio)
+	if (TASK_PREEMPTS_CURR(p, this_rq))
 		set_need_resched();
-	else {
-		if (p->prio == this_rq->curr->prio &&
-				p->time_slice > this_rq->curr->time_slice)
-			set_need_resched();
-	}
 }
 
 /*
@@ -1063,7 +1121,7 @@ can_migrate_task(task_t *p, runqueue_t *
 		return 0;
 
 	/* Aggressive migration if we're idle */
-	delta = jiffies - p->last_run;
+	delta = jiffies - p->timestamp;
 	if (!idle && (delta <= cache_decay_ticks))
 		return 0;
 
@@ -1233,20 +1291,6 @@ DEFINE_PER_CPU(struct kernel_stat, kstat
 EXPORT_PER_CPU_SYMBOL(kstat);
 
 /*
- * We place interactive tasks back into the active array, if possible.
- *
- * To guarantee that this does not starve expired tasks we ignore the
- * interactivity of a task if the first expired task had to wait more
- * than a 'reasonable' amount of time. This deadline timeout is
- * load-dependent, as the frequency of array switched decreases with
- * increasing number of running tasks:
- */
-#define EXPIRED_STARVING(rq) \
-		(STARVATION_LIMIT && ((rq)->expired_timestamp && \
-		(jiffies - (rq)->expired_timestamp >= \
-			STARVATION_LIMIT * ((rq)->nr_running) + 1)))
-
-/*
  * This function gets called by the timer code, with HZ frequency.
  * We call it with interrupts disabled.
  *
@@ -1263,17 +1307,11 @@ void scheduler_tick(int user_ticks, int 
 	if (rcu_pending(cpu))
 		rcu_check_callbacks(cpu, user_ticks);
 
-	/* note: this timer irq context must be accounted for as well */
-	if (hardirq_count() - HARDIRQ_OFFSET) {
-		cpustat->irq += sys_ticks;
-		sys_ticks = 0;
-	} else if (softirq_count()) {
-		cpustat->softirq += sys_ticks;
-		sys_ticks = 0;
-	}
-
 	if (p == rq->idle) {
-		if (atomic_read(&rq->nr_iowait) > 0)
+		/* note: this timer irq context must be accounted for as well */
+		if (irq_count() - HARDIRQ_OFFSET >= SOFTIRQ_OFFSET)
+			cpustat->system += sys_ticks;
+		else if (atomic_read(&rq->nr_iowait) > 0)
 			cpustat->iowait += sys_ticks;
 		else
 			cpustat->idle += sys_ticks;
@@ -1294,43 +1332,39 @@ void scheduler_tick(int user_ticks, int 
 	spin_lock(&rq->lock);
 	/*
 	 * The task was running during this tick - update the
-	 * time slice counter and the sleep average. Note: we
-	 * do not update a thread's priority until it either
-	 * goes to sleep or uses up its timeslice. This makes
-	 * it possible for interactive tasks to use up their
-	 * timeslices at their highest priority levels.
+	 * time slice counter. Note: we do not update a thread's
+	 * priority until it either goes to sleep or uses up its
+	 * timeslice.
 	 */
-	if (p->sleep_avg)
-		p->sleep_avg--;
 	if (unlikely(rt_task(p))) {
 		/*
 		 * RR tasks need a special form of timeslice management.
 		 * FIFO tasks have no timeslices.
 		 */
-		if ((p->policy == SCHED_RR) && !--p->time_slice) {
-			p->time_slice = task_timeslice(p);
-			p->first_time_slice = 0;
-			set_tsk_need_resched(p);
-
-			/* put it at the end of the queue: */
-			dequeue_task(p, rq->active);
-			enqueue_task(p, rq->active);
+		if (p->policy == SCHED_RR) {
+			p->used_slice++;
+			if (p->used_slice >= task_timeslice(p, rq)) {
+				p->used_slice = 0;
+				p->first_time_slice = 0;
+				set_tsk_need_resched(p);
+
+				/* put it at the end of the queue: */
+				dequeue_task(p, rq->active);
+				enqueue_task(p, rq->active);
+			}
 		}
 		goto out_unlock;
 	}
-	if (!--p->time_slice) {
+
+	p->used_slice++;
+	if (p->used_slice >= task_timeslice(p, rq)) {
 		dequeue_task(p, rq->active);
 		set_tsk_need_resched(p);
-		p->prio = effective_prio(p);
-		p->time_slice = task_timeslice(p);
+		p->prio = task_priority(p);
+		p->used_slice = 0;
 		p->first_time_slice = 0;
 
-		if (!TASK_INTERACTIVE(p) || EXPIRED_STARVING(rq)) {
-			if (!rq->expired_timestamp)
-				rq->expired_timestamp = jiffies;
-			enqueue_task(p, rq->expired);
-		} else
-			enqueue_task(p, rq->active);
+		enqueue_task(p, rq->expired);
 	}
 out_unlock:
 	spin_unlock(&rq->lock);
@@ -1349,6 +1383,8 @@ asmlinkage void schedule(void)
 	runqueue_t *rq;
 	prio_array_t *array;
 	struct list_head *queue;
+	unsigned long now;
+	unsigned long run_time;
 	int idx;
 
 	/*
@@ -1369,7 +1405,11 @@ need_resched:
 	rq = this_rq();
 
 	release_kernel_lock(prev);
-	prev->last_run = jiffies;
+	now = jiffies;
+	run_time = now - prev->timestamp;
+
+	add_task_time(prev, run_time, STIME_RUN);
+
 	spin_lock_irq(&rq->lock);
 
 	/*
@@ -1400,7 +1440,6 @@ pick_next_task:
 			goto pick_next_task;
 #endif
 		next = rq->idle;
-		rq->expired_timestamp = 0;
 		goto switch_tasks;
 	}
 
@@ -1409,10 +1448,10 @@ pick_next_task:
 		/*
 		 * Switch the active and expired arrays.
 		 */
+		rq->array_sequence++;
 		rq->active = rq->expired;
 		rq->expired = array;
 		array = rq->active;
-		rq->expired_timestamp = 0;
 	}
 
 	idx = sched_find_first_bit(array->bitmap);
@@ -1424,7 +1463,10 @@ switch_tasks:
 	clear_tsk_need_resched(prev);
 	RCU_qsctr(task_cpu(prev))++;
 
+	prev->timestamp = now;
 	if (likely(prev != next)) {
+		add_task_time(next, now - next->timestamp, STIME_WAIT);
+		next->timestamp = now;
 		rq->nr_switches++;
 		rq->curr = next;
 
@@ -1664,6 +1706,7 @@ void set_user_nice(task_t *p, long nice)
 	unsigned long flags;
 	prio_array_t *array;
 	runqueue_t *rq;
+	int old_prio, new_prio, delta;
 
 	if (TASK_NICE(p) == nice || nice < -20 || nice > 19)
 		return;
@@ -1672,6 +1715,12 @@ void set_user_nice(task_t *p, long nice)
 	 * the task might be in the middle of scheduling on another CPU.
 	 */
 	rq = task_rq_lock(p, &flags);
+	/*
+	 * The RT priorities are set via setscheduler(), but we still
+	 * allow the 'normal' nice value to be set - but as expected
+	 * it wont have any effect on scheduling until the task is
+	 * not SCHED_NORMAL:
+	 */
 	if (rt_task(p)) {
 		p->static_prio = NICE_TO_PRIO(nice);
 		goto out_unlock;
@@ -1679,16 +1728,20 @@ void set_user_nice(task_t *p, long nice)
 	array = p->array;
 	if (array)
 		dequeue_task(p, array);
+
+	old_prio = p->prio;
+	new_prio = NICE_TO_PRIO(nice);
+	delta = new_prio - old_prio;
 	p->static_prio = NICE_TO_PRIO(nice);
-	p->prio = NICE_TO_PRIO(nice);
+	p->prio += delta;
+
 	if (array) {
 		enqueue_task(p, array);
 		/*
-		 * If the task is running and lowered its priority,
-		 * or increased its priority then reschedule its CPU:
+		 * If the task increased its priority or is running and
+		 * lowered its priority, then reschedule its CPU:
 		 */
-		if ((NICE_TO_PRIO(nice) < p->static_prio) ||
-							task_running(rq, p))
+		if (delta < 0 || (delta > 0 && task_running(rq, p)))
 			resched_task(rq->curr);
 	}
 out_unlock:
@@ -2210,6 +2263,8 @@ asmlinkage long sys_sched_rr_get_interva
 	int retval = -EINVAL;
 	struct timespec t;
 	task_t *p;
+	unsigned long flags;
+	runqueue_t *rq;
 
 	if (pid < 0)
 		goto out_nounlock;
@@ -2224,8 +2279,10 @@ asmlinkage long sys_sched_rr_get_interva
 	if (retval)
 		goto out_unlock;
 
+	rq = task_rq_lock(p, &flags);
 	jiffies_to_timespec(p->policy & SCHED_FIFO ?
-				0 : task_timeslice(p), &t);
+				0 : task_timeslice(p, rq), &t);
+	task_rq_unlock(rq, &flags);
 	read_unlock(&tasklist_lock);
 	retval = copy_to_user(interval, &t, sizeof(t)) ? -EFAULT : 0;
 out_nounlock:

_