linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
@ 2002-07-01  0:26 Ingo Molnar
  2002-07-01  2:55 ` Nicholas Miell
  2002-07-04 13:04 ` Vitez Gabor
  0 siblings, 2 replies; 9+ messages in thread
From: Ingo Molnar @ 2002-07-01  0:26 UTC (permalink / raw)
  To: linux-kernel; +Cc: Linus Torvalds

[-- Attachment #1: Type: TEXT/PLAIN, Size: 6472 bytes --]


the attached patch adds a feature that was pretty high on the scheduler
features wishlist: it implements the functionality of SCHED_IDLE, in a
safe way. Another desired scheduler feature was batch scheduling, the
cache-friendly handling of lowprio, batch-like, CPU-bound, 100%
noninteractive tasks. The new SCHED_BATCH scheduler policy implements both
features.

the existing SCHED_IDLE patches floating around, despite their simplicity,
had one major flaw that prevented their integration into the scheduler: if
an unpriviledged SCHED_IDLE process uses normal kernel functionality,
which happens to grab a critical kernel resource such as the root
directory's semaphore, and schedules away still holding the semaphore,
then there is no guarantee that the task will run again in any
deterministic amount of time - keeping the critical resource potentially
forever - deadlocking every other process that attempts to use that
critical resource. This property, while being a source for soft lockups
even during ordinary use, also makes SCHED_IDLE an easy DoS exploit.

as the size of the patch suggests, the safe solution is not simple. The
basic concept is the identification of user-space preemption via a special
scheduler upcall: one safe point to delay a task's execution indefinitely
is when the task is preempted in pure user-space mode - if this happens
then the lowlevel kernel entry code calls the schedule_userspace()  
function, instead of schedule(). In every other case the task needs to
stay in the 'normal' scheduler queues, to guarantee prompt processing of
kernelspace code. Furthermore, such batch-mode tasks need to be scheduled
if they get a signal delivered - otherwise it would not be possible to eg.
kill them.

other properties: SCHED_BATCH also triggers much longer, batch-like
timeslices - the default SCHED_BATCH timeslice is 1.5 seconds. Nice values
still have a meaning for SCHED_BATCH processes as well - they determine
the relative percentage of idle CPU time allocated to SCHED_BATCH
processes. If the SCHED_BATCH process is in kernel-mode then the nice
value is used as the normal priority when preempting (or not preempting)  
other, non-SCHED_BATCH processes.

put in another way: whenever a SCHED_BATCH process is in kernel-mode, it's
"elevated" into the SCHED_NORMAL priority domain - which guarantees timely
execution of kernel-space code. When the SCHED_BATCH process is executing
user-space code then it can be put into the batch-queue, and can be
delayed indefinitely.

Timeslice distribution is a modified/simplified version of SCHED_NORMAL
scheduling: SCHED_BATCH processes are scheduled in a roundrobin way,
timeslices are distributed based on the nice value. SCHED_BATCH tasks that
use up their timeslices get suspended until all other SCHED_BATCH tasks on
that CPU exhaust their timeslices - at which point a new turn begins.  
SCHED_NORMAL, SCHED_RR and SCHED_FIFO tasks preempt SCHED_BATCH processes
immediately. All this functionality is implemented in an O(1) way. (The
interactivity estimator is active for SCHED_BATCH processes as well - this
has an effect if the task is in kernelspace mode. This also makes sure
that no artificial priority boost can be achieved by switching in/out of
SCHED_BATCH mode.)

on SMP there are per-CPU batch queues - which enables the use of hundreds
or thousands of SCHED_BATCH processes, if desired. A new, independent
load-balancer is used to distribute SCHED_BATCH processes: SCHED_BATCH
processes will populate CPUs depending on the CPU's "10 seconds history of
idleness". The more idle a CPU, the more SCHED_BATCH processes it will
handle. The weighting is done in a way to make the global distribution of
SCHED_BATCH timeslices fair. The load-balancer also honors caching
properties and tries to reduce unnecessery bouncing of SCHED_BATCH
processes. (The balancing, like in the SCHED_NORMAL case, is not intended
to be 100% 'sharp' - some statistical fuzziness is allowed to keep
overhead and complexity down.)

(to see the SMP SCHED_BATCH load-balancer in action, start up multiple
SCHED_BATCH processes on an SMP box - they populate all available CPUs
evenly. Then start up a single CPU-intensive, non-SCHED_BATCH process -
after a few seconds all SCHED_BATCH processes will migrate off to the
remaining CPUs, and the SCHED_NORMAL task will get 100% CPU time of a
single CPU.)

(design sidenote: initially i tried to integrate SCHED_BATCH scheduling
into the existing scheduler and SCHED_NORMAL balancer somehow, but gave up
on this idea. While that worked for RT scheduling, SCHED_BATCH scheduling
is quite different, and is 100% orthogonal to all the other scheduling
domains. Eg. the balancing of non-SCHED_BATCH processes *must not* be
influenced by the way SCHED_BATCH processes are distributed amongst CPUs.
The distribution of timeslices must be completely separated as well. So
since all the queues and state has to be separate, they can as well be in
separate (and simplified) data structures.)

i've also attached setbatch.c, which is a simple utility to change a given
PID's scheduling policy to SCHED_BATCH. One straightforward way of using
it is to change one shell to be SCHED_BATCH:

	./setbatch $$

and start various commands from this SCHED_BATCH shell - all forked
children inherit the SCHED_BATCH setting.

the load generated by multiple SCHED_BATCH processes does not show up in
the load average - this is the straightforward solution to not confuse
load-average-sensitive applications such as sendmail.

the runtime performance impact of SCHED_BATCH is fairly minimal. There is
a (pretty light) branch and function call cost in the entry.S preemption
codepath. Otherwise the SCHED_BATCH code triggers in slowpaths only: eg.  
when we would otherwise switch to the idle thread.

the patch was tested on x86 systems. non-x86 systems should still work
with the patch applied, but no SCHED_BATCH process will in fact be
suspended. For batch-suspension to work the architecture needs to call
schedule_userspace() instead of schedule(), when pure userspace code is
preempted.

the attached patch is against 2.5.24, it was tested on SMP and UP systems
as well, but keep in mind that this is the first version of this patch, so
some rough edges might be present. The patch can also be downloaded from
my scheduler patches homepage:

	http://redhat.com/~mingo/O(1)-scheduler/batch-sched-2.5.24-A0

bug reports, success reports, comments, suggestions are welcome,

	Ingo

[-- Attachment #2: Type: TEXT/PLAIN, Size: 23086 bytes --]

--- linux/arch/i386/kernel/entry.S.orig	Fri Jun 28 18:43:49 2002
+++ linux/arch/i386/kernel/entry.S	Mon Jul  1 01:22:32 2002
@@ -215,10 +215,22 @@
 	movl TI_FLAGS(%ebx), %ecx	# need_resched set ?
 	testb $_TIF_NEED_RESCHED, %cl
 	jz restore_all
-	testl $IF_MASK,EFLAGS(%esp)     # interrupts off (execption path) ?
+	testl $IF_MASK,EFLAGS(%esp)     # interrupts off (exception path) ?
 	jz restore_all
 	movl $PREEMPT_ACTIVE,TI_PRE_COUNT(%ebx)
 	sti
+
+	movl EFLAGS(%esp), %eax		# mix EFLAGS and CS
+	movb CS(%esp), %al
+	testl $(VM_MASK | 3), %eax
+
+	jz sched_kernelspace		# returning to kernel or vm86-space
+	call schedule_userspace
+	movl $0,TI_PRE_COUNT(%ebx) 
+	cli
+	jmp need_resched
+
+sched_kernelspace:
 	call schedule
 	movl $0,TI_PRE_COUNT(%ebx) 
 	cli
@@ -255,7 +267,19 @@
 	testb $_TIF_NEED_RESCHED, %cl
 	jz work_notifysig
 work_resched:
+	movl EFLAGS(%esp), %eax		# mix EFLAGS and CS
+	movb CS(%esp), %al
+	testl $(VM_MASK | 3), %eax
+
+	jz sched_kernelspace2		# returning to kernel or vm86-space
+
+	call schedule_userspace
+	jmp continue_resched
+
+sched_kernelspace2:
 	call schedule
+
+continue_resched:
 	cli				# make sure we don't miss an interrupt
 					# setting need_resched or sigpending
 					# between sampling and the iret
@@ -268,7 +292,7 @@
 
 work_notifysig:				# deal with pending signals and
 					# notify-resume requests
-	testl $(VM_MASK),EFLAGS(%esp)
+	testl $VM_MASK, EFLAGS(%esp)
 	movl %esp, %eax
 	jne work_notifysig_v86		# returning to kernel-space or
 					# vm86-space
--- linux/include/linux/sched.h.orig	Fri Jun 28 18:42:28 2002
+++ linux/include/linux/sched.h	Mon Jul  1 01:22:32 2002
@@ -117,9 +117,10 @@
 /*
  * Scheduling policies
  */
-#define SCHED_OTHER		0
+#define SCHED_NORMAL		0
 #define SCHED_FIFO		1
 #define SCHED_RR		2
+#define SCHED_BATCH		3
 
 struct sched_param {
 	int sched_priority;
@@ -158,6 +159,7 @@
 #define	MAX_SCHEDULE_TIMEOUT	LONG_MAX
 extern signed long FASTCALL(schedule_timeout(signed long timeout));
 asmlinkage void schedule(void);
+asmlinkage void schedule_userspace(void);
 
 extern void flush_scheduled_tasks(void);
 extern int start_context_thread(void);
@@ -208,7 +210,7 @@
 
 /*
  * Priority of a process goes from 0..MAX_PRIO-1, valid RT
- * priority is 0..MAX_RT_PRIO-1, and SCHED_OTHER tasks are
+ * priority is 0..MAX_RT_PRIO-1, and SCHED_NORMAL tasks are
  * in the range MAX_RT_PRIO..MAX_PRIO-1. Priority values
  * are inverted: lower p->prio value means higher priority.
  *
@@ -391,6 +393,7 @@
 #define PF_FREEZE	0x00010000	/* this task should be frozen for suspend */
 #define PF_IOTHREAD	0x00020000	/* this thread is needed for doing I/O to swap */
 #define PF_FROZEN	0x00040000	/* frozen for system suspend */
+#define PF_BATCH	0x00080000	/* batch-priority process */
 
 /*
  * Ptrace flags
--- linux/include/linux/init_task.h.orig	Sun Jun 30 14:54:40 2002
+++ linux/include/linux/init_task.h	Mon Jul  1 01:22:32 2002
@@ -47,7 +47,7 @@
     lock_depth:		-1,						\
     prio:		MAX_PRIO-20,					\
     static_prio:	MAX_PRIO-20,					\
-    policy:		SCHED_OTHER,					\
+    policy:		SCHED_NORMAL,					\
     cpus_allowed:	-1,						\
     mm:			NULL,						\
     active_mm:		&init_mm,					\
--- linux/kernel/timer.c.orig	Fri Jun 28 18:42:28 2002
+++ linux/kernel/timer.c	Mon Jul  1 01:22:32 2002
@@ -888,7 +888,7 @@
 
 
 	if (t.tv_sec == 0 && t.tv_nsec <= 2000000L &&
-	    current->policy != SCHED_OTHER)
+	    current->policy != SCHED_NORMAL && current->policy != SCHED_BATCH)
 	{
 		/*
 		 * Short delay requests up to 2 ms will be handled with
--- linux/kernel/sched.c.orig	Fri Jun 28 18:42:28 2002
+++ linux/kernel/sched.c	Mon Jul  1 01:22:32 2002
@@ -101,17 +101,31 @@
 	((p)->prio <= (p)->static_prio - DELTA(p))
 
 /*
- * TASK_TIMESLICE scales user-nice values [ -20 ... 19 ]
+ * BASE_TIMESLICE scales user-nice values [ -20 ... 19 ]
  * to time slice values.
  *
  * The higher a process's priority, the bigger timeslices
  * it gets during one round of execution. But even the lowest
  * priority process gets MIN_TIMESLICE worth of execution time.
+ *
+ * task_timeslice() is the interface that is used by the scheduler.
+ * SCHED_BATCH tasks get longer timeslices to make use of better
+ * caching. They are inherently noninteractive and they are
+ * immediately preempted by SCHED_NORMAL tasks so there is no
+ * downside in using shorter timeslices.
  */
 
-#define TASK_TIMESLICE(p) (MIN_TIMESLICE + \
+#define BASE_TIMESLICE(p) (MIN_TIMESLICE + \
 	((MAX_TIMESLICE - MIN_TIMESLICE) * (MAX_PRIO-1-(p)->static_prio)/39))
 
+static inline unsigned int task_timeslice(task_t *p)
+{
+	if (p->policy == SCHED_BATCH)
+		return BASE_TIMESLICE(p) * 10;
+	else
+		return BASE_TIMESLICE(p);
+}
+
 /*
  * These are the runqueue data structures:
  */
@@ -135,13 +149,31 @@
  */
 struct runqueue {
 	spinlock_t lock;
-	unsigned long nr_running, nr_switches, expired_timestamp;
-	signed long nr_uninterruptible;
+	unsigned long nr_running, nr_switches, expired_timestamp,
+			nr_uninterruptible;
 	task_t *curr, *idle;
 	prio_array_t *active, *expired, arrays[2];
 	int prev_nr_running[NR_CPUS];
+
 	task_t *migration_thread;
 	list_t migration_queue;
+
+	/*
+	 * The batch queue is a secondary ready-queue:
+	 */
+	unsigned long nr_batch;
+	list_t *batch_active, *batch_expired, batch_queues[2];
+
+	/*
+	 * Per-CPU idle CPU time tracking:
+	 */
+	#define IDLE_SLOTS 10
+	#define IDLE_TICKS (HZ)
+
+	int idle_ticks_left;
+	int idle_count[IDLE_SLOTS];
+	int idle_avg;
+
 } ____cacheline_aligned;
 
 static struct runqueue runqueues[NR_CPUS] __cacheline_aligned;
@@ -264,6 +296,14 @@
 	rq->nr_running++;
 }
 
+static inline void activate_batch_task(task_t *p, runqueue_t *rq)
+{
+	rq->nr_batch--;
+	list_del(&p->run_list);
+	activate_task(p, rq);
+	p->flags &= ~PF_BATCH;
+}
+
 static inline void deactivate_task(struct task_struct *p, runqueue_t *rq)
 {
 	rq->nr_running--;
@@ -273,6 +313,23 @@
 	p->array = NULL;
 }
 
+static inline void deactivate_batch_task(task_t *p, runqueue_t *rq)
+{
+	prio_array_t *array = p->array;
+
+	deactivate_task(p, rq);
+	rq->nr_batch++;
+	if (array == rq->expired)
+		list_add_tail(&p->run_list, rq->batch_expired);
+	else
+		list_add(&p->run_list, rq->batch_active);
+	/*
+	 * Via this bit we can tell whether a task is in the batchqueue,
+	 * this information is not available in any other cheap way.
+	 */
+	p->flags |= PF_BATCH;
+}
+
 static inline void resched_task(task_t *p)
 {
 #ifdef CONFIG_SMP
@@ -325,6 +382,7 @@
 	task_rq_unlock(rq, &flags);
 	preempt_enable();
 }
+#endif
 
 /*
  * Kick the remote CPU if the task is running currently,
@@ -334,13 +392,29 @@
  * (Note that we do this lockless - if the task does anything
  * while the message is in flight then it will notice the
  * sigpending condition anyway.)
+ *
+ * this code also activates batch processes if they get a signal.
  */
 void kick_if_running(task_t * p)
 {
-	if (p == task_rq(p)->curr)
+	if ((p == task_rq(p)->curr) && (p->thread_info->cpu != smp_processor_id()))
 		resched_task(p);
+	/*
+	 * If batch processes get signals but are not running currently
+	 * then give them a chance to handle the signal. (the kernel
+	 * side signal handling code will run for sure, the userspace
+	 * part depends on system load and might be delayed indefinitely.)
+	 */
+	if (p->policy == SCHED_BATCH) {
+		unsigned long flags;
+		runqueue_t *rq;
+
+		rq = task_rq_lock(p, &flags);
+		if (p->flags & PF_BATCH)
+			activate_batch_task(p, rq);
+		task_rq_unlock(rq, &flags);
+	}
 }
-#endif
 
 /*
  * Wake up a process. Put it on the run-queue if it's not
@@ -376,10 +450,8 @@
 		if (old_state == TASK_UNINTERRUPTIBLE)
 			rq->nr_uninterruptible--;
 		activate_task(p, rq);
-		/*
-		 * If sync is set, a resched_task() is a NOOP
-		 */
-		if (p->prio < rq->curr->prio)
+
+		if (p->prio < rq->curr->prio || rq->curr->policy == SCHED_BATCH)
 			resched_task(rq->curr);
 		success = 1;
 	}
@@ -501,7 +573,128 @@
 	return sum;
 }
 
+/*
+ * double_rq_lock - safely lock two runqueues
+ *
+ * Note this does not disable interrupts like task_rq_lock,
+ * you need to do so manually before calling.
+ */
+static inline void double_rq_lock(runqueue_t *rq1, runqueue_t *rq2)
+{
+	if (rq1 == rq2)
+		spin_lock(&rq1->lock);
+	else {
+		if (rq1 < rq2) {
+			spin_lock(&rq1->lock);
+			spin_lock(&rq2->lock);
+		} else {
+			spin_lock(&rq2->lock);
+			spin_lock(&rq1->lock);
+		}
+	}
+}
+
+/*
+ * double_rq_unlock - safely unlock two runqueues
+ *
+ * Note this does not restore interrupts like task_rq_unlock,
+ * you need to do so manually after calling.
+ */
+static inline void double_rq_unlock(runqueue_t *rq1, runqueue_t *rq2)
+{
+	spin_unlock(&rq1->lock);
+	if (rq1 != rq2)
+		spin_unlock(&rq2->lock);
+}
+
 #if CONFIG_SMP
+
+/*
+ * Batch balancing is much simpler since it's optimized for
+ * CPU-intensive workloads. The balancer keeps the batch-queue
+ * length as close to the average length as possible. It weighs
+ * runqueue distribution based on the idle percentage of each
+ * CPU - this way statistical fairness of timeslice distribution
+ * is preserved, in the long run it does not matter whether a
+ * batch task is queued to a busy CPU or not, it will get an
+ * equal share of all available idle CPU time.
+ *
+ * CPU-intensive SCHED_BATCH processes have a much lower
+ * fork()/exit() flux, so the balancing does not have to
+ * be prepared for high statistical fluctuations in queue
+ * length.
+ */
+static inline void load_balance_batch(runqueue_t *this_rq, int this_cpu)
+{
+	int i, nr_batch, nr_idle, goal, rq_goal;
+	runqueue_t *rq_src;
+
+	/*
+	 * First the unlocked fastpath - is there any work to do?
+	 * fastpath #1: no batch processes in the system,
+	 * fastpath #2: no idle time available in the system.
+	 * fastpath #3: no balancing needed for the current queue.
+	 */
+	nr_batch = 0;
+	nr_idle = 0;
+
+	for (i = 0; i < NR_CPUS; i++) {
+		if (!cpu_online(i))
+			continue;
+
+		nr_batch += cpu_rq(i)->nr_batch;
+		nr_idle += cpu_rq(i)->idle_avg;
+	}
+	if (!nr_batch || !nr_idle)
+		return;
+
+	goal = this_rq->idle_avg * nr_batch / nr_idle;
+	if (this_rq->nr_batch >= goal)
+		return;
+
+	/*
+	 * The slow path - the local batch-queue is too short and
+	 * needs balancing. We unlock the runqueue (but keep
+	 * interrupts disabled) to simplify locking. (It does not
+	 * matter if the runqueues change meanwhile - this is all
+	 * statistical balancing so only the long run effects matter.)
+	 */
+	spin_unlock(&this_rq->lock);
+
+	for (i = 0; i < NR_CPUS; i++) {
+		if (!cpu_online(i) || (i == this_cpu))
+			continue;
+
+		rq_src = cpu_rq(i);
+		double_rq_lock(this_rq, rq_src);
+
+		rq_goal = rq_src->idle_avg * nr_batch / nr_idle;
+
+		if (rq_src->nr_batch > rq_goal) {
+			/*
+			 * Migrate a single batch-process. We take the one
+			 * from the head of the expired queue - it's the
+			 * most cache-cold so it will cause the least
+			 * cache-bouncing on SMP. If the expired queue is
+			 * empty then we take the tail of the active queue.
+			 */
+			list_t *tmp = rq_src->batch_expired->next;
+
+			if (list_empty(tmp))
+				tmp = rq_src->batch_active->prev;
+			list_del(tmp);
+			list_add_tail(tmp, this_rq->batch_active);
+			rq_src->nr_batch--;
+			this_rq->nr_batch++;
+			set_task_cpu(list_entry(tmp, task_t, run_list), this_cpu);
+		}
+
+		double_rq_unlock(this_rq, rq_src);
+		if (this_rq->nr_batch >= goal)
+			break;
+	}
+	spin_lock(&this_rq->lock);
+}
 /*
  * Lock the busiest runqueue as well, this_rq is locked already.
  * Recalculate nr_running if we have to drop the runqueue lock.
@@ -535,8 +728,8 @@
  */
 static void load_balance(runqueue_t *this_rq, int idle)
 {
-	int imbalance, nr_running, load, max_load,
-		idx, i, this_cpu = smp_processor_id();
+	int imbalance, nr_running, load, max_load, idx, i,
+		this_cpu = smp_processor_id();
 	task_t *next = this_rq->idle, *tmp;
 	runqueue_t *busiest, *rq_src;
 	prio_array_t *array;
@@ -589,13 +782,13 @@
 	}
 
 	if (likely(!busiest))
-		return;
+		goto balance_batch;
 
 	imbalance = (max_load - nr_running) / 2;
 
 	/* It needs an at least ~25% imbalance to trigger balancing. */
 	if (!idle && (imbalance < (max_load + 3)/4))
-		return;
+		goto balance_batch;
 
 	nr_running = double_lock_balance(this_rq, busiest, this_cpu, idle, nr_running);
 	/*
@@ -676,6 +869,8 @@
 	}
 out_unlock:
 	spin_unlock(&busiest->lock);
+balance_batch:
+	load_balance_batch(this_rq, this_cpu);
 }
 
 /*
@@ -690,13 +885,24 @@
 #define BUSY_REBALANCE_TICK (HZ/4 ?: 1)
 #define IDLE_REBALANCE_TICK (HZ/1000 ?: 1)
 
-static inline void idle_tick(void)
+static inline int recalc_idle_avg(runqueue_t *rq)
+{
+	int i, count = 0, avg;
+
+	for (i = 1; i < IDLE_SLOTS; i++)
+		count += rq->idle_count[i];
+
+	avg = count / (IDLE_SLOTS - 1);
+	return avg;
+}
+
+static inline void idle_tick(runqueue_t *rq)
 {
 	if (jiffies % IDLE_REBALANCE_TICK)
 		return;
-	spin_lock(&this_rq()->lock);
-	load_balance(this_rq(), 1);
-	spin_unlock(&this_rq()->lock);
+	spin_lock(&rq->lock);
+	load_balance(rq, 1);
+	spin_unlock(&rq->lock);
 }
 
 #endif
@@ -719,25 +925,43 @@
  * This function gets called by the timer code, with HZ frequency.
  * We call it with interrupts disabled.
  */
-void scheduler_tick(int user_tick, int system)
+void scheduler_tick(int user_ticks, int sys_ticks)
 {
 	int cpu = smp_processor_id();
 	runqueue_t *rq = this_rq();
 	task_t *p = current;
 
+#if CONFIG_SMP
+	if (user_ticks || sys_ticks) {
+		/*
+		 * This code is rare, triggered only once per second:
+		 */
+		if (--rq->idle_ticks_left <= 0) {
+			int i;
+
+			rq->idle_ticks_left = IDLE_TICKS;
+			for (i = IDLE_SLOTS-1; i > 0; i--)
+				rq->idle_count[i] = rq->idle_count[i-1];
+			rq->idle_count[0] = 0;
+			rq->idle_avg = recalc_idle_avg(rq);
+		}
+	}
+	if (p == rq->idle || p->policy == SCHED_BATCH)
+		rq->idle_count[0]++;
+#endif
 	if (p == rq->idle) {
 		if (local_bh_count(cpu) || local_irq_count(cpu) > 1)
-			kstat.per_cpu_system[cpu] += system;
+			kstat.per_cpu_system[cpu] += sys_ticks;
 #if CONFIG_SMP
-		idle_tick();
+		idle_tick(rq);
 #endif
 		return;
 	}
 	if (TASK_NICE(p) > 0)
-		kstat.per_cpu_nice[cpu] += user_tick;
+		kstat.per_cpu_nice[cpu] += user_ticks;
 	else
-		kstat.per_cpu_user[cpu] += user_tick;
-	kstat.per_cpu_system[cpu] += system;
+		kstat.per_cpu_user[cpu] += user_ticks;
+	kstat.per_cpu_system[cpu] += sys_ticks;
 
 	/* Task might have expired already, but not scheduled off yet */
 	if (p->array != rq->active) {
@@ -751,7 +975,7 @@
 		 * FIFO tasks have no timeslices.
 		 */
 		if ((p->policy == SCHED_RR) && !--p->time_slice) {
-			p->time_slice = TASK_TIMESLICE(p);
+			p->time_slice = task_timeslice(p);
 			set_tsk_need_resched(p);
 
 			/* put it at the end of the queue: */
@@ -774,7 +998,7 @@
 		dequeue_task(p, rq->active);
 		set_tsk_need_resched(p);
 		p->prio = effective_prio(p);
-		p->time_slice = TASK_TIMESLICE(p);
+		p->time_slice = task_timeslice(p);
 
 		if (!TASK_INTERACTIVE(p) || EXPIRED_STARVING(rq)) {
 			if (!rq->expired_timestamp)
@@ -794,6 +1018,35 @@
 void scheduling_functions_start_here(void) { }
 
 /*
+ * This function is called by the lowlevel kernel entry code if
+ * pure userspace code is preempted. Such processes, if SCHED_BATCH,
+ * are candidates for batch scheduling. Every other process (including
+ * kernel-mode SCHED_BATCH processes) is scheduled in a non-batch way.
+ */
+asmlinkage void schedule_userspace(void)
+{
+	runqueue_t *rq;
+
+	if (current->policy != SCHED_BATCH) {
+		schedule();
+		return;
+	}
+
+	/*
+	 * Only handle batch tasks that are runnable.
+	 */
+	if (current->state == TASK_RUNNING) {
+		rq = this_rq_lock();
+		deactivate_batch_task(current, rq);
+
+		// we can keep irqs disabled:
+		spin_unlock(&rq->lock);
+	}
+
+	schedule();
+}
+
+/*
  * 'schedule()' is the main scheduler function.
  */
 asmlinkage void schedule(void)
@@ -845,7 +1098,29 @@
 		if (rq->nr_running)
 			goto pick_next_task;
 #endif
-		next = rq->idle;
+		/*
+		 * Pick a task from the batch queue if available.
+		 */
+		if (rq->nr_batch) {
+			list_t *tmp = rq->batch_active->next;
+
+			/*
+			 * Switch the active and expired queues if
+			 * the active queue is empty.
+			 */
+			if (list_empty(tmp)) {
+				list_t *tmp2;
+
+				tmp2 = rq->batch_active;
+				rq->batch_active = rq->batch_expired;
+				rq->batch_expired = tmp2;
+				tmp = rq->batch_active->next;
+			}
+
+			next = list_entry(tmp, task_t, run_list);
+			activate_batch_task(next, rq);
+		} else
+			next = rq->idle;
 		rq->expired_timestamp = 0;
 		goto switch_tasks;
 	}
@@ -1206,18 +1481,20 @@
 	else {
 		retval = -EINVAL;
 		if (policy != SCHED_FIFO && policy != SCHED_RR &&
-				policy != SCHED_OTHER)
+				policy != SCHED_NORMAL && policy != SCHED_BATCH)
 			goto out_unlock;
 	}
 
 	/*
 	 * Valid priorities for SCHED_FIFO and SCHED_RR are
-	 * 1..MAX_USER_RT_PRIO-1, valid priority for SCHED_OTHER is 0.
+	 * 1..MAX_USER_RT_PRIO-1, valid priority for SCHED_NORMAL and
+	 * SCHED_BATCH is 0.
 	 */
 	retval = -EINVAL;
 	if (lp.sched_priority < 0 || lp.sched_priority > MAX_USER_RT_PRIO-1)
 		goto out_unlock;
-	if ((policy == SCHED_OTHER) != (lp.sched_priority == 0))
+	if ((policy == SCHED_NORMAL || policy == SCHED_BATCH) !=
+						(lp.sched_priority == 0))
 		goto out_unlock;
 
 	retval = -EPERM;
@@ -1228,13 +1505,15 @@
 	    !capable(CAP_SYS_NICE))
 		goto out_unlock;
 
+	if (p->flags & PF_BATCH)
+		activate_batch_task(p, rq);
 	array = p->array;
 	if (array)
 		deactivate_task(p, task_rq(p));
 	retval = 0;
 	p->policy = policy;
 	p->rt_priority = lp.sched_priority;
-	if (policy != SCHED_OTHER)
+	if (policy != SCHED_NORMAL && policy != SCHED_BATCH)
 		p->prio = MAX_USER_RT_PRIO-1 - p->rt_priority;
 	else
 		p->prio = p->static_prio;
@@ -1456,7 +1735,8 @@
 	case SCHED_RR:
 		ret = MAX_USER_RT_PRIO-1;
 		break;
-	case SCHED_OTHER:
+	case SCHED_NORMAL:
+	case SCHED_BATCH:
 		ret = 0;
 		break;
 	}
@@ -1472,7 +1752,8 @@
 	case SCHED_RR:
 		ret = 1;
 		break;
-	case SCHED_OTHER:
+	case SCHED_NORMAL:
+	case SCHED_BATCH:
 		ret = 0;
 	}
 	return ret;
@@ -1492,7 +1773,7 @@
 	p = find_process_by_pid(pid);
 	if (p)
 		jiffies_to_timespec(p->policy & SCHED_FIFO ?
-					 0 : TASK_TIMESLICE(p), &t);
+					 0 : task_timeslice(p), &t);
 	read_unlock(&tasklist_lock);
 	if (p)
 		retval = copy_to_user(interval, &t, sizeof(t)) ? -EFAULT : 0;
@@ -1594,40 +1875,6 @@
 	read_unlock(&tasklist_lock);
 }
 
-/*
- * double_rq_lock - safely lock two runqueues
- *
- * Note this does not disable interrupts like task_rq_lock,
- * you need to do so manually before calling.
- */
-static inline void double_rq_lock(runqueue_t *rq1, runqueue_t *rq2)
-{
-	if (rq1 == rq2)
-		spin_lock(&rq1->lock);
-	else {
-		if (rq1 < rq2) {
-			spin_lock(&rq1->lock);
-			spin_lock(&rq2->lock);
-		} else {
-			spin_lock(&rq2->lock);
-			spin_lock(&rq1->lock);
-		}
-	}
-}
-
-/*
- * double_rq_unlock - safely unlock two runqueues
- *
- * Note this does not restore interrupts like task_rq_unlock,
- * you need to do so manually after calling.
- */
-static inline void double_rq_unlock(runqueue_t *rq1, runqueue_t *rq2)
-{
-	spin_unlock(&rq1->lock);
-	if (rq1 != rq2)
-		spin_unlock(&rq2->lock);
-}
-
 void __init init_idle(task_t *idle, int cpu)
 {
 	runqueue_t *idle_rq = cpu_rq(cpu), *rq = cpu_rq(task_cpu(idle));
@@ -1651,56 +1898,6 @@
 	idle->thread_info->preempt_count = (idle->lock_depth >= 0);
 }
 
-extern void init_timervecs(void);
-extern void timer_bh(void);
-extern void tqueue_bh(void);
-extern void immediate_bh(void);
-
-void __init sched_init(void)
-{
-	runqueue_t *rq;
-	int i, j, k;
-
-	for (i = 0; i < NR_CPUS; i++) {
-		prio_array_t *array;
-
-		rq = cpu_rq(i);
-		rq->active = rq->arrays;
-		rq->expired = rq->arrays + 1;
-		spin_lock_init(&rq->lock);
-		INIT_LIST_HEAD(&rq->migration_queue);
-
-		for (j = 0; j < 2; j++) {
-			array = rq->arrays + j;
-			for (k = 0; k < MAX_PRIO; k++) {
-				INIT_LIST_HEAD(array->queue + k);
-				__clear_bit(k, array->bitmap);
-			}
-			// delimiter for bitsearch
-			__set_bit(MAX_PRIO, array->bitmap);
-		}
-	}
-	/*
-	 * We have to do a little magic to get the first
-	 * process right in SMP mode.
-	 */
-	rq = this_rq();
-	rq->curr = current;
-	rq->idle = current;
-	wake_up_process(current);
-
-	init_timervecs();
-	init_bh(TIMER_BH, timer_bh);
-	init_bh(TQUEUE_BH, tqueue_bh);
-	init_bh(IMMEDIATE_BH, immediate_bh);
-
-	/*
-	 * The boot idle thread does lazy MMU switching as well:
-	 */
-	atomic_inc(&init_mm.mm_count);
-	enter_lazy_tlb(&init_mm, current, smp_processor_id());
-}
-
 #if CONFIG_SMP
 
 /*
@@ -1877,3 +2074,59 @@
 	}
 }
 #endif
+
+extern void init_timervecs(void);
+extern void timer_bh(void);
+extern void tqueue_bh(void);
+extern void immediate_bh(void);
+
+void __init sched_init(void)
+{
+	runqueue_t *rq;
+	int i, j, k;
+
+	for (i = 0; i < NR_CPUS; i++) {
+		prio_array_t *array;
+
+		rq = cpu_rq(i);
+		rq->active = rq->arrays;
+		rq->expired = rq->arrays + 1;
+		spin_lock_init(&rq->lock);
+		INIT_LIST_HEAD(&rq->migration_queue);
+		INIT_LIST_HEAD(&rq->batch_queues[0]);
+		INIT_LIST_HEAD(&rq->batch_queues[1]);
+		rq->batch_active = rq->batch_queues + 0;
+		rq->batch_expired = rq->batch_queues + 1;
+		rq->idle_ticks_left = IDLE_TICKS;
+
+		for (j = 0; j < 2; j++) {
+			array = rq->arrays + j;
+			for (k = 0; k < MAX_PRIO; k++) {
+				INIT_LIST_HEAD(array->queue + k);
+				__clear_bit(k, array->bitmap);
+			}
+			// delimiter for bitsearch
+			__set_bit(MAX_PRIO, array->bitmap);
+		}
+	}
+	/*
+	 * We have to do a little magic to get the first
+	 * process right in SMP mode.
+	 */
+	rq = this_rq();
+	rq->curr = current;
+	rq->idle = current;
+	wake_up_process(current);
+
+	init_timervecs();
+	init_bh(TIMER_BH, timer_bh);
+	init_bh(TQUEUE_BH, tqueue_bh);
+	init_bh(IMMEDIATE_BH, immediate_bh);
+
+	/*
+	 * The boot idle thread does lazy MMU switching as well:
+	 */
+	atomic_inc(&init_mm.mm_count);
+	enter_lazy_tlb(&init_mm, current, smp_processor_id());
+}
+
--- linux/kernel/exit.c.orig	Fri Jun 28 19:35:06 2002
+++ linux/kernel/exit.c	Mon Jul  1 01:22:32 2002
@@ -182,7 +182,7 @@
 	current->exit_signal = SIGCHLD;
 
 	current->ptrace = 0;
-	if ((current->policy == SCHED_OTHER) && (task_nice(current) < 0))
+	if ((current->policy == SCHED_NORMAL) && (task_nice(current) < 0))
 		set_user_nice(current, 0);
 	/* cpus_allowed? */
 	/* rt_priority? */

[-- Attachment #3: Type: TEXT/PLAIN, Size: 651 bytes --]


/*
 * Set a given PID to be a SCHED_BATCH process.
 * 
 * Copyright (C) 2002 Ingo Molnar
 */
#include <time.h>
#include <stdio.h>
#include <sched.h>
#include <stdlib.h>
#include <sys/types.h>
#include <linux/unistd.h>

int main (int argc, char **argv)
{
	int pid, ret;
	struct sched_param p;

	p.sched_priority = 0;

	if (argc != 2) {
		printf("usage: setbatch <pid>\n");
		exit(-1);
	}
	pid = atol(argv[1]);

	ret = sched_setscheduler(pid, 3, &p);

	if (ret) {
		printf("could not set pid %d to SCHED_BATCH: err %d.\n", pid, ret);
		return -1;
	}
	printf("pid %d is SCHED_BATCH from now on.\n", pid);
	return 0;
}

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-01  0:26 [announce] [patch] batch/idle priority scheduling, SCHED_BATCH Ingo Molnar
@ 2002-07-01  2:55 ` Nicholas Miell
  2002-07-01  6:48   ` Andreas Jaeger
  2002-07-04 13:04 ` Vitez Gabor
  1 sibling, 1 reply; 9+ messages in thread
From: Nicholas Miell @ 2002-07-01  2:55 UTC (permalink / raw)
  To: Ingo Molnar; +Cc: linux-kernel, Linus Torvalds

On Sun, 2002-06-30 at 17:26, Ingo Molnar wrote:

> -#define SCHED_OTHER		0
> +#define SCHED_NORMAL		0

>From IEEE 1003.1-2001 / Open Group Base Spec. Issue 6:
"Conforming implementations shall include one scheduling policy
identified as SCHED_OTHER (which may execute identically with either the
FIFO or round robin scheduling policy)."

So, you probably want to add a "#define SCHED_OTHER SCHED_NORMAL" here
in order to prevent future confusion, especially because the user-space
headers have SCHED_OTHER, but no SCHED_NORMAL.

- Nicholas


^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-01  2:55 ` Nicholas Miell
@ 2002-07-01  6:48   ` Andreas Jaeger
  2002-07-01  8:00     ` Ingo Molnar
  0 siblings, 1 reply; 9+ messages in thread
From: Andreas Jaeger @ 2002-07-01  6:48 UTC (permalink / raw)
  To: Nicholas Miell; +Cc: Ingo Molnar, linux-kernel, Linus Torvalds

Nicholas Miell <nmiell@attbi.com> writes:

> On Sun, 2002-06-30 at 17:26, Ingo Molnar wrote:
>
>> -#define SCHED_OTHER		0
>> +#define SCHED_NORMAL		0
>
>>From IEEE 1003.1-2001 / Open Group Base Spec. Issue 6:
> "Conforming implementations shall include one scheduling policy
> identified as SCHED_OTHER (which may execute identically with either the
> FIFO or round robin scheduling policy)."
>
> So, you probably want to add a "#define SCHED_OTHER SCHED_NORMAL" here
> in order to prevent future confusion, especially because the user-space
> headers have SCHED_OTHER, but no SCHED_NORMAL.

This can be done in glibc.  linux/sched.h should not be used by
userspace applications, glibc has the define in <bits/sched.h> which
is included from <sched.h> - and <sched.h> is the file defined by
Posix.

Andreas
-- 
 Andreas Jaeger
  SuSE Labs aj@suse.de
   private aj@arthur.inka.de
    http://www.suse.de/~aj

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-01  6:48   ` Andreas Jaeger
@ 2002-07-01  8:00     ` Ingo Molnar
  2002-07-02  0:02       ` Nicholas Miell
  0 siblings, 1 reply; 9+ messages in thread
From: Ingo Molnar @ 2002-07-01  8:00 UTC (permalink / raw)
  To: Andreas Jaeger; +Cc: Nicholas Miell, linux-kernel, Linus Torvalds


On Mon, 1 Jul 2002, Andreas Jaeger wrote:

> >> -#define SCHED_OTHER		0
> >> +#define SCHED_NORMAL		0
> >
> >>From IEEE 1003.1-2001 / Open Group Base Spec. Issue 6:
> > "Conforming implementations shall include one scheduling policy
> > identified as SCHED_OTHER (which may execute identically with either the
> > FIFO or round robin scheduling policy)."
> >
> > So, you probably want to add a "#define SCHED_OTHER SCHED_NORMAL" here
> > in order to prevent future confusion, especially because the user-space
> > headers have SCHED_OTHER, but no SCHED_NORMAL.
> 
> This can be done in glibc.  linux/sched.h should not be used by
> userspace applications, glibc has the define in <bits/sched.h> which is
> included from <sched.h> - and <sched.h> is the file defined by Posix.

yes, this was my thinking too.

the reason for the change: with the introduction of SCHED_BATCH the
regular scheduling policy cannot really be called 'other' anymore, from
the point of scheduler internals - it's in the middle of all scheduler
policies, its only speciality is that it's the default one.

(obviously for the user interface it has to be defined.)

	Ingo


^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-01  8:00     ` Ingo Molnar
@ 2002-07-02  0:02       ` Nicholas Miell
  2002-07-03  7:59         ` Ingo Molnar
  0 siblings, 1 reply; 9+ messages in thread
From: Nicholas Miell @ 2002-07-02  0:02 UTC (permalink / raw)
  To: Ingo Molnar; +Cc: Andreas Jaeger, linux-kernel, Linus Torvalds

On Mon, 2002-07-01 at 01:00, Ingo Molnar wrote:
> 
> On Mon, 1 Jul 2002, Andreas Jaeger wrote:
> > This can be done in glibc.  linux/sched.h should not be used by
> > userspace applications, glibc has the define in <bits/sched.h> which is
> > included from <sched.h> - and <sched.h> is the file defined by Posix.
> 
> yes, this was my thinking too.

OK, I can understand that line of reasoning.

Keep in mind that someday, someone who is looking for the implementation
of the SCHED_OTHER policy will be thoroughly confused by the kernel's
complete lack of reference to SCHED_OTHER. And they'll be asking you for
clarification.

Or, you could make some note in the source that SCHED_OTHER is
SCHED_NORMAL and eliminate any source of confusion now.

- Nicholas


^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-02  0:02       ` Nicholas Miell
@ 2002-07-03  7:59         ` Ingo Molnar
  0 siblings, 0 replies; 9+ messages in thread
From: Ingo Molnar @ 2002-07-03  7:59 UTC (permalink / raw)
  To: Nicholas Miell; +Cc: Andreas Jaeger, linux-kernel, Linus Torvalds


On 1 Jul 2002, Nicholas Miell wrote:

> > > This can be done in glibc.  linux/sched.h should not be used by
> > > userspace applications, glibc has the define in <bits/sched.h> which is
> > > included from <sched.h> - and <sched.h> is the file defined by Posix.
> > 
> > yes, this was my thinking too.
> 
> OK, I can understand that line of reasoning.
> 
> Keep in mind that someday, someone who is looking for the implementation
> of the SCHED_OTHER policy will be thoroughly confused by the kernel's
> complete lack of reference to SCHED_OTHER. And they'll be asking you for
> clarification.
> 
> Or, you could make some note in the source that SCHED_OTHER is
> SCHED_NORMAL and eliminate any source of confusion now.

okay, i did this.

and, to not confuse existing code that still happens to use the kernel
headers, i added the #define to the 2.4.19-pre10-ac2 backport of the O(1)
scheduler.

	Ingo


^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-01  0:26 [announce] [patch] batch/idle priority scheduling, SCHED_BATCH Ingo Molnar
  2002-07-01  2:55 ` Nicholas Miell
@ 2002-07-04 13:04 ` Vitez Gabor
  2002-07-07 12:46   ` Jakob Oestergaard
  1 sibling, 1 reply; 9+ messages in thread
From: Vitez Gabor @ 2002-07-04 13:04 UTC (permalink / raw)
  To: Ingo Molnar; +Cc: linux-kernel

On Mon, Jul 01, 2002 at 02:26:42AM +0200, Ingo Molnar wrote:
> the load generated by multiple SCHED_BATCH processes does not show up in
> the load average - this is the straightforward solution to not confuse
> load-average-sensitive applications such as sendmail.

I think this will confuse atd too, which is an obvious candidate 
for the batch scheduler; it may end up starting all jobs which 
sit in it's "batch" queue.

I think a load-average calculation scheme like this would be better:

oldload: is the load average calculated the old way
batchload: is the load average calculated only from the batch scheduler
numcpus: number of cpus...

newload(){
if (oldload > numcpus) return oldload;
if ((oldload+batchload) > numcpus) return numcpus;
return (oldload+batchload)
}

So the batch processes would show the CPUs maxed out, but would not show 
up as overload in the load average. (and you could run 
"atd -l <numcpus - 0.3>")

	regards:
		Gabor Vitez

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-04 13:04 ` Vitez Gabor
@ 2002-07-07 12:46   ` Jakob Oestergaard
  2002-07-07 19:07     ` Bernd Eckenfels
  0 siblings, 1 reply; 9+ messages in thread
From: Jakob Oestergaard @ 2002-07-07 12:46 UTC (permalink / raw)
  To: Vitez Gabor; +Cc: Ingo Molnar, linux-kernel

On Thu, Jul 04, 2002 at 03:04:49PM +0200, Vitez Gabor wrote:
> On Mon, Jul 01, 2002 at 02:26:42AM +0200, Ingo Molnar wrote:
> > the load generated by multiple SCHED_BATCH processes does not show up in
> > the load average - this is the straightforward solution to not confuse
> > load-average-sensitive applications such as sendmail.

If SCHED_BATCH load does not affect the load average, then it will *not*
confuse sendmail - because the box is not really loaded, SCHED_BATCH
will only run if sendmail wouldn't   :)

I think it's great that SCHED_BATCH doesn't show up in loadavg - it is
really confusing to look at load statistics for boxes which have these
CPU intensive batch jobs running - the load average is perhaps
constantly between 2.1 and 2.2, when the "real" load on the box would
cause it to be between .1 and .2

Idle-time is what you pay for but do not get  ;)   Having CPU hogs
running is a nice way of getting the last penny out of the investment,
but I have many boxes where I don't do it, simply because it would
render the load statistics useless.

> 
> I think this will confuse atd too, which is an obvious candidate 
> for the batch scheduler; it may end up starting all jobs which 
> sit in it's "batch" queue.

True.  I am inclined to saying that any batch system that doesn't keep
track of it's own jobs but only cares about the load average, is flawed.

Load average is "guidance", it is a heuristic, it is not something one
should use as the sole measure when deciding when/where to spawn jobs.
Stuff like that only works in theory...

But atd is in use everywhere, and it does use the load as the only
metric (which is, by the way, why it doesn't start more than one job
per minute, because the loadavg needs time to rise).

Wouldn't it be pretty simple to just make atd have a hard limit on how
many concurrent at jobs (eventually on a per-user basis) it would start?

Unless you really go to extremes (thousands of jobs), the performance of
ten "concurrently" running SCHED_BATCH jobs and the same ten jobs beeing
launched sequentially by atd, should be fairly similar - given the huge
time-slices given by the scheduler to these jobs.

> 
> I think a load-average calculation scheme like this would be better:
> 
> oldload: is the load average calculated the old way
> batchload: is the load average calculated only from the batch scheduler
> numcpus: number of cpus...
> 
> newload(){
> if (oldload > numcpus) return oldload;
> if ((oldload+batchload) > numcpus) return numcpus;
> return (oldload+batchload)
> }
> 
> So the batch processes would show the CPUs maxed out, but would not show 
> up as overload in the load average. (and you could run 
> "atd -l <numcpus - 0.3>")
> 

Hmmm...  Such a hack might work around the shortcomings in the atd
scheduling algorithm.

I don't like it. It adds yet another level of obfuscation to loadavg.

I bet you can do the proper changes to atd in the same amount of code,
keeping the kernel clean and fixing the problem where it really is.

Just my 0.02 Euro,

-- 
................................................................
:   jakob@unthought.net   : And I see the elder races,         :
:.........................: putrid forms of man                :
:   Jakob Østergaard      : See him rise and claim the earth,  :
:        OZ9ABN           : his downfall is at hand.           :
:.........................:............{Konkhra}...............:

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [announce] [patch] batch/idle priority scheduling, SCHED_BATCH
  2002-07-07 12:46   ` Jakob Oestergaard
@ 2002-07-07 19:07     ` Bernd Eckenfels
  0 siblings, 0 replies; 9+ messages in thread
From: Bernd Eckenfels @ 2002-07-07 19:07 UTC (permalink / raw)
  To: linux-kernel

In article <20020707124613.GE30476@unthought.net> you wrote:
> Idle-time is what you pay for but do not get  ;)   Having CPU hogs
> running is a nice way of getting the last penny out of the investment,
> but I have many boxes where I don't do it, simply because it would
> render the load statistics useless.

And of course it needs a bit more power and produces much more temperature
(and therefore noise).

BTW: even an batch prio will load the system more than an idle process. The
cache is dirty, the disk has a longer work queue, RSS is used up by that
programs. So for a slow system you have to expect impact.

Greetings
Bernd

^ permalink raw reply	[flat|nested] 9+ messages in thread

end of thread, other threads:[~2002-07-07 19:05 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2002-07-01  0:26 [announce] [patch] batch/idle priority scheduling, SCHED_BATCH Ingo Molnar
2002-07-01  2:55 ` Nicholas Miell
2002-07-01  6:48   ` Andreas Jaeger
2002-07-01  8:00     ` Ingo Molnar
2002-07-02  0:02       ` Nicholas Miell
2002-07-03  7:59         ` Ingo Molnar
2002-07-04 13:04 ` Vitez Gabor
2002-07-07 12:46   ` Jakob Oestergaard
2002-07-07 19:07     ` Bernd Eckenfels

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).