diff -upN reference/arch/i386/Kconfig current/arch/i386/Kconfig
--- reference/arch/i386/Kconfig	2004-04-29 10:39:28.000000000 -0700
+++ current/arch/i386/Kconfig	2004-04-29 10:39:29.000000000 -0700
@@ -1597,6 +1597,19 @@ config MAGIC_SYSRQ
 config 4KSTACKS
 	def_bool y
 
+config SCHEDSTATS
+	bool "Collect scheduler statistics"
+	depends on PROC_FS
+	default y
+	help
+	  If you say Y here, additional code will be inserted into the
+	  scheduler and related routines to collect statistics about
+	  scheduler behavior and provide them in /proc/schedstat.  These
+	  stats may be useful for both tuning and debugging the scheduler
+	  If you aren't debugging the scheduler or trying to tune a specific
+	  application, you can say N to avoid the very slight overhead
+	  this adds.
+
 config X86_FIND_SMP_CONFIG
 	bool
 	depends on X86_LOCAL_APIC || X86_VOYAGER
diff -upN reference/arch/ppc/Kconfig current/arch/ppc/Kconfig
--- reference/arch/ppc/Kconfig	2004-04-07 14:53:59.000000000 -0700
+++ current/arch/ppc/Kconfig	2004-04-29 10:39:29.000000000 -0700
@@ -1209,6 +1209,19 @@ config DEBUG_INFO
 	  debug the kernel.
 	  If you don't debug the kernel, you can say N.
 
+config SCHEDSTATS
+	bool "Collect scheduler statistics"
+	depends on PROC_FS
+	default y
+	help
+	  If you say Y here, additional code will be inserted into the
+	  scheduler and related routines to collect statistics about
+	  scheduler behavior and provide them in /proc/schedstat.  These
+	  stats may be useful for both tuning and debugging the scheduler
+	  If you aren't debugging the scheduler or trying to tune a specific
+	  application, you can say N to avoid the very slight overhead
+	  this adds.
+	  
 config BOOTX_TEXT
 	bool "Support for early boot text console (BootX or OpenFirmware only)"
 	depends PPC_OF
diff -upN reference/arch/ppc64/Kconfig current/arch/ppc64/Kconfig
--- reference/arch/ppc64/Kconfig	2004-04-29 10:39:18.000000000 -0700
+++ current/arch/ppc64/Kconfig	2004-04-29 10:39:29.000000000 -0700
@@ -396,6 +396,19 @@ config DEBUG_INFO
 	  Say Y here only if you plan to use gdb to debug the kernel.
 	  If you don't debug the kernel, you can say N.
 	  
+config SCHEDSTATS
+	bool "Collect scheduler statistics"
+	depends on PROC_FS
+	default y
+	help
+	  If you say Y here, additional code will be inserted into the
+	  scheduler and related routines to collect statistics about
+	  scheduler behavior and provide them in /proc/schedstat.  These
+	  stats may be useful for both tuning and debugging the scheduler
+	  If you aren't debugging the scheduler or trying to tune a specific
+	  application, you can say N to avoid the very slight overhead
+	  this adds.
+
 config MCOUNT
 	bool "Generate function call graph"
 	depends on DEBUG_KERNEL
diff -upN reference/arch/x86_64/Kconfig current/arch/x86_64/Kconfig
--- reference/arch/x86_64/Kconfig	2004-04-07 14:54:02.000000000 -0700
+++ current/arch/x86_64/Kconfig	2004-04-29 10:39:29.000000000 -0700
@@ -468,6 +468,19 @@ config DEBUG_INFO
 	  Say Y here only if you plan to use gdb to debug the kernel.
 	  Please note that this option requires new binutils.
 	  If you don't debug the kernel, you can say N.
+
+config SCHEDSTATS
+	bool "Collect scheduler statistics"
+	depends on PROC_FS
+	default y 
+	help
+	  If you say Y here, additional code will be inserted into the
+	  scheduler and related routines to collect statistics about
+	  scheduler behavior and provide them in /proc/schedstat.  These
+	  stats may be useful for both tuning and debugging the scheduler
+	  If you aren't debugging the scheduler or trying to tune a specific
+	  application, you can say N to avoid the very slight overhead
+	  this adds.
 	  
 config FRAME_POINTER
        bool "Compile the kernel with frame pointers"
diff -upN reference/fs/proc/array.c current/fs/proc/array.c
--- reference/fs/proc/array.c	2004-03-11 14:35:11.000000000 -0800
+++ current/fs/proc/array.c	2004-04-29 10:39:29.000000000 -0700
@@ -345,9 +345,15 @@ int proc_pid_stat(struct task_struct *ta
 	read_lock(&tasklist_lock);
 	ppid = task->pid ? task->real_parent->pid : 0;
 	read_unlock(&tasklist_lock);
+#ifdef CONFIG_SCHEDSTATS
+	res = sprintf(buffer,"%d (%s) %c %d %d %d %d %d %lu %lu \
+%lu %lu %lu %lu %lu %ld %ld %ld %ld %d %ld %llu %lu %ld %lu %lu %lu %lu %lu \
+%lu %lu %lu %lu %lu %lu %lu %lu %d %d %lu %lu %lu %lu %lu\n",
+#else
 	res = sprintf(buffer,"%d (%s) %c %d %d %d %d %d %lu %lu \
 %lu %lu %lu %lu %lu %ld %ld %ld %ld %d %ld %llu %lu %ld %lu %lu %lu %lu %lu \
 %lu %lu %lu %lu %lu %lu %lu %lu %d %d %lu %lu\n",
+#endif	/* CONFIG_SCHEDSTATS */
 		task->pid,
 		task->comm,
 		state,
@@ -393,7 +399,14 @@ int proc_pid_stat(struct task_struct *ta
 		task->exit_signal,
 		task_cpu(task),
 		task->rt_priority,
+#ifdef CONFIG_SCHEDSTATS
+		task->policy,
+		task->sched_info.cpu_time,
+		task->sched_info.run_delay,
+		task->sched_info.pcnt);
+#else
 		task->policy);
+#endif /* CONFIG_SCHEDSTATS */
 	if(mm)
 		mmput(mm);
 	return res;
diff -upN reference/fs/proc/proc_misc.c current/fs/proc/proc_misc.c
--- reference/fs/proc/proc_misc.c	2004-04-29 10:39:18.000000000 -0700
+++ current/fs/proc/proc_misc.c	2004-04-29 10:39:29.000000000 -0700
@@ -325,6 +325,10 @@ static struct file_operations proc_vmsta
 	.release	= seq_release,
 };
 
+#ifdef CONFIG_SCHEDSTATS
+extern struct file_operations proc_schedstat_operations;
+#endif
+
 #ifdef CONFIG_PROC_HARDWARE
 static int hardware_read_proc(char *page, char **start, off_t off,
 				 int count, int *eof, void *data)
@@ -836,6 +840,9 @@ void __init proc_misc_init(void)
 #ifdef CONFIG_NUMA
 	create_seq_entry("meminfo.numa",0,&proc_meminfo_numa_operations);
 #endif
+#ifdef CONFIG_SCHEDSTATS
+	create_seq_entry("schedstat", 0, &proc_schedstat_operations);
+#endif
 #ifdef CONFIG_PROC_KCORE
 	proc_root_kcore = create_proc_entry("kcore", S_IRUSR, NULL);
 	if (proc_root_kcore) {
diff -upN reference/include/linux/sched.h current/include/linux/sched.h
--- reference/include/linux/sched.h	2004-04-29 10:39:27.000000000 -0700
+++ current/include/linux/sched.h	2004-04-29 10:39:29.000000000 -0700
@@ -101,6 +101,16 @@ extern unsigned long nr_running_cpu(int 
 extern unsigned long nr_uninterruptible(void);
 extern unsigned long nr_iowait(void);
 
+#ifdef CONFIG_SCHEDSTATS
+struct sched_info;
+extern void cpu_sched_info(struct sched_info *, int);
+#define schedstat_inc(cpu, field)	schedstats[cpu].field++;
+#define schedstat_add(cpu, field, amt)	schedstats[cpu].field += amt;
+#else
+#define schedstat_inc(cpu, field)	do { } while (0);
+#define schedstat_add(cpu, field, amt)	do { } while (0);
+#endif
+
 #include <linux/time.h>
 #include <linux/param.h>
 #include <linux/resource.h>
@@ -336,6 +346,18 @@ struct k_itimer {
 	struct sigqueue *sigq;		/* signal queue entry. */
 };
 
+#ifdef CONFIG_SCHEDSTATS
+struct sched_info {
+	/* cumulative counters */
+	unsigned long	cpu_time,	/* time spent on the cpu */
+			run_delay,	/* time spent waiting on a runqueue */
+			pcnt;		/* # of timeslices run on this cpu */
+
+	/* timestamps */
+	unsigned long	last_arrival,	/* when we last ran on a cpu */
+			last_queued;	/* when we were last queued to run */
+};
+#endif /* CONFIG_SCHEDSTATS */
 
 struct io_context;			/* See blkdev.h */
 void exit_io_context(void);
@@ -389,6 +411,10 @@ struct task_struct {
 	cpumask_t cpus_allowed;
 	unsigned int time_slice, first_time_slice;
 
+#ifdef CONFIG_SCHEDSTATS
+	struct sched_info sched_info;
+#endif /* CONFIG_SCHEDSTATS */
+
 	struct list_head tasks;
 	struct list_head ptrace_children;
 	struct list_head ptrace_list;
diff -upN reference/kernel/fork.c current/kernel/fork.c
--- reference/kernel/fork.c	2004-04-29 10:39:23.000000000 -0700
+++ current/kernel/fork.c	2004-04-29 10:39:29.000000000 -0700
@@ -969,6 +969,9 @@ struct task_struct *copy_process(unsigne
 	p->security = NULL;
 	p->io_context = NULL;
 	p->io_wait = NULL;
+#ifdef CONFIG_SCHEDSTATS
+	memset(&p->sched_info, 0, sizeof(p->sched_info));
+#endif /* CONFIG_SCHEDSTATS */
 
 	retval = -ENOMEM;
 	if ((retval = security_task_alloc(p)))
diff -upN reference/kernel/sched.c current/kernel/sched.c
--- reference/kernel/sched.c	2004-04-29 10:39:29.000000000 -0700
+++ current/kernel/sched.c	2004-04-29 10:39:29.000000000 -0700
@@ -41,6 +41,8 @@
 #include <linux/cpu.h>
 #include <linux/percpu.h>
 #include <linux/kthread.h>
+#include <linux/seq_file.h>
+#include <linux/times.h>
 
 #ifdef CONFIG_NUMA
 #define cpu_to_node_mask(cpu) node_to_cpumask(cpu_to_node(cpu))
@@ -230,6 +232,9 @@ struct runqueue {
 	unsigned long expired_timestamp, nr_uninterruptible;
 	unsigned long long timestamp_last_tick;
 	task_t *curr, *idle;
+#ifdef CONFIG_SCHEDSTATS
+	int cpu;  /* to make easy reverse-lookups with per-cpu runqueues */
+#endif
 	struct mm_struct *prev_mm;
 	prio_array_t *active, *expired, arrays[2];
 	int best_expired_prio;
@@ -245,6 +250,10 @@ struct runqueue {
 	task_t *migration_thread;
 	struct list_head migration_queue;
 #endif
+
+#ifdef CONFIG_SCHEDSTATS
+	struct sched_info info;
+#endif
 };
 
 static DEFINE_PER_CPU(struct runqueue, runqueues);
@@ -291,6 +300,172 @@ static inline void task_rq_unlock(runque
 	spin_unlock_irqrestore(&rq->lock, *flags);
 }
 
+
+#ifdef CONFIG_SCHEDSTATS
+struct schedstat {
+	/* sys_sched_yield stats */
+	unsigned long yld_exp_empty;
+	unsigned long yld_act_empty;
+	unsigned long yld_both_empty;
+	unsigned long yld_cnt;
+
+	/* schedule stats */
+	unsigned long sched_noswitch;
+	unsigned long sched_switch;
+	unsigned long sched_cnt;
+
+	/* load_balance stats */
+	unsigned long lb_imbalance;
+	unsigned long lb_idle;
+	unsigned long lb_busy;
+	unsigned long lb_cnt;
+	unsigned long lb_nobusyg;
+	unsigned long lb_nobusyq;
+
+	/* pull_task stats */
+	unsigned long pt_gained_newidle;
+	unsigned long pt_lost_newidle;
+	unsigned long pt_gained_idle;
+	unsigned long pt_lost_idle;
+	unsigned long pt_gained_notidle;
+	unsigned long pt_lost_notidle;
+
+	/* active_load_balance stats */
+	unsigned long alb_cnt;
+	unsigned long alb_gained;
+	unsigned long alb_lost;
+
+	/* load_balance_newidle stats */
+	unsigned long lbni_cnt;
+	unsigned long lbni_imbalance;
+
+	/* migrate_to_cpu stats */
+	unsigned long mtc_cnt;
+
+	/* sched_balance_exec stats */
+	unsigned long sbe_cnt;
+} ____cacheline_aligned;
+
+/*
+ * bump this up when changing the output format or the meaning of an existing
+ * format, so that tools can adapt (or abort)
+ */
+#define SCHEDSTAT_VERSION	6
+
+struct schedstat schedstats[NR_CPUS];
+
+static int show_schedstat(struct seq_file *seq, void *v)
+{
+	struct schedstat sums;
+	int i;
+	struct sched_info info, infosums;
+
+
+	memset(&sums, 0, sizeof(sums));
+	memset(&infosums, 0, sizeof(infosums));
+	seq_printf(seq, "version %d\n", SCHEDSTAT_VERSION);
+	seq_printf(seq, "timestamp %lu\n", jiffies);
+	for (i = 0; i < NR_CPUS; i++) {
+
+		if (!cpu_online(i)) continue;
+
+		cpu_sched_info(&info, i);
+
+		sums.yld_exp_empty += schedstats[i].yld_exp_empty;
+		sums.yld_act_empty += schedstats[i].yld_act_empty;
+		sums.yld_both_empty += schedstats[i].yld_both_empty;
+		sums.yld_cnt += schedstats[i].yld_cnt;
+		sums.sched_noswitch += schedstats[i].sched_noswitch;
+		sums.sched_switch += schedstats[i].sched_switch;
+		sums.sched_cnt += schedstats[i].sched_cnt;
+		sums.lb_idle += schedstats[i].lb_idle;
+		sums.lb_busy += schedstats[i].lb_busy;
+		sums.lb_cnt += schedstats[i].lb_cnt;
+		sums.lb_imbalance += schedstats[i].lb_imbalance;
+		sums.lb_nobusyg += schedstats[i].lb_nobusyg;
+		sums.lb_nobusyq += schedstats[i].lb_nobusyq;
+		sums.pt_gained_newidle += schedstats[i].pt_gained_newidle;
+		sums.pt_lost_newidle += schedstats[i].pt_lost_newidle;
+		sums.pt_gained_idle += schedstats[i].pt_gained_idle;
+		sums.pt_lost_idle += schedstats[i].pt_lost_idle;
+		sums.pt_gained_notidle += schedstats[i].pt_gained_notidle;
+		sums.pt_lost_notidle += schedstats[i].pt_lost_notidle;
+		sums.alb_cnt += schedstats[i].alb_cnt;
+		sums.alb_gained += schedstats[i].alb_gained;
+		sums.alb_lost += schedstats[i].alb_lost;
+		sums.sbe_cnt += schedstats[i].sbe_cnt;
+		sums.mtc_cnt += schedstats[i].mtc_cnt;
+		sums.lbni_cnt += schedstats[i].lbni_cnt;
+		sums.lbni_imbalance += schedstats[i].lbni_imbalance;
+		infosums.cpu_time += info.cpu_time;
+		infosums.run_delay += info.run_delay;
+		infosums.pcnt += info.pcnt;
+		seq_printf(seq, 
+		    "cpu%d %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu "
+		    "%lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu "
+		    "%lu %lu %lu %lu %lu\n",
+		    i, schedstats[i].yld_both_empty,
+		    schedstats[i].yld_act_empty, schedstats[i].yld_exp_empty,
+		    schedstats[i].yld_cnt, schedstats[i].sched_noswitch,
+		    schedstats[i].sched_switch, schedstats[i].sched_cnt,
+		    schedstats[i].lb_idle, schedstats[i].lb_busy,
+		    schedstats[i].lb_cnt, schedstats[i].lb_imbalance,
+		    schedstats[i].lb_nobusyg, schedstats[i].lb_nobusyq,
+		    schedstats[i].pt_gained_newidle, schedstats[i].pt_lost_newidle,
+		    schedstats[i].pt_gained_idle, schedstats[i].pt_lost_idle,
+		    schedstats[i].pt_gained_notidle, schedstats[i].pt_lost_notidle,
+		    schedstats[i].alb_cnt,
+		    schedstats[i].alb_gained, schedstats[i].alb_lost,
+		    schedstats[i].sbe_cnt, schedstats[i].mtc_cnt,
+		    schedstats[i].lbni_cnt, schedstats[i].lbni_imbalance,
+		    info.cpu_time, info.run_delay, info.pcnt);
+	}
+	seq_printf(seq, 
+	    "totals %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu "
+	    "%lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu %lu "
+	    "%lu %lu %lu %lu %lu\n",
+	    sums.yld_both_empty, sums.yld_act_empty, sums.yld_exp_empty,
+	    sums.yld_cnt, sums.sched_noswitch, sums.sched_switch,
+	    sums.sched_cnt, sums.lb_idle, sums.lb_busy,
+	    sums.lb_cnt, sums.lb_imbalance, sums.lb_nobusyg, sums.lb_nobusyq,
+	    sums.pt_gained_newidle, sums.pt_lost_newidle,
+	    sums.pt_gained_idle, sums.pt_lost_idle,
+	    sums.pt_gained_notidle, sums.pt_lost_notidle,
+	    sums.alb_cnt, sums.alb_gained, sums.alb_lost,
+	    sums.sbe_cnt, sums.mtc_cnt,
+	    sums.lbni_cnt, sums.lbni_imbalance,
+	    infosums.cpu_time, infosums.run_delay, infosums.pcnt);
+
+	return 0;
+}
+
+static int schedstat_open(struct inode *inode, struct file *file)
+{
+	unsigned size = 4096 * (1 + num_online_cpus() / 32);
+	char *buf = kmalloc(size, GFP_KERNEL);
+	struct seq_file *m;
+	int res;
+
+	if (!buf)
+		return -ENOMEM;
+	res = single_open(file, show_schedstat, NULL);
+	if (!res) {
+		m = file->private_data;
+		m->buf = buf;
+		m->size = size;
+	} else
+		kfree(buf);
+	return res;
+}
+
+struct file_operations proc_schedstat_operations = {
+	.open    = schedstat_open,
+	.read    = seq_read,
+	.llseek  = seq_lseek,
+	.release = single_release,
+};
+#endif
+
 /*
  * rq_lock - lock a given runqueue and disable interrupts.
  */
@@ -310,6 +485,113 @@ static inline void rq_unlock(runqueue_t 
 	spin_unlock_irq(&rq->lock);
 }
 
+#ifdef CONFIG_SCHEDSTATS
+/*
+ * Called when a process is dequeued from the active array and given
+ * the cpu.  We should note that with the exception of interactive
+ * tasks, the expired queue will become the active queue after the active
+ * queue is empty, without explicitly dequeuing and requeuing tasks in the
+ * expired queue.  (Interactive tasks may be requeued directly to the
+ * active queue, thus delaying tasks in the expired queue from running;
+ * see scheduler_tick()).
+ *
+ * This function is only called from sched_info_arrive(), rather than
+ * dequeue_task(). Even though a task may be queued and dequeued multiple
+ * times as it is shuffled about, we're really interested in knowing how
+ * long it was from the *first* time it was queued to the time that it
+ * finally hit a cpu.
+ */
+static inline void sched_info_dequeued(task_t *t)
+{
+	t->sched_info.last_queued = 0;
+}
+
+/*
+ * Called when a task finally hits the cpu.  We can now calculate how
+ * long it was waiting to run.  We also note when it began so that we
+ * can keep stats on how long its timeslice is.
+ */
+static inline void sched_info_arrive(task_t *t)
+{
+	unsigned long now  = jiffies;
+	unsigned long diff = 0;
+	struct runqueue *rq = task_rq(t);
+
+	if (t->sched_info.last_queued)
+		diff = now - t->sched_info.last_queued;
+	sched_info_dequeued(t);
+	t->sched_info.run_delay += diff;
+	t->sched_info.last_arrival = now;
+	t->sched_info.pcnt++;
+
+	if (!rq)
+		return;
+	
+	rq->info.run_delay += diff;
+	rq->info.pcnt++;
+}
+
+/*
+ * Called when a process is queued into either the active or expired
+ * array.  The time is noted and later used to determine how long we
+ * had to wait for us to reach the cpu.  Since the expired queue will
+ * become the active queue after active queue is empty, without dequeuing
+ * and requeuing any tasks, we are interested in queuing to either. It
+ * is unusual but not impossible for tasks to be dequeued and immediately
+ * requeued in the same or another array: this can happen in sched_yield(),
+ * set_user_nice(), and even load_balance() as it moves tasks from runqueue
+ * to runqueue.
+ *
+ * This function is only called from enqueue_task(), but also only updates
+ * the timestamp if it is already not set.  It's assumed that
+ * sched_info_dequeued() will clear that stamp when appropriate.
+ */
+static inline void sched_info_queued(task_t *t)
+{
+	if (!t->sched_info.last_queued)
+		t->sched_info.last_queued = jiffies;
+}
+
+/*
+ * Called when a process ceases being the active-running process, either
+ * voluntarily or involuntarily.  Now we can calculate how long we ran.
+ */
+static inline void sched_info_depart(task_t *t)
+{
+	struct runqueue *rq = task_rq(t);
+	unsigned long diff = jiffies - t->sched_info.last_arrival;
+
+	t->sched_info.cpu_time += diff;
+
+	if (rq)
+		rq->info.cpu_time += diff;
+}
+
+/*
+ * Called when tasks are switched involuntarily due, typically, to expiring
+ * their time slice.  (This may also be called when switching to or from
+ * the idle task.)  We are only called when prev != next.
+ */
+static inline void sched_info_switch(task_t *prev, task_t *next)
+{
+	struct runqueue *rq = task_rq(prev);
+
+	/*
+	 * prev now departs the cpu.  It's not interesting to record
+	 * stats about how efficient we were at scheduling the idle
+	 * process, however.
+	 */
+	if (prev != rq->idle)
+		sched_info_depart(prev);
+
+	if (next != rq->idle)
+		sched_info_arrive(next);
+}
+#else
+#define sched_info_queued(t)		{}
+#define sched_info_switch(t, next)	{}
+#endif /* CONFIG_SCHEDSTATS */
+
 /*
  * Adding/removing a task to/from a priority array:
  */
@@ -323,6 +605,7 @@ static void dequeue_task(struct task_str
 
 static void enqueue_task(struct task_struct *p, prio_array_t *array)
 {
+	sched_info_queued(p);
 	list_add_tail(&p->run_list, array->queue + p->prio);
 	__set_bit(p->prio, array->bitmap);
 	array->nr_active++;
@@ -1114,6 +1397,13 @@ unsigned long nr_iowait(void)
 	return sum;
 }
 
+#ifdef CONFIG_SCHEDSTATS
+void cpu_sched_info(struct sched_info *info, int cpu)
+{
+	memcpy(info, &cpu_rq(cpu)->info, sizeof(struct sched_info));
+}
+#endif /* CONFIG_SCHEDSTATS */
+
 /*
  * double_rq_lock - safely lock two runqueues
  *
@@ -1173,6 +1463,7 @@ static void sched_migrate_task(task_t *p
 	if (!cpu_isset(dest_cpu, p->cpus_allowed))
 		goto out;
 
+	schedstat_inc(smp_processor_id(), mtc_cnt);
 	/* force the process onto the specified CPU */
 	if (migrate_task(p, dest_cpu, &req)) {
 		/* Need to wait for migration thread. */
@@ -1239,6 +1530,7 @@ void sched_balance_exec(void)
 	struct sched_domain *sd, *best_sd = NULL;
 	int new_cpu, this_cpu = get_cpu();
 
+ 	schedstat_inc(this_cpu, sbe_cnt);
 	/* Prefer the current CPU if there's only this task running: */
 	if (this_rq()->nr_running <= 1)
 		goto out;
@@ -1388,6 +1680,26 @@ skip_queue:
 		idx++;
 		goto skip_bitmap;
 	}
+#ifdef CONFIG_SCHEDSTATS
+	/*
+	 * Right now, this is the only place pull_task() is called,
+	 * so we can safely collect pull_task() stats here.
+	 */
+	switch (idle) {
+	    case NEWLY_IDLE:
+		schedstat_inc(this_cpu, pt_gained_newidle);
+		schedstat_inc(busiest->cpu, pt_lost_newidle);
+		break;
+	    case IDLE:
+		schedstat_inc(this_cpu, pt_gained_idle);
+		schedstat_inc(busiest->cpu, pt_lost_idle);
+		break;
+	    case NOT_IDLE:
+		schedstat_inc(this_cpu, pt_gained_notidle);
+		schedstat_inc(busiest->cpu, pt_lost_notidle);
+		break;
+	}
+#endif
 	pull_task(busiest, array, tmp, this_rq, dst_array, this_cpu);
 	pulled++;
 
@@ -1532,18 +1844,25 @@ static int load_balance(int this_cpu, ru
 	int nr_moved;
 
 	spin_lock(&this_rq->lock);
+	schedstat_inc(this_cpu, lb_cnt);
 
 	group = find_busiest_group(sd, this_cpu, &imbalance, idle);
-	if (!group)
+	if (!group) {
+		schedstat_inc(this_cpu, lb_nobusyg);
 		goto out_balanced;
+	}
 
 	busiest = find_busiest_queue(group);
-	if (!busiest)
+	if (!busiest) {
+		schedstat_inc(this_cpu, lb_nobusyq);
 		goto out_balanced;
+	}
+  
 	if (unlikely(busiest == this_rq)) {
 		WARN_ON(1);
 		goto out_balanced;
 	}
+	schedstat_add(this_cpu, lb_imbalance, imbalance);
 
 	/* Attempt to move tasks */
 	double_lock_balance(this_rq, busiest);
@@ -1605,6 +1924,7 @@ static int load_balance_newidle(int this
 	unsigned long imbalance;
 	int nr_moved = 0;
 
+ 	schedstat_inc(this_cpu, lbni_cnt);
 	group = find_busiest_group(sd, this_cpu, &imbalance, NEWLY_IDLE);
 	if (!group)
 		goto out;
@@ -1616,6 +1936,7 @@ static int load_balance_newidle(int this
 	/* Attempt to move tasks */
 	double_lock_balance(this_rq, busiest);
 
+	schedstat_inc(this_cpu, lbni_imbalance);
 	nr_moved = move_tasks(this_rq, this_cpu, busiest,
 					imbalance, sd, NEWLY_IDLE);
 
@@ -1660,6 +1981,7 @@ static void active_load_balance(runqueue
 	struct sched_domain *sd;
 	int i;
 
+	schedstat_inc(busiest_cpu, alb_cnt);
 	if (busiest->nr_running <= 1)
 		return;
 
@@ -1699,6 +2021,8 @@ static void active_load_balance(runqueue
 		rq = cpu_rq(push_cpu);
 		double_lock_balance(busiest, rq);
 		move_tasks(rq, push_cpu, busiest, 1, sd, IDLE);
+		schedstat_inc(busiest_cpu, alb_lost);
+		schedstat_inc(push_cpu, alb_gained);
 		spin_unlock(&rq->lock);
 next_group:
 		group = group->next;
@@ -1738,6 +2062,13 @@ static void rebalance_tick(int this_cpu,
 			interval = 1;
 
 		if (j - sd->last_balance >= interval) {
+#ifdef CONFIG_SCHEDSTATS
+ 			if (idle == IDLE) {
+ 				schedstat_inc(this_cpu, lb_idle);
+ 			} else {
+ 				schedstat_inc(this_cpu, lb_busy);
+ 			}
+#endif /* CONFIG_SCHEDSTATS */
 			if (load_balance(this_cpu, this_rq, sd, idle)) {
 				/* We've pulled tasks over so no longer idle */
 				idle = NOT_IDLE;
@@ -2021,6 +2352,7 @@ asmlinkage void schedule(void)
 	 * schedule() atomically, we ignore that path for now.
 	 * Otherwise, whine if we are scheduling when we should not be.
 	 */
+ 	schedstat_inc(smp_processor_id(), sched_cnt);
 	if (likely(!(current->state & (TASK_DEAD | TASK_ZOMBIE)))) {
 		if (unlikely(in_atomic())) {
 			printk(KERN_ERR "bad: scheduling while atomic!\n");
@@ -2080,12 +2412,14 @@ need_resched:
 		/*
 		 * Switch the active and expired arrays.
 		 */
+		schedstat_inc(cpu, sched_switch);
 		rq->active = rq->expired;
 		rq->expired = array;
 		array = rq->active;
 		rq->expired_timestamp = 0;
 		rq->best_expired_prio = MAX_PRIO;
 	}
+	schedstat_inc(cpu, sched_noswitch);
 
 	idx = sched_find_first_bit(array->bitmap);
 	queue = array->queue + idx;
@@ -2121,6 +2455,7 @@ switch_tasks:
 	}
 	prev->timestamp = now;
 
+	sched_info_switch(prev, next);
 	if (likely(prev != next)) {
 		next->timestamp = now;
 		rq->nr_switches++;
@@ -2823,7 +3158,11 @@ asmlinkage long sys_sched_yield(void)
 	runqueue_t *rq = this_rq_lock();
 	prio_array_t *array = current->array;
 	prio_array_t *target = rq->expired;
+#ifdef CONFIG_SCHEDSTATS
+	int this_cpu = smp_processor_id();
+#endif /* CONFIG_SCHEDSTATS */
 
+	schedstat_inc(this_cpu, yld_cnt);
 	/*
 	 * We implement yielding by moving the task into the expired
 	 * queue.
@@ -3654,6 +3993,9 @@ void __init sched_init(void)
 		spin_lock_init(&rq->lock);
 		rq->active = rq->arrays;
 		rq->expired = rq->arrays + 1;
+#ifdef CONFIG_SCHEDSTATS
+		rq->cpu = i;
+#endif /* CONFIG_SCHEDSTATS */
 		rq->best_expired_prio = MAX_PRIO;
 
 #ifdef CONFIG_SMP