Remove prefetch() from <linux/skbuff.h> and "netlabel_addrlist.h"
[pandora-kernel.git] / kernel / rcutree.c
index dd4aea8..f07d2f0 100644 (file)
@@ -47,6 +47,9 @@
 #include <linux/mutex.h>
 #include <linux/time.h>
 #include <linux/kernel_stat.h>
+#include <linux/wait.h>
+#include <linux/kthread.h>
+#include <linux/prefetch.h>
 
 #include "rcutree.h"
 
@@ -79,9 +82,40 @@ DEFINE_PER_CPU(struct rcu_data, rcu_sched_data);
 struct rcu_state rcu_bh_state = RCU_STATE_INITIALIZER(rcu_bh_state);
 DEFINE_PER_CPU(struct rcu_data, rcu_bh_data);
 
+static struct rcu_state *rcu_state;
+
 int rcu_scheduler_active __read_mostly;
 EXPORT_SYMBOL_GPL(rcu_scheduler_active);
 
+/*
+ * Control variables for per-CPU and per-rcu_node kthreads.  These
+ * handle all flavors of RCU.
+ */
+static DEFINE_PER_CPU(struct task_struct *, rcu_cpu_kthread_task);
+DEFINE_PER_CPU(unsigned int, rcu_cpu_kthread_status);
+DEFINE_PER_CPU(int, rcu_cpu_kthread_cpu);
+DEFINE_PER_CPU(unsigned int, rcu_cpu_kthread_loops);
+static DEFINE_PER_CPU(wait_queue_head_t, rcu_cpu_wq);
+DEFINE_PER_CPU(char, rcu_cpu_has_work);
+static char rcu_kthreads_spawnable;
+
+static void rcu_node_kthread_setaffinity(struct rcu_node *rnp, int outgoingcpu);
+static void invoke_rcu_cpu_kthread(void);
+
+#define RCU_KTHREAD_PRIO 1     /* RT priority for per-CPU kthreads. */
+
+/*
+ * Track the rcutorture test sequence number and the update version
+ * number within a given test.  The rcutorture_testseq is incremented
+ * on every rcutorture module load and unload, so has an odd value
+ * when a test is running.  The rcutorture_vernum is set to zero
+ * when rcutorture starts and is incremented on each rcutorture update.
+ * These variables enable correlating rcutorture output with the
+ * RCU tracing information.
+ */
+unsigned long rcutorture_testseq;
+unsigned long rcutorture_vernum;
+
 /*
  * Return true if an RCU grace period is in progress.  The ACCESS_ONCE()s
  * permit this function to be invoked without holding the root rcu_node
@@ -124,6 +158,7 @@ void rcu_note_context_switch(int cpu)
        rcu_sched_qs(cpu);
        rcu_preempt_note_context_switch(cpu);
 }
+EXPORT_SYMBOL_GPL(rcu_note_context_switch);
 
 #ifdef CONFIG_NO_HZ
 DEFINE_PER_CPU(struct rcu_dynticks, rcu_dynticks) = {
@@ -140,10 +175,8 @@ module_param(blimit, int, 0);
 module_param(qhimark, int, 0);
 module_param(qlowmark, int, 0);
 
-#ifdef CONFIG_RCU_CPU_STALL_DETECTOR
-int rcu_cpu_stall_suppress __read_mostly = RCU_CPU_STALL_SUPPRESS_INIT;
+int rcu_cpu_stall_suppress __read_mostly;
 module_param(rcu_cpu_stall_suppress, int, 0644);
-#endif /* #ifdef CONFIG_RCU_CPU_STALL_DETECTOR */
 
 static void force_quiescent_state(struct rcu_state *rsp, int relaxed);
 static int rcu_pending(int cpu);
@@ -175,6 +208,31 @@ void rcu_bh_force_quiescent_state(void)
 }
 EXPORT_SYMBOL_GPL(rcu_bh_force_quiescent_state);
 
+/*
+ * Record the number of times rcutorture tests have been initiated and
+ * terminated.  This information allows the debugfs tracing stats to be
+ * correlated to the rcutorture messages, even when the rcutorture module
+ * is being repeatedly loaded and unloaded.  In other words, we cannot
+ * store this state in rcutorture itself.
+ */
+void rcutorture_record_test_transition(void)
+{
+       rcutorture_testseq++;
+       rcutorture_vernum = 0;
+}
+EXPORT_SYMBOL_GPL(rcutorture_record_test_transition);
+
+/*
+ * Record the number of writer passes through the current rcutorture test.
+ * This is also used to correlate debugfs tracing stats with the rcutorture
+ * messages.
+ */
+void rcutorture_record_progress(unsigned long vernum)
+{
+       rcutorture_vernum++;
+}
+EXPORT_SYMBOL_GPL(rcutorture_record_progress);
+
 /*
  * Force a quiescent state for RCU-sched.
  */
@@ -234,8 +292,8 @@ static int rcu_implicit_offline_qs(struct rcu_data *rdp)
                return 1;
        }
 
-       /* If preemptable RCU, no point in sending reschedule IPI. */
-       if (rdp->preemptable)
+       /* If preemptible RCU, no point in sending reschedule IPI. */
+       if (rdp->preemptible)
                return 0;
 
        /* The CPU is online, so send it a reschedule IPI. */
@@ -450,8 +508,6 @@ static int rcu_implicit_dynticks_qs(struct rcu_data *rdp)
 
 #endif /* #else #ifdef CONFIG_NO_HZ */
 
-#ifdef CONFIG_RCU_CPU_STALL_DETECTOR
-
 int rcu_cpu_stall_suppress __read_mostly;
 
 static void record_gp_stall_check_time(struct rcu_state *rsp)
@@ -537,21 +593,24 @@ static void print_cpu_stall(struct rcu_state *rsp)
 
 static void check_cpu_stall(struct rcu_state *rsp, struct rcu_data *rdp)
 {
-       long delta;
+       unsigned long j;
+       unsigned long js;
        struct rcu_node *rnp;
 
        if (rcu_cpu_stall_suppress)
                return;
-       delta = jiffies - ACCESS_ONCE(rsp->jiffies_stall);
+       j = ACCESS_ONCE(jiffies);
+       js = ACCESS_ONCE(rsp->jiffies_stall);
        rnp = rdp->mynode;
-       if ((ACCESS_ONCE(rnp->qsmask) & rdp->grpmask) && delta >= 0) {
+       if ((ACCESS_ONCE(rnp->qsmask) & rdp->grpmask) && ULONG_CMP_GE(j, js)) {
 
                /* We haven't checked in, so go dump stack. */
                print_cpu_stall(rsp);
 
-       } else if (rcu_gp_in_progress(rsp) && delta >= RCU_STALL_RAT_DELAY) {
+       } else if (rcu_gp_in_progress(rsp) &&
+                  ULONG_CMP_GE(j, js + RCU_STALL_RAT_DELAY)) {
 
-               /* They had two time units to dump stack, so complain. */
+               /* They had a few time units to dump stack, so complain. */
                print_other_cpu_stall(rsp);
        }
 }
@@ -587,26 +646,6 @@ static void __init check_cpu_stall_init(void)
        atomic_notifier_chain_register(&panic_notifier_list, &rcu_panic_block);
 }
 
-#else /* #ifdef CONFIG_RCU_CPU_STALL_DETECTOR */
-
-static void record_gp_stall_check_time(struct rcu_state *rsp)
-{
-}
-
-static void check_cpu_stall(struct rcu_state *rsp, struct rcu_data *rdp)
-{
-}
-
-void rcu_cpu_stall_reset(void)
-{
-}
-
-static void __init check_cpu_stall_init(void)
-{
-}
-
-#endif /* #else #ifdef CONFIG_RCU_CPU_STALL_DETECTOR */
-
 /*
  * Update CPU-local rcu_data state to record the newly noticed grace period.
  * This is used both when we started the grace period and when we notice
@@ -809,6 +848,7 @@ rcu_start_gp(struct rcu_state *rsp, unsigned long flags)
                rnp->completed = rsp->completed;
                rsp->signaled = RCU_SIGNAL_INIT; /* force_quiescent_state OK. */
                rcu_start_gp_per_cpu(rsp, rnp, rdp);
+               rcu_preempt_boost_start_gp(rnp);
                raw_spin_unlock_irqrestore(&rnp->lock, flags);
                return;
        }
@@ -844,6 +884,7 @@ rcu_start_gp(struct rcu_state *rsp, unsigned long flags)
                rnp->completed = rsp->completed;
                if (rnp == rdp->mynode)
                        rcu_start_gp_per_cpu(rsp, rnp, rdp);
+               rcu_preempt_boost_start_gp(rnp);
                raw_spin_unlock(&rnp->lock);    /* irqs remain disabled. */
        }
 
@@ -864,7 +905,12 @@ rcu_start_gp(struct rcu_state *rsp, unsigned long flags)
 static void rcu_report_qs_rsp(struct rcu_state *rsp, unsigned long flags)
        __releases(rcu_get_root(rsp)->lock)
 {
+       unsigned long gp_duration;
+
        WARN_ON_ONCE(!rcu_gp_in_progress(rsp));
+       gp_duration = jiffies - rsp->gp_start;
+       if (gp_duration > rsp->gp_max)
+               rsp->gp_max = gp_duration;
        rsp->completed = rsp->gpnum;
        rsp->signaled = RCU_GP_IDLE;
        rcu_start_gp(rsp, flags);  /* releases root node's rnp->lock. */
@@ -894,7 +940,7 @@ rcu_report_qs_rnp(unsigned long mask, struct rcu_state *rsp,
                        return;
                }
                rnp->qsmask &= ~mask;
-               if (rnp->qsmask != 0 || rcu_preempted_readers(rnp)) {
+               if (rnp->qsmask != 0 || rcu_preempt_blocked_readers_cgp(rnp)) {
 
                        /* Other bits still set at this level, so done. */
                        raw_spin_unlock_irqrestore(&rnp->lock, flags);
@@ -1037,6 +1083,8 @@ static void rcu_send_cbs_to_online(struct rcu_state *rsp)
 /*
  * Remove the outgoing CPU from the bitmasks in the rcu_node hierarchy
  * and move all callbacks from the outgoing CPU to the current one.
+ * There can only be one CPU hotplug operation at a time, so no other
+ * CPU can be attempting to update rcu_cpu_kthread_task.
  */
 static void __rcu_offline_cpu(int cpu, struct rcu_state *rsp)
 {
@@ -1045,6 +1093,14 @@ static void __rcu_offline_cpu(int cpu, struct rcu_state *rsp)
        int need_report = 0;
        struct rcu_data *rdp = per_cpu_ptr(rsp->rda, cpu);
        struct rcu_node *rnp;
+       struct task_struct *t;
+
+       /* Stop the CPU's kthread. */
+       t = per_cpu(rcu_cpu_kthread_task, cpu);
+       if (t != NULL) {
+               per_cpu(rcu_cpu_kthread_task, cpu) = NULL;
+               kthread_stop(t);
+       }
 
        /* Exclude any attempts to start a new grace period. */
        raw_spin_lock_irqsave(&rsp->onofflock, flags);
@@ -1082,6 +1138,7 @@ static void __rcu_offline_cpu(int cpu, struct rcu_state *rsp)
                raw_spin_unlock_irqrestore(&rnp->lock, flags);
        if (need_report & RCU_OFL_TASKS_EXP_GP)
                rcu_report_exp_rnp(rsp, rnp);
+       rcu_node_kthread_setaffinity(rnp, -1);
 }
 
 /*
@@ -1143,7 +1200,7 @@ static void rcu_do_batch(struct rcu_state *rsp, struct rcu_data *rdp)
                next = list->next;
                prefetch(next);
                debug_rcu_head_unqueue(list);
-               list->func(list);
+               __rcu_reclaim(list);
                list = next;
                if (++count >= rdp->blimit)
                        break;
@@ -1179,7 +1236,7 @@ static void rcu_do_batch(struct rcu_state *rsp, struct rcu_data *rdp)
 
        /* Re-raise the RCU softirq if there are callbacks remaining. */
        if (cpu_has_callbacks_ready_to_invoke(rdp))
-               raise_softirq(RCU_SOFTIRQ);
+               invoke_rcu_cpu_kthread();
 }
 
 /*
@@ -1225,7 +1282,7 @@ void rcu_check_callbacks(int cpu, int user)
        }
        rcu_preempt_check_callbacks(cpu);
        if (rcu_pending(cpu))
-               raise_softirq(RCU_SOFTIRQ);
+               invoke_rcu_cpu_kthread();
 }
 
 #ifdef CONFIG_SMP
@@ -1233,6 +1290,8 @@ void rcu_check_callbacks(int cpu, int user)
 /*
  * Scan the leaf rcu_node structures, processing dyntick state for any that
  * have not yet encountered a quiescent state, using the function specified.
+ * Also initiate boosting for any threads blocked on the root rcu_node.
+ *
  * The caller must have suppressed start of new grace periods.
  */
 static void force_qs_rnp(struct rcu_state *rsp, int (*f)(struct rcu_data *))
@@ -1251,7 +1310,7 @@ static void force_qs_rnp(struct rcu_state *rsp, int (*f)(struct rcu_data *))
                        return;
                }
                if (rnp->qsmask == 0) {
-                       raw_spin_unlock_irqrestore(&rnp->lock, flags);
+                       rcu_initiate_boost(rnp, flags); /* releases rnp->lock */
                        continue;
                }
                cpu = rnp->grplo;
@@ -1269,6 +1328,11 @@ static void force_qs_rnp(struct rcu_state *rsp, int (*f)(struct rcu_data *))
                }
                raw_spin_unlock_irqrestore(&rnp->lock, flags);
        }
+       rnp = rcu_get_root(rsp);
+       if (rnp->qsmask == 0) {
+               raw_spin_lock_irqsave(&rnp->lock, flags);
+               rcu_initiate_boost(rnp, flags); /* releases rnp->lock. */
+       }
 }
 
 /*
@@ -1389,7 +1453,7 @@ __rcu_process_callbacks(struct rcu_state *rsp, struct rcu_data *rdp)
 /*
  * Do softirq processing for the current CPU.
  */
-static void rcu_process_callbacks(struct softirq_action *unused)
+static void rcu_process_callbacks(void)
 {
        /*
         * Memory references from any prior RCU read-side critical sections
@@ -1414,6 +1478,347 @@ static void rcu_process_callbacks(struct softirq_action *unused)
        rcu_needs_cpu_flush();
 }
 
+/*
+ * Wake up the current CPU's kthread.  This replaces raise_softirq()
+ * in earlier versions of RCU.  Note that because we are running on
+ * the current CPU with interrupts disabled, the rcu_cpu_kthread_task
+ * cannot disappear out from under us.
+ */
+static void invoke_rcu_cpu_kthread(void)
+{
+       unsigned long flags;
+
+       local_irq_save(flags);
+       __this_cpu_write(rcu_cpu_has_work, 1);
+       if (__this_cpu_read(rcu_cpu_kthread_task) == NULL) {
+               local_irq_restore(flags);
+               return;
+       }
+       wake_up(&__get_cpu_var(rcu_cpu_wq));
+       local_irq_restore(flags);
+}
+
+/*
+ * Wake up the specified per-rcu_node-structure kthread.
+ * Because the per-rcu_node kthreads are immortal, we don't need
+ * to do anything to keep them alive.
+ */
+static void invoke_rcu_node_kthread(struct rcu_node *rnp)
+{
+       struct task_struct *t;
+
+       t = rnp->node_kthread_task;
+       if (t != NULL)
+               wake_up_process(t);
+}
+
+/*
+ * Set the specified CPU's kthread to run RT or not, as specified by
+ * the to_rt argument.  The CPU-hotplug locks are held, so the task
+ * is not going away.
+ */
+static void rcu_cpu_kthread_setrt(int cpu, int to_rt)
+{
+       int policy;
+       struct sched_param sp;
+       struct task_struct *t;
+
+       t = per_cpu(rcu_cpu_kthread_task, cpu);
+       if (t == NULL)
+               return;
+       if (to_rt) {
+               policy = SCHED_FIFO;
+               sp.sched_priority = RCU_KTHREAD_PRIO;
+       } else {
+               policy = SCHED_NORMAL;
+               sp.sched_priority = 0;
+       }
+       sched_setscheduler_nocheck(t, policy, &sp);
+}
+
+/*
+ * Timer handler to initiate the waking up of per-CPU kthreads that
+ * have yielded the CPU due to excess numbers of RCU callbacks.
+ * We wake up the per-rcu_node kthread, which in turn will wake up
+ * the booster kthread.
+ */
+static void rcu_cpu_kthread_timer(unsigned long arg)
+{
+       unsigned long flags;
+       struct rcu_data *rdp = per_cpu_ptr(rcu_state->rda, arg);
+       struct rcu_node *rnp = rdp->mynode;
+
+       raw_spin_lock_irqsave(&rnp->lock, flags);
+       rnp->wakemask |= rdp->grpmask;
+       raw_spin_unlock_irqrestore(&rnp->lock, flags);
+       invoke_rcu_node_kthread(rnp);
+}
+
+/*
+ * Drop to non-real-time priority and yield, but only after posting a
+ * timer that will cause us to regain our real-time priority if we
+ * remain preempted.  Either way, we restore our real-time priority
+ * before returning.
+ */
+static void rcu_yield(void (*f)(unsigned long), unsigned long arg)
+{
+       struct sched_param sp;
+       struct timer_list yield_timer;
+
+       setup_timer_on_stack(&yield_timer, f, arg);
+       mod_timer(&yield_timer, jiffies + 2);
+       sp.sched_priority = 0;
+       sched_setscheduler_nocheck(current, SCHED_NORMAL, &sp);
+       set_user_nice(current, 19);
+       schedule();
+       sp.sched_priority = RCU_KTHREAD_PRIO;
+       sched_setscheduler_nocheck(current, SCHED_FIFO, &sp);
+       del_timer(&yield_timer);
+}
+
+/*
+ * Handle cases where the rcu_cpu_kthread() ends up on the wrong CPU.
+ * This can happen while the corresponding CPU is either coming online
+ * or going offline.  We cannot wait until the CPU is fully online
+ * before starting the kthread, because the various notifier functions
+ * can wait for RCU grace periods.  So we park rcu_cpu_kthread() until
+ * the corresponding CPU is online.
+ *
+ * Return 1 if the kthread needs to stop, 0 otherwise.
+ *
+ * Caller must disable bh.  This function can momentarily enable it.
+ */
+static int rcu_cpu_kthread_should_stop(int cpu)
+{
+       while (cpu_is_offline(cpu) ||
+              !cpumask_equal(&current->cpus_allowed, cpumask_of(cpu)) ||
+              smp_processor_id() != cpu) {
+               if (kthread_should_stop())
+                       return 1;
+               per_cpu(rcu_cpu_kthread_status, cpu) = RCU_KTHREAD_OFFCPU;
+               per_cpu(rcu_cpu_kthread_cpu, cpu) = raw_smp_processor_id();
+               local_bh_enable();
+               schedule_timeout_uninterruptible(1);
+               if (!cpumask_equal(&current->cpus_allowed, cpumask_of(cpu)))
+                       set_cpus_allowed_ptr(current, cpumask_of(cpu));
+               local_bh_disable();
+       }
+       per_cpu(rcu_cpu_kthread_cpu, cpu) = cpu;
+       return 0;
+}
+
+/*
+ * Per-CPU kernel thread that invokes RCU callbacks.  This replaces the
+ * earlier RCU softirq.
+ */
+static int rcu_cpu_kthread(void *arg)
+{
+       int cpu = (int)(long)arg;
+       unsigned long flags;
+       int spincnt = 0;
+       unsigned int *statusp = &per_cpu(rcu_cpu_kthread_status, cpu);
+       wait_queue_head_t *wqp = &per_cpu(rcu_cpu_wq, cpu);
+       char work;
+       char *workp = &per_cpu(rcu_cpu_has_work, cpu);
+
+       for (;;) {
+               *statusp = RCU_KTHREAD_WAITING;
+               wait_event_interruptible(*wqp,
+                                        *workp != 0 || kthread_should_stop());
+               local_bh_disable();
+               if (rcu_cpu_kthread_should_stop(cpu)) {
+                       local_bh_enable();
+                       break;
+               }
+               *statusp = RCU_KTHREAD_RUNNING;
+               per_cpu(rcu_cpu_kthread_loops, cpu)++;
+               local_irq_save(flags);
+               work = *workp;
+               *workp = 0;
+               local_irq_restore(flags);
+               if (work)
+                       rcu_process_callbacks();
+               local_bh_enable();
+               if (*workp != 0)
+                       spincnt++;
+               else
+                       spincnt = 0;
+               if (spincnt > 10) {
+                       *statusp = RCU_KTHREAD_YIELDING;
+                       rcu_yield(rcu_cpu_kthread_timer, (unsigned long)cpu);
+                       spincnt = 0;
+               }
+       }
+       *statusp = RCU_KTHREAD_STOPPED;
+       return 0;
+}
+
+/*
+ * Spawn a per-CPU kthread, setting up affinity and priority.
+ * Because the CPU hotplug lock is held, no other CPU will be attempting
+ * to manipulate rcu_cpu_kthread_task.  There might be another CPU
+ * attempting to access it during boot, but the locking in kthread_bind()
+ * will enforce sufficient ordering.
+ */
+static int __cpuinit rcu_spawn_one_cpu_kthread(int cpu)
+{
+       struct sched_param sp;
+       struct task_struct *t;
+
+       if (!rcu_kthreads_spawnable ||
+           per_cpu(rcu_cpu_kthread_task, cpu) != NULL)
+               return 0;
+       t = kthread_create(rcu_cpu_kthread, (void *)(long)cpu, "rcuc%d", cpu);
+       if (IS_ERR(t))
+               return PTR_ERR(t);
+       kthread_bind(t, cpu);
+       per_cpu(rcu_cpu_kthread_cpu, cpu) = cpu;
+       WARN_ON_ONCE(per_cpu(rcu_cpu_kthread_task, cpu) != NULL);
+       per_cpu(rcu_cpu_kthread_task, cpu) = t;
+       wake_up_process(t);
+       sp.sched_priority = RCU_KTHREAD_PRIO;
+       sched_setscheduler_nocheck(t, SCHED_FIFO, &sp);
+       return 0;
+}
+
+/*
+ * Per-rcu_node kthread, which is in charge of waking up the per-CPU
+ * kthreads when needed.  We ignore requests to wake up kthreads
+ * for offline CPUs, which is OK because force_quiescent_state()
+ * takes care of this case.
+ */
+static int rcu_node_kthread(void *arg)
+{
+       int cpu;
+       unsigned long flags;
+       unsigned long mask;
+       struct rcu_node *rnp = (struct rcu_node *)arg;
+       struct sched_param sp;
+       struct task_struct *t;
+
+       for (;;) {
+               rnp->node_kthread_status = RCU_KTHREAD_WAITING;
+               wait_event_interruptible(rnp->node_wq, rnp->wakemask != 0);
+               rnp->node_kthread_status = RCU_KTHREAD_RUNNING;
+               raw_spin_lock_irqsave(&rnp->lock, flags);
+               mask = rnp->wakemask;
+               rnp->wakemask = 0;
+               rcu_initiate_boost(rnp, flags); /* releases rnp->lock. */
+               for (cpu = rnp->grplo; cpu <= rnp->grphi; cpu++, mask >>= 1) {
+                       if ((mask & 0x1) == 0)
+                               continue;
+                       preempt_disable();
+                       t = per_cpu(rcu_cpu_kthread_task, cpu);
+                       if (!cpu_online(cpu) || t == NULL) {
+                               preempt_enable();
+                               continue;
+                       }
+                       per_cpu(rcu_cpu_has_work, cpu) = 1;
+                       sp.sched_priority = RCU_KTHREAD_PRIO;
+                       sched_setscheduler_nocheck(t, SCHED_FIFO, &sp);
+                       preempt_enable();
+               }
+       }
+       /* NOTREACHED */
+       rnp->node_kthread_status = RCU_KTHREAD_STOPPED;
+       return 0;
+}
+
+/*
+ * Set the per-rcu_node kthread's affinity to cover all CPUs that are
+ * served by the rcu_node in question.  The CPU hotplug lock is still
+ * held, so the value of rnp->qsmaskinit will be stable.
+ *
+ * We don't include outgoingcpu in the affinity set, use -1 if there is
+ * no outgoing CPU.  If there are no CPUs left in the affinity set,
+ * this function allows the kthread to execute on any CPU.
+ */
+static void rcu_node_kthread_setaffinity(struct rcu_node *rnp, int outgoingcpu)
+{
+       cpumask_var_t cm;
+       int cpu;
+       unsigned long mask = rnp->qsmaskinit;
+
+       if (rnp->node_kthread_task == NULL)
+               return;
+       if (!alloc_cpumask_var(&cm, GFP_KERNEL))
+               return;
+       cpumask_clear(cm);
+       for (cpu = rnp->grplo; cpu <= rnp->grphi; cpu++, mask >>= 1)
+               if ((mask & 0x1) && cpu != outgoingcpu)
+                       cpumask_set_cpu(cpu, cm);
+       if (cpumask_weight(cm) == 0) {
+               cpumask_setall(cm);
+               for (cpu = rnp->grplo; cpu <= rnp->grphi; cpu++)
+                       cpumask_clear_cpu(cpu, cm);
+               WARN_ON_ONCE(cpumask_weight(cm) == 0);
+       }
+       set_cpus_allowed_ptr(rnp->node_kthread_task, cm);
+       rcu_boost_kthread_setaffinity(rnp, cm);
+       free_cpumask_var(cm);
+}
+
+/*
+ * Spawn a per-rcu_node kthread, setting priority and affinity.
+ * Called during boot before online/offline can happen, or, if
+ * during runtime, with the main CPU-hotplug locks held.  So only
+ * one of these can be executing at a time.
+ */
+static int __cpuinit rcu_spawn_one_node_kthread(struct rcu_state *rsp,
+                                               struct rcu_node *rnp)
+{
+       unsigned long flags;
+       int rnp_index = rnp - &rsp->node[0];
+       struct sched_param sp;
+       struct task_struct *t;
+
+       if (!rcu_kthreads_spawnable ||
+           rnp->qsmaskinit == 0)
+               return 0;
+       if (rnp->node_kthread_task == NULL) {
+               t = kthread_create(rcu_node_kthread, (void *)rnp,
+                                  "rcun%d", rnp_index);
+               if (IS_ERR(t))
+                       return PTR_ERR(t);
+               raw_spin_lock_irqsave(&rnp->lock, flags);
+               rnp->node_kthread_task = t;
+               raw_spin_unlock_irqrestore(&rnp->lock, flags);
+               wake_up_process(t);
+               sp.sched_priority = 99;
+               sched_setscheduler_nocheck(t, SCHED_FIFO, &sp);
+       }
+       return rcu_spawn_one_boost_kthread(rsp, rnp, rnp_index);
+}
+
+/*
+ * Spawn all kthreads -- called as soon as the scheduler is running.
+ */
+static int __init rcu_spawn_kthreads(void)
+{
+       int cpu;
+       struct rcu_node *rnp;
+
+       rcu_kthreads_spawnable = 1;
+       for_each_possible_cpu(cpu) {
+               init_waitqueue_head(&per_cpu(rcu_cpu_wq, cpu));
+               per_cpu(rcu_cpu_has_work, cpu) = 0;
+               if (cpu_online(cpu))
+                       (void)rcu_spawn_one_cpu_kthread(cpu);
+       }
+       rnp = rcu_get_root(rcu_state);
+       init_waitqueue_head(&rnp->node_wq);
+       rcu_init_boost_waitqueue(rnp);
+       (void)rcu_spawn_one_node_kthread(rcu_state, rnp);
+       if (NUM_RCU_NODES > 1)
+               rcu_for_each_leaf_node(rcu_state, rnp) {
+                       init_waitqueue_head(&rnp->node_wq);
+                       rcu_init_boost_waitqueue(rnp);
+                       (void)rcu_spawn_one_node_kthread(rcu_state, rnp);
+               }
+       return 0;
+}
+early_initcall(rcu_spawn_kthreads);
+
 static void
 __call_rcu(struct rcu_head *head, void (*func)(struct rcu_head *rcu),
           struct rcu_state *rsp)
@@ -1439,6 +1844,13 @@ __call_rcu(struct rcu_head *head, void (*func)(struct rcu_head *rcu),
        /* Add the callback to our list. */
        *rdp->nxttail[RCU_NEXT_TAIL] = head;
        rdp->nxttail[RCU_NEXT_TAIL] = &head->next;
+       rdp->qlen++;
+
+       /* If interrupts were disabled, don't dive into RCU core. */
+       if (irqs_disabled_flags(flags)) {
+               local_irq_restore(flags);
+               return;
+       }
 
        /*
         * Force the grace period if too many callbacks or too long waiting.
@@ -1447,7 +1859,7 @@ __call_rcu(struct rcu_head *head, void (*func)(struct rcu_head *rcu),
         * invoking force_quiescent_state() if the newly enqueued callback
         * is the only one waiting for a grace period to complete.
         */
-       if (unlikely(++rdp->qlen > rdp->qlen_last_fqs_check + qhimark)) {
+       if (unlikely(rdp->qlen > rdp->qlen_last_fqs_check + qhimark)) {
 
                /* Are we ignoring a completed grace period? */
                rcu_process_gp_end(rsp, rdp);
@@ -1583,7 +1995,7 @@ static int __rcu_pending(struct rcu_state *rsp, struct rcu_data *rdp)
                 * or RCU-bh, force a local reschedule.
                 */
                rdp->n_rp_qs_pending++;
-               if (!rdp->preemptable &&
+               if (!rdp->preemptible &&
                    ULONG_CMP_LT(ACCESS_ONCE(rsp->jiffies_force_qs) - 1,
                                 jiffies))
                        set_need_resched();
@@ -1760,7 +2172,7 @@ rcu_boot_init_percpu_data(int cpu, struct rcu_state *rsp)
  * that this CPU cannot possibly have any RCU callbacks in flight yet.
  */
 static void __cpuinit
-rcu_init_percpu_data(int cpu, struct rcu_state *rsp, int preemptable)
+rcu_init_percpu_data(int cpu, struct rcu_state *rsp, int preemptible)
 {
        unsigned long flags;
        unsigned long mask;
@@ -1772,7 +2184,7 @@ rcu_init_percpu_data(int cpu, struct rcu_state *rsp, int preemptable)
        rdp->passed_quiesc = 0;  /* We could be racing with new GP, */
        rdp->qs_pending = 1;     /*  so set up to respond to current GP. */
        rdp->beenonline = 1;     /* We have now been online. */
-       rdp->preemptable = preemptable;
+       rdp->preemptible = preemptible;
        rdp->qlen_last_fqs_check = 0;
        rdp->n_force_qs_snap = rsp->n_force_qs;
        rdp->blimit = blimit;
@@ -1813,6 +2225,19 @@ static void __cpuinit rcu_online_cpu(int cpu)
        rcu_preempt_init_percpu_data(cpu);
 }
 
+static void __cpuinit rcu_online_kthreads(int cpu)
+{
+       struct rcu_data *rdp = per_cpu_ptr(rcu_state->rda, cpu);
+       struct rcu_node *rnp = rdp->mynode;
+
+       /* Fire up the incoming CPU's kthread and leaf rcu_node kthread. */
+       if (rcu_kthreads_spawnable) {
+               (void)rcu_spawn_one_cpu_kthread(cpu);
+               if (rnp->node_kthread_task == NULL)
+                       (void)rcu_spawn_one_node_kthread(rcu_state, rnp);
+       }
+}
+
 /*
  * Handle CPU online/offline notification events.
  */
@@ -1820,11 +2245,23 @@ static int __cpuinit rcu_cpu_notify(struct notifier_block *self,
                                    unsigned long action, void *hcpu)
 {
        long cpu = (long)hcpu;
+       struct rcu_data *rdp = per_cpu_ptr(rcu_state->rda, cpu);
+       struct rcu_node *rnp = rdp->mynode;
 
        switch (action) {
        case CPU_UP_PREPARE:
        case CPU_UP_PREPARE_FROZEN:
                rcu_online_cpu(cpu);
+               rcu_online_kthreads(cpu);
+               break;
+       case CPU_ONLINE:
+       case CPU_DOWN_FAILED:
+               rcu_node_kthread_setaffinity(rnp, -1);
+               rcu_cpu_kthread_setrt(cpu, 1);
+               break;
+       case CPU_DOWN_PREPARE:
+               rcu_node_kthread_setaffinity(rnp, cpu);
+               rcu_cpu_kthread_setrt(cpu, 0);
                break;
        case CPU_DYING:
        case CPU_DYING_FROZEN:
@@ -1943,10 +2380,7 @@ static void __init rcu_init_one(struct rcu_state *rsp,
                                              j / rsp->levelspread[i - 1];
                        }
                        rnp->level = i;
-                       INIT_LIST_HEAD(&rnp->blocked_tasks[0]);
-                       INIT_LIST_HEAD(&rnp->blocked_tasks[1]);
-                       INIT_LIST_HEAD(&rnp->blocked_tasks[2]);
-                       INIT_LIST_HEAD(&rnp->blocked_tasks[3]);
+                       INIT_LIST_HEAD(&rnp->blkd_tasks);
                }
        }
 
@@ -1968,7 +2402,6 @@ void __init rcu_init(void)
        rcu_init_one(&rcu_sched_state, &rcu_sched_data);
        rcu_init_one(&rcu_bh_state, &rcu_bh_data);
        __rcu_init_preempt();
-       open_softirq(RCU_SOFTIRQ, rcu_process_callbacks);
 
        /*
         * We don't need protection against CPU-hotplug here because