rcu: Fix long-grace-period race between forcing and initialization
[safe/jmp/linux-2.6] / kernel / rcutree.c
index 0108570..f3077c0 100644 (file)
                NUM_RCU_LVL_2, \
                NUM_RCU_LVL_3, /* == MAX_RCU_LVLS */ \
        }, \
-       .signaled = RCU_SIGNAL_INIT, \
+       .signaled = RCU_GP_IDLE, \
        .gpnum = -300, \
        .completed = -300, \
        .onofflock = __SPIN_LOCK_UNLOCKED(&name.onofflock), \
+       .orphan_cbs_list = NULL, \
+       .orphan_cbs_tail = &name.orphan_cbs_list, \
+       .orphan_qlen = 0, \
        .fqslock = __SPIN_LOCK_UNLOCKED(&name.fqslock), \
        .n_force_qs = 0, \
        .n_force_qs_ngp = 0, \
@@ -654,14 +657,17 @@ rcu_start_gp(struct rcu_state *rsp, unsigned long flags)
         * irqs disabled.
         */
        rcu_for_each_node_breadth_first(rsp, rnp) {
-               spin_lock(&rnp->lock);  /* irqs already disabled. */
+               spin_lock(&rnp->lock);          /* irqs already disabled. */
                rcu_preempt_check_blocked_tasks(rnp);
                rnp->qsmask = rnp->qsmaskinit;
                rnp->gpnum = rsp->gpnum;
-               spin_unlock(&rnp->lock);        /* irqs already disabled. */
+               spin_unlock(&rnp->lock);        /* irqs remain disabled. */
        }
 
+       rnp = rcu_get_root(rsp);
+       spin_lock(&rnp->lock);                  /* irqs already disabled. */
        rsp->signaled = RCU_SIGNAL_INIT; /* force_quiescent_state now OK. */
+       spin_unlock(&rnp->lock);                /* irqs remain disabled. */
        spin_unlock_irqrestore(&rsp->onofflock, flags);
 }
 
@@ -703,6 +709,7 @@ static void cpu_quiet_msk_finish(struct rcu_state *rsp, unsigned long flags)
 {
        WARN_ON_ONCE(!rcu_gp_in_progress(rsp));
        rsp->completed = rsp->gpnum;
+       rsp->signaled = RCU_GP_IDLE;
        rcu_process_gp_end(rsp, rsp->rda[smp_processor_id()]);
        rcu_start_gp(rsp, flags);  /* releases root node's rnp->lock. */
 }
@@ -838,17 +845,63 @@ rcu_check_quiescent_state(struct rcu_state *rsp, struct rcu_data *rdp)
 #ifdef CONFIG_HOTPLUG_CPU
 
 /*
+ * Move a dying CPU's RCU callbacks to the ->orphan_cbs_list for the
+ * specified flavor of RCU.  The callbacks will be adopted by the next
+ * _rcu_barrier() invocation or by the CPU_DEAD notifier, whichever
+ * comes first.  Because this is invoked from the CPU_DYING notifier,
+ * irqs are already disabled.
+ */
+static void rcu_send_cbs_to_orphanage(struct rcu_state *rsp)
+{
+       int i;
+       struct rcu_data *rdp = rsp->rda[smp_processor_id()];
+
+       if (rdp->nxtlist == NULL)
+               return;  /* irqs disabled, so comparison is stable. */
+       spin_lock(&rsp->onofflock);  /* irqs already disabled. */
+       *rsp->orphan_cbs_tail = rdp->nxtlist;
+       rsp->orphan_cbs_tail = rdp->nxttail[RCU_NEXT_TAIL];
+       rdp->nxtlist = NULL;
+       for (i = 0; i < RCU_NEXT_SIZE; i++)
+               rdp->nxttail[i] = &rdp->nxtlist;
+       rsp->orphan_qlen += rdp->qlen;
+       rdp->qlen = 0;
+       spin_unlock(&rsp->onofflock);  /* irqs remain disabled. */
+}
+
+/*
+ * Adopt previously orphaned RCU callbacks.
+ */
+static void rcu_adopt_orphan_cbs(struct rcu_state *rsp)
+{
+       unsigned long flags;
+       struct rcu_data *rdp;
+
+       spin_lock_irqsave(&rsp->onofflock, flags);
+       rdp = rsp->rda[smp_processor_id()];
+       if (rsp->orphan_cbs_list == NULL) {
+               spin_unlock_irqrestore(&rsp->onofflock, flags);
+               return;
+       }
+       *rdp->nxttail[RCU_NEXT_TAIL] = rsp->orphan_cbs_list;
+       rdp->nxttail[RCU_NEXT_TAIL] = rsp->orphan_cbs_tail;
+       rdp->qlen += rsp->orphan_qlen;
+       rsp->orphan_cbs_list = NULL;
+       rsp->orphan_cbs_tail = &rsp->orphan_cbs_list;
+       rsp->orphan_qlen = 0;
+       spin_unlock_irqrestore(&rsp->onofflock, flags);
+}
+
+/*
  * Remove the outgoing CPU from the bitmasks in the rcu_node hierarchy
  * and move all callbacks from the outgoing CPU to the current one.
  */
 static void __rcu_offline_cpu(int cpu, struct rcu_state *rsp)
 {
-       int i;
        unsigned long flags;
        long lastcomp;
        unsigned long mask;
        struct rcu_data *rdp = rsp->rda[cpu];
-       struct rcu_data *rdp_me;
        struct rcu_node *rnp;
 
        /* Exclude any attempts to start a new grace period. */
@@ -864,39 +917,29 @@ static void __rcu_offline_cpu(int cpu, struct rcu_state *rsp)
                        spin_unlock(&rnp->lock); /* irqs remain disabled. */
                        break;
                }
-               rcu_preempt_offline_tasks(rsp, rnp, rdp);
+
+               /*
+                * If there was a task blocking the current grace period,
+                * and if all CPUs have checked in, we need to propagate
+                * the quiescent state up the rcu_node hierarchy.  But that
+                * is inconvenient at the moment due to deadlock issues if
+                * this should end the current grace period.  So set the
+                * offlined CPU's bit in ->qsmask in order to force the
+                * next force_quiescent_state() invocation to clean up this
+                * mess in a deadlock-free manner.
+                */
+               if (rcu_preempt_offline_tasks(rsp, rnp, rdp) && !rnp->qsmask)
+                       rnp->qsmask |= mask;
+
                mask = rnp->grpmask;
                spin_unlock(&rnp->lock);        /* irqs remain disabled. */
                rnp = rnp->parent;
        } while (rnp != NULL);
        lastcomp = rsp->completed;
 
-       spin_unlock(&rsp->onofflock);           /* irqs remain disabled. */
+       spin_unlock_irqrestore(&rsp->onofflock, flags);
 
-       /*
-        * Move callbacks from the outgoing CPU to the running CPU.
-        * Note that the outgoing CPU is now quiescent, so it is now
-        * (uncharacteristically) safe to access its rcu_data structure.
-        * Note also that we must carefully retain the order of the
-        * outgoing CPU's callbacks in order for rcu_barrier() to work
-        * correctly.  Finally, note that we start all the callbacks
-        * afresh, even those that have passed through a grace period
-        * and are therefore ready to invoke.  The theory is that hotplug
-        * events are rare, and that if they are frequent enough to
-        * indefinitely delay callbacks, you have far worse things to
-        * be worrying about.
-        */
-       if (rdp->nxtlist != NULL) {
-               rdp_me = rsp->rda[smp_processor_id()];
-               *rdp_me->nxttail[RCU_NEXT_TAIL] = rdp->nxtlist;
-               rdp_me->nxttail[RCU_NEXT_TAIL] = rdp->nxttail[RCU_NEXT_TAIL];
-               rdp->nxtlist = NULL;
-               for (i = 0; i < RCU_NEXT_SIZE; i++)
-                       rdp->nxttail[i] = &rdp->nxtlist;
-               rdp_me->qlen += rdp->qlen;
-               rdp->qlen = 0;
-       }
-       local_irq_restore(flags);
+       rcu_adopt_orphan_cbs(rsp);
 }
 
 /*
@@ -914,6 +957,14 @@ static void rcu_offline_cpu(int cpu)
 
 #else /* #ifdef CONFIG_HOTPLUG_CPU */
 
+static void rcu_send_cbs_to_orphanage(struct rcu_state *rsp)
+{
+}
+
+static void rcu_adopt_orphan_cbs(struct rcu_state *rsp)
+{
+}
+
 static void rcu_offline_cpu(int cpu)
 {
 }
@@ -924,7 +975,7 @@ static void rcu_offline_cpu(int cpu)
  * Invoke any RCU callbacks that have made it to the end of their grace
  * period.  Thottle as specified by rdp->blimit.
  */
-static void rcu_do_batch(struct rcu_data *rdp)
+static void rcu_do_batch(struct rcu_state *rsp, struct rcu_data *rdp)
 {
        unsigned long flags;
        struct rcu_head *next, *list, **tail;
@@ -977,6 +1028,13 @@ static void rcu_do_batch(struct rcu_data *rdp)
        if (rdp->blimit == LONG_MAX && rdp->qlen <= qlowmark)
                rdp->blimit = blimit;
 
+       /* Reset ->qlen_last_fqs_check trigger if enough CBs have drained. */
+       if (rdp->qlen == 0 && rdp->qlen_last_fqs_check != 0) {
+               rdp->qlen_last_fqs_check = 0;
+               rdp->n_force_qs_snap = rsp->n_force_qs;
+       } else if (rdp->qlen < rdp->qlen_last_fqs_check - qhimark)
+               rdp->qlen_last_fqs_check = rdp->qlen;
+
        local_irq_restore(flags);
 
        /* Re-raise the RCU softirq if there are callbacks remaining. */
@@ -1108,9 +1166,10 @@ static void force_quiescent_state(struct rcu_state *rsp, int relaxed)
        }
        spin_unlock(&rnp->lock);
        switch (signaled) {
+       case RCU_GP_IDLE:
        case RCU_GP_INIT:
 
-               break; /* grace period still initializing, ignore. */
+               break; /* grace period idle or initializing, ignore. */
 
        case RCU_SAVE_DYNTICK:
 
@@ -1124,7 +1183,8 @@ static void force_quiescent_state(struct rcu_state *rsp, int relaxed)
 
                /* Update state, record completion counter. */
                spin_lock(&rnp->lock);
-               if (lastcomp == rsp->completed) {
+               if (lastcomp == rsp->completed &&
+                   rsp->signaled == RCU_SAVE_DYNTICK) {
                        rsp->signaled = RCU_FORCE_QS;
                        dyntick_record_completed(rsp, lastcomp);
                }
@@ -1190,7 +1250,7 @@ __rcu_process_callbacks(struct rcu_state *rsp, struct rcu_data *rdp)
        }
 
        /* If there are callbacks ready, invoke them. */
-       rcu_do_batch(rdp);
+       rcu_do_batch(rsp, rdp);
 }
 
 /*
@@ -1254,10 +1314,20 @@ __call_rcu(struct rcu_head *head, void (*func)(struct rcu_head *rcu),
                rcu_start_gp(rsp, nestflag);  /* releases rnp_root->lock. */
        }
 
-       /* Force the grace period if too many callbacks or too long waiting. */
-       if (unlikely(++rdp->qlen > qhimark)) {
+       /*
+        * Force the grace period if too many callbacks or too long waiting.
+        * Enforce hysteresis, and don't invoke force_quiescent_state()
+        * if some other CPU has recently done so.  Also, don't bother
+        * 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)) {
                rdp->blimit = LONG_MAX;
-               force_quiescent_state(rsp, 0);
+               if (rsp->n_force_qs == rdp->n_force_qs_snap &&
+                   *rdp->nxttail[RCU_DONE_TAIL] != head)
+                       force_quiescent_state(rsp, 0);
+               rdp->n_force_qs_snap = rsp->n_force_qs;
+               rdp->qlen_last_fqs_check = rdp->qlen;
        } else if ((long)(ACCESS_ONCE(rsp->jiffies_force_qs) - jiffies) < 0)
                force_quiescent_state(rsp, 1);
        local_irq_restore(flags);
@@ -1367,9 +1437,6 @@ static DEFINE_PER_CPU(struct rcu_head, rcu_barrier_head) = {NULL};
 static atomic_t rcu_barrier_cpu_count;
 static DEFINE_MUTEX(rcu_barrier_mutex);
 static struct completion rcu_barrier_completion;
-static atomic_t rcu_migrate_type_count = ATOMIC_INIT(0);
-static struct rcu_head rcu_migrate_head[3];
-static DECLARE_WAIT_QUEUE_HEAD(rcu_migrate_wq);
 
 static void rcu_barrier_callback(struct rcu_head *notused)
 {
@@ -1392,21 +1459,16 @@ static void rcu_barrier_func(void *type)
        call_rcu_func(head, rcu_barrier_callback);
 }
 
-static inline void wait_migrated_callbacks(void)
-{
-       wait_event(rcu_migrate_wq, !atomic_read(&rcu_migrate_type_count));
-       smp_mb(); /* In case we didn't sleep. */
-}
-
 /*
  * Orchestrate the specified type of RCU barrier, waiting for all
  * RCU callbacks of the specified type to complete.
  */
-static void _rcu_barrier(void (*call_rcu_func)(struct rcu_head *head,
+static void _rcu_barrier(struct rcu_state *rsp,
+                        void (*call_rcu_func)(struct rcu_head *head,
                                               void (*func)(struct rcu_head *head)))
 {
        BUG_ON(in_interrupt());
-       /* Take cpucontrol mutex to protect against CPU hotplug */
+       /* Take mutex to serialize concurrent rcu_barrier() requests. */
        mutex_lock(&rcu_barrier_mutex);
        init_completion(&rcu_barrier_completion);
        /*
@@ -1419,29 +1481,22 @@ static void _rcu_barrier(void (*call_rcu_func)(struct rcu_head *head,
         * early.
         */
        atomic_set(&rcu_barrier_cpu_count, 1);
+       preempt_disable(); /* stop CPU_DYING from filling orphan_cbs_list */
+       rcu_adopt_orphan_cbs(rsp);
        on_each_cpu(rcu_barrier_func, (void *)call_rcu_func, 1);
+       preempt_enable(); /* CPU_DYING can again fill orphan_cbs_list */
        if (atomic_dec_and_test(&rcu_barrier_cpu_count))
                complete(&rcu_barrier_completion);
        wait_for_completion(&rcu_barrier_completion);
        mutex_unlock(&rcu_barrier_mutex);
-       wait_migrated_callbacks();
 }
 
 /**
- * rcu_barrier - Wait until all in-flight call_rcu() callbacks complete.
- */
-void rcu_barrier(void)
-{
-       _rcu_barrier(call_rcu);
-}
-EXPORT_SYMBOL_GPL(rcu_barrier);
-
-/**
  * rcu_barrier_bh - Wait until all in-flight call_rcu_bh() callbacks complete.
  */
 void rcu_barrier_bh(void)
 {
-       _rcu_barrier(call_rcu_bh);
+       _rcu_barrier(&rcu_bh_state, call_rcu_bh);
 }
 EXPORT_SYMBOL_GPL(rcu_barrier_bh);
 
@@ -1450,16 +1505,10 @@ EXPORT_SYMBOL_GPL(rcu_barrier_bh);
  */
 void rcu_barrier_sched(void)
 {
-       _rcu_barrier(call_rcu_sched);
+       _rcu_barrier(&rcu_sched_state, call_rcu_sched);
 }
 EXPORT_SYMBOL_GPL(rcu_barrier_sched);
 
-static void rcu_migrate_callback(struct rcu_head *notused)
-{
-       if (atomic_dec_and_test(&rcu_migrate_type_count))
-               wake_up(&rcu_migrate_wq);
-}
-
 /*
  * Do boot-time initialization of a CPU's per-CPU RCU data.
  */
@@ -1510,6 +1559,8 @@ rcu_init_percpu_data(int cpu, struct rcu_state *rsp, int preemptable)
        rdp->beenonline = 1;     /* We have now been online. */
        rdp->preemptable = preemptable;
        rdp->passed_quiesc_completed = lastcomp - 1;
+       rdp->qlen_last_fqs_check = 0;
+       rdp->n_force_qs_snap = rsp->n_force_qs;
        rdp->blimit = blimit;
        spin_unlock(&rnp->lock);                /* irqs remain disabled. */
 
@@ -1556,27 +1607,21 @@ int __cpuinit rcu_cpu_notify(struct notifier_block *self,
        case CPU_UP_PREPARE_FROZEN:
                rcu_online_cpu(cpu);
                break;
-       case CPU_DOWN_PREPARE:
-       case CPU_DOWN_PREPARE_FROZEN:
-               /* Don't need to wait until next removal operation. */
-               /* rcu_migrate_head is protected by cpu_add_remove_lock */
-               wait_migrated_callbacks();
-               break;
        case CPU_DYING:
        case CPU_DYING_FROZEN:
                /*
-                * preempt_disable() in on_each_cpu() prevents stop_machine(),
+                * preempt_disable() in _rcu_barrier() prevents stop_machine(),
                 * so when "on_each_cpu(rcu_barrier_func, (void *)type, 1);"
-                * returns, all online cpus have queued rcu_barrier_func(),
-                * and the dead cpu(if it exist) queues rcu_migrate_callback()s.
-                *
-                * These callbacks ensure _rcu_barrier() waits for all
-                * RCU callbacks of the specified type to complete.
+                * returns, all online cpus have queued rcu_barrier_func().
+                * The dying CPU clears its cpu_online_mask bit and
+                * moves all of its RCU callbacks to ->orphan_cbs_list
+                * in the context of stop_machine(), so subsequent calls
+                * to _rcu_barrier() will adopt these callbacks and only
+                * then queue rcu_barrier_func() on all remaining CPUs.
                 */
-               atomic_set(&rcu_migrate_type_count, 3);
-               call_rcu_bh(rcu_migrate_head, rcu_migrate_callback);
-               call_rcu_sched(rcu_migrate_head + 1, rcu_migrate_callback);
-               call_rcu(rcu_migrate_head + 2, rcu_migrate_callback);
+               rcu_send_cbs_to_orphanage(&rcu_bh_state);
+               rcu_send_cbs_to_orphanage(&rcu_sched_state);
+               rcu_preempt_send_cbs_to_orphanage();
                break;
        case CPU_DEAD:
        case CPU_DEAD_FROZEN:
@@ -1640,7 +1685,8 @@ static void __init rcu_init_one(struct rcu_state *rsp)
                cpustride *= rsp->levelspread[i];
                rnp = rsp->level[i];
                for (j = 0; j < rsp->levelcnt[i]; j++, rnp++) {
-                       spin_lock_init(&rnp->lock);
+                       if (rnp != rcu_get_root(rsp))
+                               spin_lock_init(&rnp->lock);
                        rnp->gpnum = 0;
                        rnp->qsmask = 0;
                        rnp->qsmaskinit = 0;
@@ -1663,6 +1709,7 @@ static void __init rcu_init_one(struct rcu_state *rsp)
                        INIT_LIST_HEAD(&rnp->blocked_tasks[1]);
                }
        }
+       spin_lock_init(&rcu_get_root(rsp)->lock);
 }
 
 /*