summaryrefslogtreecommitdiff
path: root/kernel/rcu/tasks.h
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/rcu/tasks.h')
-rw-r--r--kernel/rcu/tasks.h541
1 files changed, 336 insertions, 205 deletions
diff --git a/kernel/rcu/tasks.h b/kernel/rcu/tasks.h
index 3925e32159b5..83c7e6620d40 100644
--- a/kernel/rcu/tasks.h
+++ b/kernel/rcu/tasks.h
@@ -14,7 +14,7 @@
struct rcu_tasks;
typedef void (*rcu_tasks_gp_func_t)(struct rcu_tasks *rtp);
-typedef void (*pregp_func_t)(void);
+typedef void (*pregp_func_t)(struct list_head *hop);
typedef void (*pertask_func_t)(struct task_struct *t, struct list_head *hop);
typedef void (*postscan_func_t)(struct list_head *hop);
typedef void (*holdouts_func_t)(struct list_head *hop, bool ndrpt, bool *frptp);
@@ -29,6 +29,7 @@ typedef void (*postgp_func_t)(struct rcu_tasks *rtp);
* @rtp_work: Work queue for invoking callbacks.
* @rtp_irq_work: IRQ work queue for deferred wakeups.
* @barrier_q_head: RCU callback for barrier operation.
+ * @rtp_blkd_tasks: List of tasks blocked as readers.
* @cpu: CPU number corresponding to this entry.
* @rtpp: Pointer to the rcu_tasks structure.
*/
@@ -40,6 +41,7 @@ struct rcu_tasks_percpu {
struct work_struct rtp_work;
struct irq_work rtp_irq_work;
struct rcu_head barrier_q_head;
+ struct list_head rtp_blkd_tasks;
int cpu;
struct rcu_tasks *rtpp;
};
@@ -48,6 +50,7 @@ struct rcu_tasks_percpu {
* struct rcu_tasks - Definition for a Tasks-RCU-like mechanism.
* @cbs_wait: RCU wait allowing a new callback to get kthread's attention.
* @cbs_gbl_lock: Lock protecting callback list.
+ * @tasks_gp_mutex: Mutex protecting grace period, needed during mid-boot dead zone.
* @kthread_ptr: This flavor's grace-period/callback-invocation kthread.
* @gp_func: This flavor's grace-period-wait function.
* @gp_state: Grace period's most recent state transition (debugging).
@@ -79,6 +82,7 @@ struct rcu_tasks_percpu {
struct rcu_tasks {
struct rcuwait cbs_wait;
raw_spinlock_t cbs_gbl_lock;
+ struct mutex tasks_gp_mutex;
int gp_state;
int gp_sleep;
int init_fract;
@@ -119,6 +123,7 @@ static struct rcu_tasks rt_name = \
{ \
.cbs_wait = __RCUWAIT_INITIALIZER(rt_name.wait), \
.cbs_gbl_lock = __RAW_SPIN_LOCK_UNLOCKED(rt_name.cbs_gbl_lock), \
+ .tasks_gp_mutex = __MUTEX_INITIALIZER(rt_name.tasks_gp_mutex), \
.gp_func = gp, \
.call_func = call, \
.rtpcpu = &rt_name ## __percpu, \
@@ -140,6 +145,7 @@ static int rcu_task_ipi_delay __read_mostly = RCU_TASK_IPI_DELAY;
module_param(rcu_task_ipi_delay, int, 0644);
/* Control stall timeouts. Disable with <= 0, otherwise jiffies till stall. */
+#define RCU_TASK_BOOT_STALL_TIMEOUT (HZ * 30)
#define RCU_TASK_STALL_TIMEOUT (HZ * 60 * 10)
static int rcu_task_stall_timeout __read_mostly = RCU_TASK_STALL_TIMEOUT;
module_param(rcu_task_stall_timeout, int, 0644);
@@ -253,6 +259,8 @@ static void cblist_init_generic(struct rcu_tasks *rtp)
INIT_WORK(&rtpcp->rtp_work, rcu_tasks_invoke_cbs_wq);
rtpcp->cpu = cpu;
rtpcp->rtpp = rtp;
+ if (!rtpcp->rtp_blkd_tasks.next)
+ INIT_LIST_HEAD(&rtpcp->rtp_blkd_tasks);
raw_spin_unlock_rcu_node(rtpcp); // irqs remain disabled.
}
raw_spin_unlock_irqrestore(&rtp->cbs_gbl_lock, flags);
@@ -323,17 +331,6 @@ static void call_rcu_tasks_generic(struct rcu_head *rhp, rcu_callback_t func,
irq_work_queue(&rtpcp->rtp_irq_work);
}
-// Wait for a grace period for the specified flavor of Tasks RCU.
-static void synchronize_rcu_tasks_generic(struct rcu_tasks *rtp)
-{
- /* Complain if the scheduler has not started. */
- RCU_LOCKDEP_WARN(rcu_scheduler_active == RCU_SCHEDULER_INACTIVE,
- "synchronize_rcu_tasks called too soon");
-
- /* Wait for the grace period. */
- wait_rcu_gp(rtp->call_func);
-}
-
// RCU callback function for rcu_barrier_tasks_generic().
static void rcu_barrier_tasks_generic_cb(struct rcu_head *rhp)
{
@@ -439,6 +436,11 @@ static int rcu_tasks_need_gpcb(struct rcu_tasks *rtp)
WRITE_ONCE(rtp->percpu_dequeue_lim, 1);
pr_info("Completing switch %s to CPU-0 callback queuing.\n", rtp->name);
}
+ for (cpu = rtp->percpu_dequeue_lim; cpu < nr_cpu_ids; cpu++) {
+ struct rcu_tasks_percpu *rtpcp = per_cpu_ptr(rtp->rtpcpu, cpu);
+
+ WARN_ON_ONCE(rcu_segcblist_n_cbs(&rtpcp->cblist));
+ }
raw_spin_unlock_irqrestore(&rtp->cbs_gbl_lock, flags);
}
@@ -497,10 +499,41 @@ static void rcu_tasks_invoke_cbs_wq(struct work_struct *wp)
rcu_tasks_invoke_cbs(rtp, rtpcp);
}
-/* RCU-tasks kthread that detects grace periods and invokes callbacks. */
-static int __noreturn rcu_tasks_kthread(void *arg)
+// Wait for one grace period.
+static void rcu_tasks_one_gp(struct rcu_tasks *rtp, bool midboot)
{
int needgpcb;
+
+ mutex_lock(&rtp->tasks_gp_mutex);
+
+ // If there were none, wait a bit and start over.
+ if (unlikely(midboot)) {
+ needgpcb = 0x2;
+ } else {
+ set_tasks_gp_state(rtp, RTGS_WAIT_CBS);
+ rcuwait_wait_event(&rtp->cbs_wait,
+ (needgpcb = rcu_tasks_need_gpcb(rtp)),
+ TASK_IDLE);
+ }
+
+ if (needgpcb & 0x2) {
+ // Wait for one grace period.
+ set_tasks_gp_state(rtp, RTGS_WAIT_GP);
+ rtp->gp_start = jiffies;
+ rcu_seq_start(&rtp->tasks_gp_seq);
+ rtp->gp_func(rtp);
+ rcu_seq_end(&rtp->tasks_gp_seq);
+ }
+
+ // Invoke callbacks.
+ set_tasks_gp_state(rtp, RTGS_INVOKE_CBS);
+ rcu_tasks_invoke_cbs(rtp, per_cpu_ptr(rtp->rtpcpu, 0));
+ mutex_unlock(&rtp->tasks_gp_mutex);
+}
+
+// RCU-tasks kthread that detects grace periods and invokes callbacks.
+static int __noreturn rcu_tasks_kthread(void *arg)
+{
struct rcu_tasks *rtp = arg;
/* Run on housekeeping CPUs by default. Sysadm can move if desired. */
@@ -514,29 +547,28 @@ static int __noreturn rcu_tasks_kthread(void *arg)
* This loop is terminated by the system going down. ;-)
*/
for (;;) {
- set_tasks_gp_state(rtp, RTGS_WAIT_CBS);
+ // Wait for one grace period and invoke any callbacks
+ // that are ready.
+ rcu_tasks_one_gp(rtp, false);
- /* If there were none, wait a bit and start over. */
- rcuwait_wait_event(&rtp->cbs_wait,
- (needgpcb = rcu_tasks_need_gpcb(rtp)),
- TASK_IDLE);
-
- if (needgpcb & 0x2) {
- // Wait for one grace period.
- set_tasks_gp_state(rtp, RTGS_WAIT_GP);
- rtp->gp_start = jiffies;
- rcu_seq_start(&rtp->tasks_gp_seq);
- rtp->gp_func(rtp);
- rcu_seq_end(&rtp->tasks_gp_seq);
- }
+ // Paranoid sleep to keep this from entering a tight loop.
+ schedule_timeout_idle(rtp->gp_sleep);
+ }
+}
- /* Invoke callbacks. */
- set_tasks_gp_state(rtp, RTGS_INVOKE_CBS);
- rcu_tasks_invoke_cbs(rtp, per_cpu_ptr(rtp->rtpcpu, 0));
+// Wait for a grace period for the specified flavor of Tasks RCU.
+static void synchronize_rcu_tasks_generic(struct rcu_tasks *rtp)
+{
+ /* Complain if the scheduler has not started. */
+ RCU_LOCKDEP_WARN(rcu_scheduler_active == RCU_SCHEDULER_INACTIVE,
+ "synchronize_rcu_tasks called too soon");
- /* Paranoid sleep to keep this from entering a tight loop */
- schedule_timeout_idle(rtp->gp_sleep);
+ // If the grace-period kthread is running, use it.
+ if (READ_ONCE(rtp->kthread_ptr)) {
+ wait_rcu_gp(rtp->call_func);
+ return;
}
+ rcu_tasks_one_gp(rtp, true);
}
/* Spawn RCU-tasks grace-period kthread. */
@@ -630,7 +662,7 @@ static void rcu_tasks_wait_gp(struct rcu_tasks *rtp)
struct task_struct *t;
set_tasks_gp_state(rtp, RTGS_PRE_WAIT_GP);
- rtp->pregp_func();
+ rtp->pregp_func(&holdouts);
/*
* There were callbacks, so we need to wait for an RCU-tasks
@@ -639,10 +671,12 @@ static void rcu_tasks_wait_gp(struct rcu_tasks *rtp)
* and make a list of them in holdouts.
*/
set_tasks_gp_state(rtp, RTGS_SCAN_TASKLIST);
- rcu_read_lock();
- for_each_process_thread(g, t)
- rtp->pertask_func(t, &holdouts);
- rcu_read_unlock();
+ if (rtp->pertask_func) {
+ rcu_read_lock();
+ for_each_process_thread(g, t)
+ rtp->pertask_func(t, &holdouts);
+ rcu_read_unlock();
+ }
set_tasks_gp_state(rtp, RTGS_POST_SCAN_TASKLIST);
rtp->postscan_func(&holdouts);
@@ -760,7 +794,7 @@ static void rcu_tasks_wait_gp(struct rcu_tasks *rtp)
// disabling.
/* Pre-grace-period preparation. */
-static void rcu_tasks_pregp_step(void)
+static void rcu_tasks_pregp_step(struct list_head *hop)
{
/*
* Wait for all pre-existing t->on_rq and t->nvcsw transitions
@@ -1105,11 +1139,10 @@ EXPORT_SYMBOL_GPL(show_rcu_tasks_rude_gp_kthread);
// 3. Avoids expensive read-side instructions, having overhead similar
// to that of Preemptible RCU.
//
-// There are of course downsides. The grace-period code can send IPIs to
-// CPUs, even when those CPUs are in the idle loop or in nohz_full userspace.
-// It is necessary to scan the full tasklist, much as for Tasks RCU. There
-// is a single callback queue guarded by a single lock, again, much as for
-// Tasks RCU. If needed, these downsides can be at least partially remedied.
+// There are of course downsides. For example, the grace-period code
+// can send IPIs to CPUs, even when those CPUs are in the idle loop or
+// in nohz_full userspace. If needed, these downsides can be at least
+// partially remedied.
//
// Perhaps most important, this variant of RCU does not affect the vanilla
// flavors, rcu_preempt and rcu_sched. The fact that RCU Tasks Trace
@@ -1122,38 +1155,30 @@ EXPORT_SYMBOL_GPL(show_rcu_tasks_rude_gp_kthread);
// invokes these functions in this order:
//
// rcu_tasks_trace_pregp_step():
-// Initialize the count of readers and block CPU-hotplug operations.
-// rcu_tasks_trace_pertask(), invoked on every non-idle task:
-// Initialize per-task state and attempt to identify an immediate
-// quiescent state for that task, or, failing that, attempt to
-// set that task's .need_qs flag so that task's next outermost
-// rcu_read_unlock_trace() will report the quiescent state (in which
-// case the count of readers is incremented). If both attempts fail,
-// the task is added to a "holdout" list. Note that IPIs are used
-// to invoke trc_read_check_handler() in the context of running tasks
-// in order to avoid ordering overhead on common-case shared-variable
-// accessses.
+// Disables CPU hotplug, adds all currently executing tasks to the
+// holdout list, then checks the state of all tasks that blocked
+// or were preempted within their current RCU Tasks Trace read-side
+// critical section, adding them to the holdout list if appropriate.
+// Finally, this function re-enables CPU hotplug.
+// The ->pertask_func() pointer is NULL, so there is no per-task processing.
// rcu_tasks_trace_postscan():
-// Initialize state and attempt to identify an immediate quiescent
-// state as above (but only for idle tasks), unblock CPU-hotplug
-// operations, and wait for an RCU grace period to avoid races with
-// tasks that are in the process of exiting.
+// Invokes synchronize_rcu() to wait for late-stage exiting tasks
+// to finish exiting.
// check_all_holdout_tasks_trace(), repeatedly until holdout list is empty:
// Scans the holdout list, attempting to identify a quiescent state
// for each task on the list. If there is a quiescent state, the
-// corresponding task is removed from the holdout list.
+// corresponding task is removed from the holdout list. Once this
+// list is empty, the grace period has completed.
// rcu_tasks_trace_postgp():
-// Wait for the count of readers do drop to zero, reporting any stalls.
-// Also execute full memory barriers to maintain ordering with code
-// executing after the grace period.
+// Provides the needed full memory barrier and does debug checks.
//
// The exit_tasks_rcu_finish_trace() synchronizes with exiting tasks.
//
-// Pre-grace-period update-side code is ordered before the grace
-// period via the ->cbs_lock and barriers in rcu_tasks_kthread().
-// Pre-grace-period read-side code is ordered before the grace period by
-// atomic_dec_and_test() of the count of readers (for IPIed readers) and by
-// scheduler context-switch ordering (for locked-down non-running readers).
+// Pre-grace-period update-side code is ordered before the grace period
+// via the ->cbs_lock and barriers in rcu_tasks_kthread(). Pre-grace-period
+// read-side code is ordered before the grace period by atomic operations
+// on .b.need_qs flag of each task involved in this process, or by scheduler
+// context-switch ordering (for locked-down non-running readers).
// The lockdep state must be outside of #ifdef to be useful.
#ifdef CONFIG_DEBUG_LOCK_ALLOC
@@ -1165,9 +1190,6 @@ EXPORT_SYMBOL_GPL(rcu_trace_lock_map);
#ifdef CONFIG_TASKS_TRACE_RCU
-static atomic_t trc_n_readers_need_end; // Number of waited-for readers.
-static DECLARE_WAIT_QUEUE_HEAD(trc_wait); // List of holdout tasks.
-
// Record outstanding IPIs to each CPU. No point in sending two...
static DEFINE_PER_CPU(bool, trc_ipi_to_cpu);
@@ -1176,44 +1198,104 @@ static DEFINE_PER_CPU(bool, trc_ipi_to_cpu);
static unsigned long n_heavy_reader_attempts;
static unsigned long n_heavy_reader_updates;
static unsigned long n_heavy_reader_ofl_updates;
+static unsigned long n_trc_holdouts;
void call_rcu_tasks_trace(struct rcu_head *rhp, rcu_callback_t func);
DEFINE_RCU_TASKS(rcu_tasks_trace, rcu_tasks_wait_gp, call_rcu_tasks_trace,
"RCU Tasks Trace");
+/* Load from ->trc_reader_special.b.need_qs with proper ordering. */
+static u8 rcu_ld_need_qs(struct task_struct *t)
+{
+ smp_mb(); // Enforce full grace-period ordering.
+ return smp_load_acquire(&t->trc_reader_special.b.need_qs);
+}
+
+/* Store to ->trc_reader_special.b.need_qs with proper ordering. */
+static void rcu_st_need_qs(struct task_struct *t, u8 v)
+{
+ smp_store_release(&t->trc_reader_special.b.need_qs, v);
+ smp_mb(); // Enforce full grace-period ordering.
+}
+
/*
- * This irq_work handler allows rcu_read_unlock_trace() to be invoked
- * while the scheduler locks are held.
+ * Do a cmpxchg() on ->trc_reader_special.b.need_qs, allowing for
+ * the four-byte operand-size restriction of some platforms.
+ * Returns the old value, which is often ignored.
*/
-static void rcu_read_unlock_iw(struct irq_work *iwp)
+u8 rcu_trc_cmpxchg_need_qs(struct task_struct *t, u8 old, u8 new)
{
- wake_up(&trc_wait);
+ union rcu_special ret;
+ union rcu_special trs_old = READ_ONCE(t->trc_reader_special);
+ union rcu_special trs_new = trs_old;
+
+ if (trs_old.b.need_qs != old)
+ return trs_old.b.need_qs;
+ trs_new.b.need_qs = new;
+ ret.s = cmpxchg(&t->trc_reader_special.s, trs_old.s, trs_new.s);
+ return ret.b.need_qs;
}
-static DEFINE_IRQ_WORK(rcu_tasks_trace_iw, rcu_read_unlock_iw);
+EXPORT_SYMBOL_GPL(rcu_trc_cmpxchg_need_qs);
-/* If we are the last reader, wake up the grace-period kthread. */
+/*
+ * If we are the last reader, signal the grace-period kthread.
+ * Also remove from the per-CPU list of blocked tasks.
+ */
void rcu_read_unlock_trace_special(struct task_struct *t)
{
- int nq = READ_ONCE(t->trc_reader_special.b.need_qs);
+ unsigned long flags;
+ struct rcu_tasks_percpu *rtpcp;
+ union rcu_special trs;
+
+ // Open-coded full-word version of rcu_ld_need_qs().
+ smp_mb(); // Enforce full grace-period ordering.
+ trs = smp_load_acquire(&t->trc_reader_special);
- if (IS_ENABLED(CONFIG_TASKS_TRACE_RCU_READ_MB) &&
- t->trc_reader_special.b.need_mb)
+ if (IS_ENABLED(CONFIG_TASKS_TRACE_RCU_READ_MB) && t->trc_reader_special.b.need_mb)
smp_mb(); // Pairs with update-side barriers.
// Update .need_qs before ->trc_reader_nesting for irq/NMI handlers.
- if (nq)
- WRITE_ONCE(t->trc_reader_special.b.need_qs, false);
+ if (trs.b.need_qs == (TRC_NEED_QS_CHECKED | TRC_NEED_QS)) {
+ u8 result = rcu_trc_cmpxchg_need_qs(t, TRC_NEED_QS_CHECKED | TRC_NEED_QS,
+ TRC_NEED_QS_CHECKED);
+
+ WARN_ONCE(result != trs.b.need_qs, "%s: result = %d", __func__, result);
+ }
+ if (trs.b.blocked) {
+ rtpcp = per_cpu_ptr(rcu_tasks_trace.rtpcpu, t->trc_blkd_cpu);
+ raw_spin_lock_irqsave_rcu_node(rtpcp, flags);
+ list_del_init(&t->trc_blkd_node);
+ WRITE_ONCE(t->trc_reader_special.b.blocked, false);
+ raw_spin_unlock_irqrestore_rcu_node(rtpcp, flags);
+ }
WRITE_ONCE(t->trc_reader_nesting, 0);
- if (nq && atomic_dec_and_test(&trc_n_readers_need_end))
- irq_work_queue(&rcu_tasks_trace_iw);
}
EXPORT_SYMBOL_GPL(rcu_read_unlock_trace_special);
+/* Add a newly blocked reader task to its CPU's list. */
+void rcu_tasks_trace_qs_blkd(struct task_struct *t)
+{
+ unsigned long flags;
+ struct rcu_tasks_percpu *rtpcp;
+
+ local_irq_save(flags);
+ rtpcp = this_cpu_ptr(rcu_tasks_trace.rtpcpu);
+ raw_spin_lock_rcu_node(rtpcp); // irqs already disabled
+ t->trc_blkd_cpu = smp_processor_id();
+ if (!rtpcp->rtp_blkd_tasks.next)
+ INIT_LIST_HEAD(&rtpcp->rtp_blkd_tasks);
+ list_add(&t->trc_blkd_node, &rtpcp->rtp_blkd_tasks);
+ WRITE_ONCE(t->trc_reader_special.b.blocked, true);
+ raw_spin_unlock_irqrestore_rcu_node(rtpcp, flags);
+}
+EXPORT_SYMBOL_GPL(rcu_tasks_trace_qs_blkd);
+
/* Add a task to the holdout list, if it is not already on the list. */
static void trc_add_holdout(struct task_struct *t, struct list_head *bhp)
{
if (list_empty(&t->trc_holdout_list)) {
get_task_struct(t);
list_add(&t->trc_holdout_list, bhp);
+ n_trc_holdouts++;
}
}
@@ -1223,37 +1305,36 @@ static void trc_del_holdout(struct task_struct *t)
if (!list_empty(&t->trc_holdout_list)) {
list_del_init(&t->trc_holdout_list);
put_task_struct(t);
+ n_trc_holdouts--;
}
}
/* IPI handler to check task state. */
static void trc_read_check_handler(void *t_in)
{
+ int nesting;
struct task_struct *t = current;
struct task_struct *texp = t_in;
// If the task is no longer running on this CPU, leave.
- if (unlikely(texp != t)) {
+ if (unlikely(texp != t))
goto reset_ipi; // Already on holdout list, so will check later.
- }
// If the task is not in a read-side critical section, and
// if this is the last reader, awaken the grace-period kthread.
- if (likely(!READ_ONCE(t->trc_reader_nesting))) {
- WRITE_ONCE(t->trc_reader_checked, true);
+ nesting = READ_ONCE(t->trc_reader_nesting);
+ if (likely(!nesting)) {
+ rcu_trc_cmpxchg_need_qs(t, 0, TRC_NEED_QS_CHECKED);
goto reset_ipi;
}
// If we are racing with an rcu_read_unlock_trace(), try again later.
- if (unlikely(READ_ONCE(t->trc_reader_nesting) < 0))
+ if (unlikely(nesting < 0))
goto reset_ipi;
- WRITE_ONCE(t->trc_reader_checked, true);
- // Get here if the task is in a read-side critical section. Set
- // its state so that it will awaken the grace-period kthread upon
- // exit from that critical section.
- atomic_inc(&trc_n_readers_need_end); // One more to wait on.
- WARN_ON_ONCE(READ_ONCE(t->trc_reader_special.b.need_qs));
- WRITE_ONCE(t->trc_reader_special.b.need_qs, true);
+ // Get here if the task is in a read-side critical section.
+ // Set its state so that it will update state for the grace-period
+ // kthread upon exit from that critical section.
+ rcu_trc_cmpxchg_need_qs(t, 0, TRC_NEED_QS | TRC_NEED_QS_CHECKED);
reset_ipi:
// Allow future IPIs to be sent on CPU and for task.
@@ -1264,48 +1345,50 @@ reset_ipi:
}
/* Callback function for scheduler to check locked-down task. */
-static int trc_inspect_reader(struct task_struct *t, void *arg)
+static int trc_inspect_reader(struct task_struct *t, void *bhp_in)
{
+ struct list_head *bhp = bhp_in;
int cpu = task_cpu(t);
int nesting;
bool ofl = cpu_is_offline(cpu);
- if (task_curr(t)) {
- WARN_ON_ONCE(ofl && !is_idle_task(t));
-
+ if (task_curr(t) && !ofl) {
// If no chance of heavyweight readers, do it the hard way.
- if (!ofl && !IS_ENABLED(CONFIG_TASKS_TRACE_RCU_READ_MB))
+ if (!IS_ENABLED(CONFIG_TASKS_TRACE_RCU_READ_MB))
return -EINVAL;
// If heavyweight readers are enabled on the remote task,
// we can inspect its state despite its currently running.
// However, we cannot safely change its state.
n_heavy_reader_attempts++;
- if (!ofl && // Check for "running" idle tasks on offline CPUs.
- !rcu_dynticks_zero_in_eqs(cpu, &t->trc_reader_nesting))
+ // Check for "running" idle tasks on offline CPUs.
+ if (!rcu_dynticks_zero_in_eqs(cpu, &t->trc_reader_nesting))
return -EINVAL; // No quiescent state, do it the hard way.
n_heavy_reader_updates++;
- if (ofl)
- n_heavy_reader_ofl_updates++;
nesting = 0;
} else {
// The task is not running, so C-language access is safe.
nesting = t->trc_reader_nesting;
+ WARN_ON_ONCE(ofl && task_curr(t) && !is_idle_task(t));
+ if (IS_ENABLED(CONFIG_TASKS_TRACE_RCU_READ_MB) && ofl)
+ n_heavy_reader_ofl_updates++;
}
// If not exiting a read-side critical section, mark as checked
// so that the grace-period kthread will remove it from the
// holdout list.
- t->trc_reader_checked = nesting >= 0;
- if (nesting <= 0)
- return nesting ? -EINVAL : 0; // If in QS, done, otherwise try again later.
+ if (!nesting) {
+ rcu_trc_cmpxchg_need_qs(t, 0, TRC_NEED_QS_CHECKED);
+ return 0; // In QS, so done.
+ }
+ if (nesting < 0)
+ return -EINVAL; // Reader transitioning, try again later.
// The task is in a read-side critical section, so set up its
- // state so that it will awaken the grace-period kthread upon exit
- // from that critical section.
- atomic_inc(&trc_n_readers_need_end); // One more to wait on.
- WARN_ON_ONCE(READ_ONCE(t->trc_reader_special.b.need_qs));
- WRITE_ONCE(t->trc_reader_special.b.need_qs, true);
+ // state so that it will update state upon exit from that critical
+ // section.
+ if (!rcu_trc_cmpxchg_need_qs(t, 0, TRC_NEED_QS | TRC_NEED_QS_CHECKED))
+ trc_add_holdout(t, bhp);
return 0;
}
@@ -1321,14 +1404,14 @@ static void trc_wait_for_one_reader(struct task_struct *t,
// The current task had better be in a quiescent state.
if (t == current) {
- t->trc_reader_checked = true;
+ rcu_trc_cmpxchg_need_qs(t, 0, TRC_NEED_QS_CHECKED);
WARN_ON_ONCE(READ_ONCE(t->trc_reader_nesting));
return;
}
// Attempt to nail down the task for inspection.
get_task_struct(t);
- if (!task_call_func(t, trc_inspect_reader, NULL)) {
+ if (!task_call_func(t, trc_inspect_reader, bhp)) {
put_task_struct(t);
return;
}
@@ -1366,56 +1449,93 @@ static void trc_wait_for_one_reader(struct task_struct *t,
}
}
+/*
+ * Initialize for first-round processing for the specified task.
+ * Return false if task is NULL or already taken care of, true otherwise.
+ */
+static bool rcu_tasks_trace_pertask_prep(struct task_struct *t, bool notself)
+{
+ // During early boot when there is only the one boot CPU, there
+ // is no idle task for the other CPUs. Also, the grace-period
+ // kthread is always in a quiescent state. In addition, just return
+ // if this task is already on the list.
+ if (unlikely(t == NULL) || (t == current && notself) || !list_empty(&t->trc_holdout_list))
+ return false;
+
+ rcu_st_need_qs(t, 0);
+ t->trc_ipi_to_cpu = -1;
+ return true;
+}
+
+/* Do first-round processing for the specified task. */
+static void rcu_tasks_trace_pertask(struct task_struct *t, struct list_head *hop)
+{
+ if (rcu_tasks_trace_pertask_prep(t, true))
+ trc_wait_for_one_reader(t, hop);
+}
+
/* Initialize for a new RCU-tasks-trace grace period. */
-static void rcu_tasks_trace_pregp_step(void)
+static void rcu_tasks_trace_pregp_step(struct list_head *hop)
{
+ LIST_HEAD(blkd_tasks);
int cpu;
-
- // Allow for fast-acting IPIs.
- atomic_set(&trc_n_readers_need_end, 1);
+ unsigned long flags;
+ struct rcu_tasks_percpu *rtpcp;
+ struct task_struct *t;
// There shouldn't be any old IPIs, but...
for_each_possible_cpu(cpu)
WARN_ON_ONCE(per_cpu(trc_ipi_to_cpu, cpu));
- // Disable CPU hotplug across the tasklist scan.
- // This also waits for all readers in CPU-hotplug code paths.
+ // Disable CPU hotplug across the CPU scan for the benefit of
+ // any IPIs that might be needed. This also waits for all readers
+ // in CPU-hotplug code paths.
cpus_read_lock();
-}
-/* Do first-round processing for the specified task. */
-static void rcu_tasks_trace_pertask(struct task_struct *t,
- struct list_head *hop)
-{
- // During early boot when there is only the one boot CPU, there
- // is no idle task for the other CPUs. Just return.
- if (unlikely(t == NULL))
- return;
+ // These rcu_tasks_trace_pertask_prep() calls are serialized to
+ // allow safe access to the hop list.
+ for_each_online_cpu(cpu) {
+ rcu_read_lock();
+ t = cpu_curr_snapshot(cpu);
+ if (rcu_tasks_trace_pertask_prep(t, true))
+ trc_add_holdout(t, hop);
+ rcu_read_unlock();
+ }
- WRITE_ONCE(t->trc_reader_special.b.need_qs, false);
- WRITE_ONCE(t->trc_reader_checked, false);
- t->trc_ipi_to_cpu = -1;
- trc_wait_for_one_reader(t, hop);
+ // Only after all running tasks have been accounted for is it
+ // safe to take care of the tasks that have blocked within their
+ // current RCU tasks trace read-side critical section.
+ for_each_possible_cpu(cpu) {
+ rtpcp = per_cpu_ptr(rcu_tasks_trace.rtpcpu, cpu);
+ raw_spin_lock_irqsave_rcu_node(rtpcp, flags);
+ list_splice_init(&rtpcp->rtp_blkd_tasks, &blkd_tasks);
+ while (!list_empty(&blkd_tasks)) {
+ rcu_read_lock();
+ t = list_first_entry(&blkd_tasks, struct task_struct, trc_blkd_node);
+ list_del_init(&t->trc_blkd_node);
+ list_add(&t->trc_blkd_node, &rtpcp->rtp_blkd_tasks);
+ raw_spin_unlock_irqrestore_rcu_node(rtpcp, flags);
+ rcu_tasks_trace_pertask(t, hop);
+ rcu_read_unlock();
+ raw_spin_lock_irqsave_rcu_node(rtpcp, flags);
+ }
+ raw_spin_unlock_irqrestore_rcu_node(rtpcp, flags);
+ }
+
+ // Re-enable CPU hotplug now that the holdout list is populated.
+ cpus_read_unlock();
}
/*
- * Do intermediate processing between task and holdout scans and
- * pick up the idle tasks.
+ * Do intermediate processing between task and holdout scans.
*/
static void rcu_tasks_trace_postscan(struct list_head *hop)
{
- int cpu;
-
- for_each_possible_cpu(cpu)
- rcu_tasks_trace_pertask(idle_task(cpu), hop);
-
- // Re-enable CPU hotplug now that the tasklist scan has completed.
- cpus_read_unlock();
-
// Wait for late-stage exiting tasks to finish exiting.
// These might have passed the call to exit_tasks_rcu_finish().
synchronize_rcu();
- // Any tasks that exit after this point will set ->trc_reader_checked.
+ // Any tasks that exit after this point will set
+ // TRC_NEED_QS_CHECKED in ->trc_reader_special.b.need_qs.
}
/* Communicate task state back to the RCU tasks trace stall warning request. */
@@ -1429,11 +1549,11 @@ static int trc_check_slow_task(struct task_struct *t, void *arg)
{
struct trc_stall_chk_rdr *trc_rdrp = arg;
- if (task_curr(t))
+ if (task_curr(t) && cpu_online(task_cpu(t)))
return false; // It is running, so decline to inspect it.
trc_rdrp->nesting = READ_ONCE(t->trc_reader_nesting);
trc_rdrp->ipi_to_cpu = READ_ONCE(t->trc_ipi_to_cpu);
- trc_rdrp->needqs = READ_ONCE(t->trc_reader_special.b.need_qs);
+ trc_rdrp->needqs = rcu_ld_need_qs(t);
return true;
}
@@ -1450,18 +1570,21 @@ static void show_stalled_task_trace(struct task_struct *t, bool *firstreport)
}
cpu = task_cpu(t);
if (!task_call_func(t, trc_check_slow_task, &trc_rdr))
- pr_alert("P%d: %c\n",
+ pr_alert("P%d: %c%c\n",
t->pid,
+ ".I"[t->trc_ipi_to_cpu >= 0],
".i"[is_idle_tsk]);
else
- pr_alert("P%d: %c%c%c nesting: %d%c cpu: %d\n",
+ pr_alert("P%d: %c%c%c%c nesting: %d%c%c cpu: %d%s\n",
t->pid,
".I"[trc_rdr.ipi_to_cpu >= 0],
".i"[is_idle_tsk],
".N"[cpu >= 0 && tick_nohz_full_cpu(cpu)],
+ ".B"[!!data_race(t->trc_reader_special.b.blocked)],
trc_rdr.nesting,
- " N"[!!trc_rdr.needqs],
- cpu);
+ " !CN"[trc_rdr.needqs & 0x3],
+ " ?"[trc_rdr.needqs > 0x3],
+ cpu, cpu_online(cpu) ? "" : "(offline)");
sched_show_task(t);
}
@@ -1481,18 +1604,18 @@ static void check_all_holdout_tasks_trace(struct list_head *hop,
{
struct task_struct *g, *t;
- // Disable CPU hotplug across the holdout list scan.
+ // Disable CPU hotplug across the holdout list scan for IPIs.
cpus_read_lock();
list_for_each_entry_safe(t, g, hop, trc_holdout_list) {
// If safe and needed, try to check the current task.
if (READ_ONCE(t->trc_ipi_to_cpu) == -1 &&
- !READ_ONCE(t->trc_reader_checked))
+ !(rcu_ld_need_qs(t) & TRC_NEED_QS_CHECKED))
trc_wait_for_one_reader(t, hop);
// If check succeeded, remove this task from the list.
if (smp_load_acquire(&t->trc_ipi_to_cpu) == -1 &&
- READ_ONCE(t->trc_reader_checked))
+ rcu_ld_need_qs(t) == TRC_NEED_QS_CHECKED)
trc_del_holdout(t);
else if (needreport)
show_stalled_task_trace(t, firstreport);
@@ -1516,10 +1639,6 @@ static void rcu_tasks_trace_empty_fn(void *unused)
static void rcu_tasks_trace_postgp(struct rcu_tasks *rtp)
{
int cpu;
- bool firstreport;
- struct task_struct *g, *t;
- LIST_HEAD(holdouts);
- long ret;
// Wait for any lingering IPI handlers to complete. Note that
// if a CPU has gone offline or transitioned to userspace in the
@@ -1530,37 +1649,6 @@ static void rcu_tasks_trace_postgp(struct rcu_tasks *rtp)
if (WARN_ON_ONCE(smp_load_acquire(per_cpu_ptr(&trc_ipi_to_cpu, cpu))))
smp_call_function_single(cpu, rcu_tasks_trace_empty_fn, NULL, 1);
- // Remove the safety count.
- smp_mb__before_atomic(); // Order vs. earlier atomics
- atomic_dec(&trc_n_readers_need_end);
- smp_mb__after_atomic(); // Order vs. later atomics
-
- // Wait for readers.
- set_tasks_gp_state(rtp, RTGS_WAIT_READERS);
- for (;;) {
- ret = wait_event_idle_exclusive_timeout(
- trc_wait,
- atomic_read(&trc_n_readers_need_end) == 0,
- READ_ONCE(rcu_task_stall_timeout));
- if (ret)
- break; // Count reached zero.
- // Stall warning time, so make a list of the offenders.
- rcu_read_lock();
- for_each_process_thread(g, t)
- if (READ_ONCE(t->trc_reader_special.b.need_qs))
- trc_add_holdout(t, &holdouts);
- rcu_read_unlock();
- firstreport = true;
- list_for_each_entry_safe(t, g, &holdouts, trc_holdout_list) {
- if (READ_ONCE(t->trc_reader_special.b.need_qs))
- show_stalled_task_trace(t, &firstreport);
- trc_del_holdout(t); // Release task_struct reference.
- }
- if (firstreport)
- pr_err("INFO: rcu_tasks_trace detected stalls? (Counter/taskslist mismatch?)\n");
- show_stalled_ipi_trace();
- pr_err("\t%d holdouts\n", atomic_read(&trc_n_readers_need_end));
- }
smp_mb(); // Caller's code must be ordered after wakeup.
// Pairs with pretty much every ordering primitive.
}
@@ -1568,11 +1656,14 @@ static void rcu_tasks_trace_postgp(struct rcu_tasks *rtp)
/* Report any needed quiescent state for this exiting task. */
static void exit_tasks_rcu_finish_trace(struct task_struct *t)
{
- WRITE_ONCE(t->trc_reader_checked, true);
+ union rcu_special trs = READ_ONCE(t->trc_reader_special);
+
+ rcu_trc_cmpxchg_need_qs(t, 0, TRC_NEED_QS_CHECKED);
WARN_ON_ONCE(READ_ONCE(t->trc_reader_nesting));
- WRITE_ONCE(t->trc_reader_nesting, 0);
- if (WARN_ON_ONCE(READ_ONCE(t->trc_reader_special.b.need_qs)))
+ if (WARN_ON_ONCE(rcu_ld_need_qs(t) & TRC_NEED_QS || trs.b.blocked))
rcu_read_unlock_trace_special(t);
+ else
+ WRITE_ONCE(t->trc_reader_nesting, 0);
}
/**
@@ -1646,7 +1737,6 @@ static int __init rcu_spawn_tasks_trace_kthread(void)
rcu_tasks_trace.init_fract = 1;
}
rcu_tasks_trace.pregp_func = rcu_tasks_trace_pregp_step;
- rcu_tasks_trace.pertask_func = rcu_tasks_trace_pertask;
rcu_tasks_trace.postscan_func = rcu_tasks_trace_postscan;
rcu_tasks_trace.holdouts_func = check_all_holdout_tasks_trace;
rcu_tasks_trace.postgp_func = rcu_tasks_trace_postgp;
@@ -1659,7 +1749,8 @@ void show_rcu_tasks_trace_gp_kthread(void)
{
char buf[64];
- sprintf(buf, "N%d h:%lu/%lu/%lu", atomic_read(&trc_n_readers_need_end),
+ sprintf(buf, "N%lu h:%lu/%lu/%lu",
+ data_race(n_trc_holdouts),
data_race(n_heavy_reader_ofl_updates),
data_race(n_heavy_reader_updates),
data_race(n_heavy_reader_attempts));
@@ -1686,23 +1777,24 @@ struct rcu_tasks_test_desc {
struct rcu_head rh;
const char *name;
bool notrun;
+ unsigned long runstart;
};
static struct rcu_tasks_test_desc tests[] = {
{
.name = "call_rcu_tasks()",
/* If not defined, the test is skipped. */
- .notrun = !IS_ENABLED(CONFIG_TASKS_RCU),
+ .notrun = IS_ENABLED(CONFIG_TASKS_RCU),
},
{
.name = "call_rcu_tasks_rude()",
/* If not defined, the test is skipped. */
- .notrun = !IS_ENABLED(CONFIG_TASKS_RUDE_RCU),
+ .notrun = IS_ENABLED(CONFIG_TASKS_RUDE_RCU),
},
{
.name = "call_rcu_tasks_trace()",
/* If not defined, the test is skipped. */
- .notrun = !IS_ENABLED(CONFIG_TASKS_TRACE_RCU)
+ .notrun = IS_ENABLED(CONFIG_TASKS_TRACE_RCU)
}
};
@@ -1713,46 +1805,85 @@ static void test_rcu_tasks_callback(struct rcu_head *rhp)
pr_info("Callback from %s invoked.\n", rttd->name);
- rttd->notrun = true;
+ rttd->notrun = false;
}
static void rcu_tasks_initiate_self_tests(void)
{
+ unsigned long j = jiffies;
+
pr_info("Running RCU-tasks wait API self tests\n");
#ifdef CONFIG_TASKS_RCU
+ tests[0].runstart = j;
synchronize_rcu_tasks();
call_rcu_tasks(&tests[0].rh, test_rcu_tasks_callback);
#endif
#ifdef CONFIG_TASKS_RUDE_RCU
+ tests[1].runstart = j;
synchronize_rcu_tasks_rude();
call_rcu_tasks_rude(&tests[1].rh, test_rcu_tasks_callback);
#endif
#ifdef CONFIG_TASKS_TRACE_RCU
+ tests[2].runstart = j;
synchronize_rcu_tasks_trace();
call_rcu_tasks_trace(&tests[2].rh, test_rcu_tasks_callback);
#endif
}
+/*
+ * Return: 0 - test passed
+ * 1 - test failed, but have not timed out yet
+ * -1 - test failed and timed out
+ */
static int rcu_tasks_verify_self_tests(void)
{
int ret = 0;
int i;
+ unsigned long bst = rcu_task_stall_timeout;
+ if (bst <= 0 || bst > RCU_TASK_BOOT_STALL_TIMEOUT)
+ bst = RCU_TASK_BOOT_STALL_TIMEOUT;
for (i = 0; i < ARRAY_SIZE(tests); i++) {
- if (!tests[i].notrun) { // still hanging.
- pr_err("%s has been failed.\n", tests[i].name);
- ret = -1;
+ while (tests[i].notrun) { // still hanging.
+ if (time_after(jiffies, tests[i].runstart + bst)) {
+ pr_err("%s has failed boot-time tests.\n", tests[i].name);
+ ret = -1;
+ break;
+ }
+ ret = 1;
+ break;
}
}
-
- if (ret)
- WARN_ON(1);
+ WARN_ON(ret < 0);
return ret;
}
-late_initcall(rcu_tasks_verify_self_tests);
+
+/*
+ * Repeat the rcu_tasks_verify_self_tests() call once every second until the
+ * test passes or has timed out.
+ */
+static struct delayed_work rcu_tasks_verify_work;
+static void rcu_tasks_verify_work_fn(struct work_struct *work __maybe_unused)
+{
+ int ret = rcu_tasks_verify_self_tests();
+
+ if (ret <= 0)
+ return;
+
+ /* Test fails but not timed out yet, reschedule another check */
+ schedule_delayed_work(&rcu_tasks_verify_work, HZ);
+}
+
+static int rcu_tasks_verify_schedule_work(void)
+{
+ INIT_DELAYED_WORK(&rcu_tasks_verify_work, rcu_tasks_verify_work_fn);
+ rcu_tasks_verify_work_fn(NULL);
+ return 0;
+}
+late_initcall(rcu_tasks_verify_schedule_work);
#else /* #ifdef CONFIG_PROVE_RCU */
static void rcu_tasks_initiate_self_tests(void) { }
#endif /* #else #ifdef CONFIG_PROVE_RCU */