summaryrefslogtreecommitdiff
path: root/kernel/rcu/srcutree.c
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/rcu/srcutree.c')
-rw-r--r--kernel/rcu/srcutree.c211
1 files changed, 112 insertions, 99 deletions
diff --git a/kernel/rcu/srcutree.c b/kernel/rcu/srcutree.c
index 5e2e53464794..d2a694944553 100644
--- a/kernel/rcu/srcutree.c
+++ b/kernel/rcu/srcutree.c
@@ -116,8 +116,9 @@ do { \
/*
* Initialize SRCU per-CPU data. Note that statically allocated
* srcu_struct structures might already have srcu_read_lock() and
- * srcu_read_unlock() running against them. So if the is_static parameter
- * is set, don't initialize ->srcu_lock_count[] and ->srcu_unlock_count[].
+ * srcu_read_unlock() running against them. So if the is_static
+ * parameter is set, don't initialize ->srcu_ctrs[].srcu_locks and
+ * ->srcu_ctrs[].srcu_unlocks.
*/
static void init_srcu_struct_data(struct srcu_struct *ssp)
{
@@ -128,8 +129,6 @@ static void init_srcu_struct_data(struct srcu_struct *ssp)
* Initialize the per-CPU srcu_data array, which feeds into the
* leaves of the srcu_node tree.
*/
- BUILD_BUG_ON(ARRAY_SIZE(sdp->srcu_lock_count) !=
- ARRAY_SIZE(sdp->srcu_unlock_count));
for_each_possible_cpu(cpu) {
sdp = per_cpu_ptr(ssp->sda, cpu);
spin_lock_init(&ACCESS_PRIVATE(sdp, lock));
@@ -247,15 +246,16 @@ static int init_srcu_struct_fields(struct srcu_struct *ssp, bool is_static)
ssp->srcu_sup->node = NULL;
mutex_init(&ssp->srcu_sup->srcu_cb_mutex);
mutex_init(&ssp->srcu_sup->srcu_gp_mutex);
- ssp->srcu_idx = 0;
ssp->srcu_sup->srcu_gp_seq = SRCU_GP_SEQ_INITIAL_VAL;
ssp->srcu_sup->srcu_barrier_seq = 0;
mutex_init(&ssp->srcu_sup->srcu_barrier_mutex);
atomic_set(&ssp->srcu_sup->srcu_barrier_cpu_cnt, 0);
INIT_DELAYED_WORK(&ssp->srcu_sup->work, process_srcu);
ssp->srcu_sup->sda_is_static = is_static;
- if (!is_static)
+ if (!is_static) {
ssp->sda = alloc_percpu(struct srcu_data);
+ ssp->srcu_ctrp = &ssp->sda->srcu_ctrs[0];
+ }
if (!ssp->sda)
goto err_free_sup;
init_srcu_struct_data(ssp);
@@ -429,10 +429,10 @@ static bool srcu_gp_is_expedited(struct srcu_struct *ssp)
}
/*
- * Computes approximate total of the readers' ->srcu_lock_count[] values
- * for the rank of per-CPU counters specified by idx, and returns true if
- * the caller did the proper barrier (gp), and if the count of the locks
- * matches that of the unlocks passed in.
+ * Computes approximate total of the readers' ->srcu_ctrs[].srcu_locks
+ * values for the rank of per-CPU counters specified by idx, and returns
+ * true if the caller did the proper barrier (gp), and if the count of
+ * the locks matches that of the unlocks passed in.
*/
static bool srcu_readers_lock_idx(struct srcu_struct *ssp, int idx, bool gp, unsigned long unlocks)
{
@@ -443,20 +443,20 @@ static bool srcu_readers_lock_idx(struct srcu_struct *ssp, int idx, bool gp, uns
for_each_possible_cpu(cpu) {
struct srcu_data *sdp = per_cpu_ptr(ssp->sda, cpu);
- sum += atomic_long_read(&sdp->srcu_lock_count[idx]);
+ sum += atomic_long_read(&sdp->srcu_ctrs[idx].srcu_locks);
if (IS_ENABLED(CONFIG_PROVE_RCU))
mask = mask | READ_ONCE(sdp->srcu_reader_flavor);
}
WARN_ONCE(IS_ENABLED(CONFIG_PROVE_RCU) && (mask & (mask - 1)),
"Mixed reader flavors for srcu_struct at %ps.\n", ssp);
- if (mask & SRCU_READ_FLAVOR_LITE && !gp)
+ if (mask & SRCU_READ_FLAVOR_SLOWGP && !gp)
return false;
return sum == unlocks;
}
/*
- * Returns approximate total of the readers' ->srcu_unlock_count[] values
- * for the rank of per-CPU counters specified by idx.
+ * Returns approximate total of the readers' ->srcu_ctrs[].srcu_unlocks
+ * values for the rank of per-CPU counters specified by idx.
*/
static unsigned long srcu_readers_unlock_idx(struct srcu_struct *ssp, int idx, unsigned long *rdm)
{
@@ -467,7 +467,7 @@ static unsigned long srcu_readers_unlock_idx(struct srcu_struct *ssp, int idx, u
for_each_possible_cpu(cpu) {
struct srcu_data *sdp = per_cpu_ptr(ssp->sda, cpu);
- sum += atomic_long_read(&sdp->srcu_unlock_count[idx]);
+ sum += atomic_long_read(&sdp->srcu_ctrs[idx].srcu_unlocks);
mask = mask | READ_ONCE(sdp->srcu_reader_flavor);
}
WARN_ONCE(IS_ENABLED(CONFIG_PROVE_RCU) && (mask & (mask - 1)),
@@ -487,7 +487,7 @@ static bool srcu_readers_active_idx_check(struct srcu_struct *ssp, int idx)
unsigned long unlocks;
unlocks = srcu_readers_unlock_idx(ssp, idx, &rdm);
- did_gp = !!(rdm & SRCU_READ_FLAVOR_LITE);
+ did_gp = !!(rdm & SRCU_READ_FLAVOR_SLOWGP);
/*
* Make sure that a lock is always counted if the corresponding
@@ -509,48 +509,49 @@ static bool srcu_readers_active_idx_check(struct srcu_struct *ssp, int idx)
* If the locks are the same as the unlocks, then there must have
* been no readers on this index at some point in this function.
* But there might be more readers, as a task might have read
- * the current ->srcu_idx but not yet have incremented its CPU's
- * ->srcu_lock_count[idx] counter. In fact, it is possible
+ * the current ->srcu_ctrp but not yet have incremented its CPU's
+ * ->srcu_ctrs[idx].srcu_locks counter. In fact, it is possible
* that most of the tasks have been preempted between fetching
- * ->srcu_idx and incrementing ->srcu_lock_count[idx]. And there
- * could be almost (ULONG_MAX / sizeof(struct task_struct)) tasks
- * in a system whose address space was fully populated with memory.
- * Call this quantity Nt.
+ * ->srcu_ctrp and incrementing ->srcu_ctrs[idx].srcu_locks. And
+ * there could be almost (ULONG_MAX / sizeof(struct task_struct))
+ * tasks in a system whose address space was fully populated
+ * with memory. Call this quantity Nt.
*
- * So suppose that the updater is preempted at this point in the
- * code for a long time. That now-preempted updater has already
- * flipped ->srcu_idx (possibly during the preceding grace period),
- * done an smp_mb() (again, possibly during the preceding grace
- * period), and summed up the ->srcu_unlock_count[idx] counters.
- * How many times can a given one of the aforementioned Nt tasks
- * increment the old ->srcu_idx value's ->srcu_lock_count[idx]
- * counter, in the absence of nesting?
+ * So suppose that the updater is preempted at this
+ * point in the code for a long time. That now-preempted
+ * updater has already flipped ->srcu_ctrp (possibly during
+ * the preceding grace period), done an smp_mb() (again,
+ * possibly during the preceding grace period), and summed up
+ * the ->srcu_ctrs[idx].srcu_unlocks counters. How many times
+ * can a given one of the aforementioned Nt tasks increment the
+ * old ->srcu_ctrp value's ->srcu_ctrs[idx].srcu_locks counter,
+ * in the absence of nesting?
*
* It can clearly do so once, given that it has already fetched
- * the old value of ->srcu_idx and is just about to use that value
- * to index its increment of ->srcu_lock_count[idx]. But as soon as
- * it leaves that SRCU read-side critical section, it will increment
- * ->srcu_unlock_count[idx], which must follow the updater's above
- * read from that same value. Thus, as soon the reading task does
- * an smp_mb() and a later fetch from ->srcu_idx, that task will be
- * guaranteed to get the new index. Except that the increment of
- * ->srcu_unlock_count[idx] in __srcu_read_unlock() is after the
- * smp_mb(), and the fetch from ->srcu_idx in __srcu_read_lock()
- * is before the smp_mb(). Thus, that task might not see the new
- * value of ->srcu_idx until the -second- __srcu_read_lock(),
- * which in turn means that this task might well increment
- * ->srcu_lock_count[idx] for the old value of ->srcu_idx twice,
- * not just once.
+ * the old value of ->srcu_ctrp and is just about to use that
+ * value to index its increment of ->srcu_ctrs[idx].srcu_locks.
+ * But as soon as it leaves that SRCU read-side critical section,
+ * it will increment ->srcu_ctrs[idx].srcu_unlocks, which must
+ * follow the updater's above read from that same value. Thus,
+ as soon the reading task does an smp_mb() and a later fetch from
+ * ->srcu_ctrp, that task will be guaranteed to get the new index.
+ * Except that the increment of ->srcu_ctrs[idx].srcu_unlocks
+ * in __srcu_read_unlock() is after the smp_mb(), and the fetch
+ * from ->srcu_ctrp in __srcu_read_lock() is before the smp_mb().
+ * Thus, that task might not see the new value of ->srcu_ctrp until
+ * the -second- __srcu_read_lock(), which in turn means that this
+ * task might well increment ->srcu_ctrs[idx].srcu_locks for the
+ * old value of ->srcu_ctrp twice, not just once.
*
* However, it is important to note that a given smp_mb() takes
* effect not just for the task executing it, but also for any
* later task running on that same CPU.
*
- * That is, there can be almost Nt + Nc further increments of
- * ->srcu_lock_count[idx] for the old index, where Nc is the number
- * of CPUs. But this is OK because the size of the task_struct
- * structure limits the value of Nt and current systems limit Nc
- * to a few thousand.
+ * That is, there can be almost Nt + Nc further increments
+ * of ->srcu_ctrs[idx].srcu_locks for the old index, where Nc
+ * is the number of CPUs. But this is OK because the size of
+ * the task_struct structure limits the value of Nt and current
+ * systems limit Nc to a few thousand.
*
* OK, but what about nesting? This does impose a limit on
* nesting of half of the size of the task_struct structure
@@ -581,10 +582,10 @@ static bool srcu_readers_active(struct srcu_struct *ssp)
for_each_possible_cpu(cpu) {
struct srcu_data *sdp = per_cpu_ptr(ssp->sda, cpu);
- sum += atomic_long_read(&sdp->srcu_lock_count[0]);
- sum += atomic_long_read(&sdp->srcu_lock_count[1]);
- sum -= atomic_long_read(&sdp->srcu_unlock_count[0]);
- sum -= atomic_long_read(&sdp->srcu_unlock_count[1]);
+ sum += atomic_long_read(&sdp->srcu_ctrs[0].srcu_locks);
+ sum += atomic_long_read(&sdp->srcu_ctrs[1].srcu_locks);
+ sum -= atomic_long_read(&sdp->srcu_ctrs[0].srcu_unlocks);
+ sum -= atomic_long_read(&sdp->srcu_ctrs[1].srcu_unlocks);
}
return sum;
}
@@ -647,6 +648,7 @@ static unsigned long srcu_get_delay(struct srcu_struct *ssp)
unsigned long jbase = SRCU_INTERVAL;
struct srcu_usage *sup = ssp->srcu_sup;
+ lockdep_assert_held(&ACCESS_PRIVATE(ssp->srcu_sup, lock));
if (srcu_gp_is_expedited(ssp))
jbase = 0;
if (rcu_seq_state(READ_ONCE(sup->srcu_gp_seq))) {
@@ -674,9 +676,13 @@ static unsigned long srcu_get_delay(struct srcu_struct *ssp)
void cleanup_srcu_struct(struct srcu_struct *ssp)
{
int cpu;
+ unsigned long delay;
struct srcu_usage *sup = ssp->srcu_sup;
- if (WARN_ON(!srcu_get_delay(ssp)))
+ spin_lock_irq_rcu_node(ssp->srcu_sup);
+ delay = srcu_get_delay(ssp);
+ spin_unlock_irq_rcu_node(ssp->srcu_sup);
+ if (WARN_ON(!delay))
return; /* Just leak it! */
if (WARN_ON(srcu_readers_active(ssp)))
return; /* Just leak it! */
@@ -738,16 +744,16 @@ EXPORT_SYMBOL_GPL(__srcu_check_read_flavor);
/*
* Counts the new reader in the appropriate per-CPU element of the
* srcu_struct.
- * Returns an index that must be passed to the matching srcu_read_unlock().
+ * Returns a guaranteed non-negative index that must be passed to the
+ * matching __srcu_read_unlock().
*/
int __srcu_read_lock(struct srcu_struct *ssp)
{
- int idx;
+ struct srcu_ctr __percpu *scp = READ_ONCE(ssp->srcu_ctrp);
- idx = READ_ONCE(ssp->srcu_idx) & 0x1;
- this_cpu_inc(ssp->sda->srcu_lock_count[idx].counter);
+ this_cpu_inc(scp->srcu_locks.counter);
smp_mb(); /* B */ /* Avoid leaking the critical section. */
- return idx;
+ return __srcu_ptr_to_ctr(ssp, scp);
}
EXPORT_SYMBOL_GPL(__srcu_read_lock);
@@ -759,7 +765,7 @@ EXPORT_SYMBOL_GPL(__srcu_read_lock);
void __srcu_read_unlock(struct srcu_struct *ssp, int idx)
{
smp_mb(); /* C */ /* Avoid leaking the critical section. */
- this_cpu_inc(ssp->sda->srcu_unlock_count[idx].counter);
+ this_cpu_inc(__srcu_ctr_to_ptr(ssp, idx)->srcu_unlocks.counter);
}
EXPORT_SYMBOL_GPL(__srcu_read_unlock);
@@ -772,13 +778,12 @@ EXPORT_SYMBOL_GPL(__srcu_read_unlock);
*/
int __srcu_read_lock_nmisafe(struct srcu_struct *ssp)
{
- int idx;
- struct srcu_data *sdp = raw_cpu_ptr(ssp->sda);
+ struct srcu_ctr __percpu *scpp = READ_ONCE(ssp->srcu_ctrp);
+ struct srcu_ctr *scp = raw_cpu_ptr(scpp);
- idx = READ_ONCE(ssp->srcu_idx) & 0x1;
- atomic_long_inc(&sdp->srcu_lock_count[idx]);
+ atomic_long_inc(&scp->srcu_locks);
smp_mb__after_atomic(); /* B */ /* Avoid leaking the critical section. */
- return idx;
+ return __srcu_ptr_to_ctr(ssp, scpp);
}
EXPORT_SYMBOL_GPL(__srcu_read_lock_nmisafe);
@@ -789,10 +794,8 @@ EXPORT_SYMBOL_GPL(__srcu_read_lock_nmisafe);
*/
void __srcu_read_unlock_nmisafe(struct srcu_struct *ssp, int idx)
{
- struct srcu_data *sdp = raw_cpu_ptr(ssp->sda);
-
smp_mb__before_atomic(); /* C */ /* Avoid leaking the critical section. */
- atomic_long_inc(&sdp->srcu_unlock_count[idx]);
+ atomic_long_inc(&raw_cpu_ptr(__srcu_ctr_to_ptr(ssp, idx))->srcu_unlocks);
}
EXPORT_SYMBOL_GPL(__srcu_read_unlock_nmisafe);
@@ -1076,7 +1079,6 @@ static void srcu_funnel_gp_start(struct srcu_struct *ssp, struct srcu_data *sdp,
/* If grace period not already in progress, start it. */
if (!WARN_ON_ONCE(rcu_seq_done(&sup->srcu_gp_seq, s)) &&
rcu_seq_state(sup->srcu_gp_seq) == SRCU_STATE_IDLE) {
- WARN_ON_ONCE(ULONG_CMP_GE(sup->srcu_gp_seq, sup->srcu_gp_seq_needed));
srcu_gp_start(ssp);
// And how can that list_add() in the "else" clause
@@ -1096,13 +1098,15 @@ static void srcu_funnel_gp_start(struct srcu_struct *ssp, struct srcu_data *sdp,
/*
* Wait until all readers counted by array index idx complete, but
* loop an additional time if there is an expedited grace period pending.
- * The caller must ensure that ->srcu_idx is not changed while checking.
+ * The caller must ensure that ->srcu_ctrp is not changed while checking.
*/
static bool try_check_zero(struct srcu_struct *ssp, int idx, int trycount)
{
unsigned long curdelay;
+ spin_lock_irq_rcu_node(ssp->srcu_sup);
curdelay = !srcu_get_delay(ssp);
+ spin_unlock_irq_rcu_node(ssp->srcu_sup);
for (;;) {
if (srcu_readers_active_idx_check(ssp, idx))
@@ -1114,30 +1118,30 @@ static bool try_check_zero(struct srcu_struct *ssp, int idx, int trycount)
}
/*
- * Increment the ->srcu_idx counter so that future SRCU readers will
+ * Increment the ->srcu_ctrp counter so that future SRCU readers will
* use the other rank of the ->srcu_(un)lock_count[] arrays. This allows
* us to wait for pre-existing readers in a starvation-free manner.
*/
static void srcu_flip(struct srcu_struct *ssp)
{
/*
- * Because the flip of ->srcu_idx is executed only if the
+ * Because the flip of ->srcu_ctrp is executed only if the
* preceding call to srcu_readers_active_idx_check() found that
- * the ->srcu_unlock_count[] and ->srcu_lock_count[] sums matched
- * and because that summing uses atomic_long_read(), there is
- * ordering due to a control dependency between that summing and
- * the WRITE_ONCE() in this call to srcu_flip(). This ordering
- * ensures that if this updater saw a given reader's increment from
- * __srcu_read_lock(), that reader was using a value of ->srcu_idx
- * from before the previous call to srcu_flip(), which should be
- * quite rare. This ordering thus helps forward progress because
- * the grace period could otherwise be delayed by additional
- * calls to __srcu_read_lock() using that old (soon to be new)
- * value of ->srcu_idx.
+ * the ->srcu_ctrs[].srcu_unlocks and ->srcu_ctrs[].srcu_locks sums
+ * matched and because that summing uses atomic_long_read(),
+ * there is ordering due to a control dependency between that
+ * summing and the WRITE_ONCE() in this call to srcu_flip().
+ * This ordering ensures that if this updater saw a given reader's
+ * increment from __srcu_read_lock(), that reader was using a value
+ * of ->srcu_ctrp from before the previous call to srcu_flip(),
+ * which should be quite rare. This ordering thus helps forward
+ * progress because the grace period could otherwise be delayed
+ * by additional calls to __srcu_read_lock() using that old (soon
+ * to be new) value of ->srcu_ctrp.
*
* This sum-equality check and ordering also ensures that if
* a given call to __srcu_read_lock() uses the new value of
- * ->srcu_idx, this updater's earlier scans cannot have seen
+ * ->srcu_ctrp, this updater's earlier scans cannot have seen
* that reader's increments, which is all to the good, because
* this grace period need not wait on that reader. After all,
* if those earlier scans had seen that reader, there would have
@@ -1152,7 +1156,8 @@ static void srcu_flip(struct srcu_struct *ssp)
*/
smp_mb(); /* E */ /* Pairs with B and C. */
- WRITE_ONCE(ssp->srcu_idx, ssp->srcu_idx + 1); // Flip the counter.
+ WRITE_ONCE(ssp->srcu_ctrp,
+ &ssp->sda->srcu_ctrs[!(ssp->srcu_ctrp - &ssp->sda->srcu_ctrs[0])]);
/*
* Ensure that if the updater misses an __srcu_read_unlock()
@@ -1198,7 +1203,7 @@ static bool srcu_should_expedite(struct srcu_struct *ssp)
check_init_srcu_struct(ssp);
/* If _lite() readers, don't do unsolicited expediting. */
- if (this_cpu_read(ssp->sda->srcu_reader_flavor) & SRCU_READ_FLAVOR_LITE)
+ if (this_cpu_read(ssp->sda->srcu_reader_flavor) & SRCU_READ_FLAVOR_SLOWGP)
return false;
/* If the local srcu_data structure has callbacks, not idle. */
sdp = raw_cpu_ptr(ssp->sda);
@@ -1398,8 +1403,12 @@ static void __call_srcu(struct srcu_struct *ssp, struct rcu_head *rhp,
* read-side critical sections are delimited by srcu_read_lock() and
* srcu_read_unlock(), and may be nested.
*
- * The callback will be invoked from process context, but must nevertheless
- * be fast and must not block.
+ * The callback will be invoked from process context, but with bh
+ * disabled. The callback function must therefore be fast and must
+ * not block.
+ *
+ * See the description of call_rcu() for more detailed information on
+ * memory ordering guarantees.
*/
void call_srcu(struct srcu_struct *ssp, struct rcu_head *rhp,
rcu_callback_t func)
@@ -1465,8 +1474,9 @@ EXPORT_SYMBOL_GPL(synchronize_srcu_expedited);
*
* Wait for the count to drain to zero of both indexes. To avoid the
* possible starvation of synchronize_srcu(), it waits for the count of
- * the index=((->srcu_idx & 1) ^ 1) to drain to zero at first,
- * and then flip the srcu_idx and wait for the count of the other index.
+ * the index=!(ssp->srcu_ctrp - &ssp->sda->srcu_ctrs[0]) to drain to zero
+ * at first, and then flip the ->srcu_ctrp and wait for the count of the
+ * other index.
*
* Can block; must be called from process context.
*
@@ -1675,7 +1685,7 @@ EXPORT_SYMBOL_GPL(srcu_barrier);
*/
unsigned long srcu_batches_completed(struct srcu_struct *ssp)
{
- return READ_ONCE(ssp->srcu_idx);
+ return READ_ONCE(ssp->srcu_sup->srcu_gp_seq);
}
EXPORT_SYMBOL_GPL(srcu_batches_completed);
@@ -1692,7 +1702,7 @@ static void srcu_advance_state(struct srcu_struct *ssp)
/*
* Because readers might be delayed for an extended period after
- * fetching ->srcu_idx for their index, at any point in time there
+ * fetching ->srcu_ctrp for their index, at any point in time there
* might well be readers using both idx=0 and idx=1. We therefore
* need to wait for readers to clear from both index values before
* invoking a callback.
@@ -1720,7 +1730,7 @@ static void srcu_advance_state(struct srcu_struct *ssp)
}
if (rcu_seq_state(READ_ONCE(ssp->srcu_sup->srcu_gp_seq)) == SRCU_STATE_SCAN1) {
- idx = 1 ^ (ssp->srcu_idx & 1);
+ idx = !(ssp->srcu_ctrp - &ssp->sda->srcu_ctrs[0]);
if (!try_check_zero(ssp, idx, 1)) {
mutex_unlock(&ssp->srcu_sup->srcu_gp_mutex);
return; /* readers present, retry later. */
@@ -1738,7 +1748,7 @@ static void srcu_advance_state(struct srcu_struct *ssp)
* SRCU read-side critical sections are normally short,
* so check at least twice in quick succession after a flip.
*/
- idx = 1 ^ (ssp->srcu_idx & 1);
+ idx = !(ssp->srcu_ctrp - &ssp->sda->srcu_ctrs[0]);
if (!try_check_zero(ssp, idx, 2)) {
mutex_unlock(&ssp->srcu_sup->srcu_gp_mutex);
return; /* readers present, retry later. */
@@ -1849,7 +1859,9 @@ static void process_srcu(struct work_struct *work)
ssp = sup->srcu_ssp;
srcu_advance_state(ssp);
+ spin_lock_irq_rcu_node(ssp->srcu_sup);
curdelay = srcu_get_delay(ssp);
+ spin_unlock_irq_rcu_node(ssp->srcu_sup);
if (curdelay) {
WRITE_ONCE(sup->reschedule_count, 0);
} else {
@@ -1896,7 +1908,7 @@ void srcu_torture_stats_print(struct srcu_struct *ssp, char *tt, char *tf)
int ss_state = READ_ONCE(ssp->srcu_sup->srcu_size_state);
int ss_state_idx = ss_state;
- idx = ssp->srcu_idx & 0x1;
+ idx = ssp->srcu_ctrp - &ssp->sda->srcu_ctrs[0];
if (ss_state < 0 || ss_state >= ARRAY_SIZE(srcu_size_state_name))
ss_state_idx = ARRAY_SIZE(srcu_size_state_name) - 1;
pr_alert("%s%s Tree SRCU g%ld state %d (%s)",
@@ -1914,8 +1926,8 @@ void srcu_torture_stats_print(struct srcu_struct *ssp, char *tt, char *tf)
struct srcu_data *sdp;
sdp = per_cpu_ptr(ssp->sda, cpu);
- u0 = data_race(atomic_long_read(&sdp->srcu_unlock_count[!idx]));
- u1 = data_race(atomic_long_read(&sdp->srcu_unlock_count[idx]));
+ u0 = data_race(atomic_long_read(&sdp->srcu_ctrs[!idx].srcu_unlocks));
+ u1 = data_race(atomic_long_read(&sdp->srcu_ctrs[idx].srcu_unlocks));
/*
* Make sure that a lock is always counted if the corresponding
@@ -1923,8 +1935,8 @@ void srcu_torture_stats_print(struct srcu_struct *ssp, char *tt, char *tf)
*/
smp_rmb();
- l0 = data_race(atomic_long_read(&sdp->srcu_lock_count[!idx]));
- l1 = data_race(atomic_long_read(&sdp->srcu_lock_count[idx]));
+ l0 = data_race(atomic_long_read(&sdp->srcu_ctrs[!idx].srcu_locks));
+ l1 = data_race(atomic_long_read(&sdp->srcu_ctrs[idx].srcu_locks));
c0 = l0 - u0;
c1 = l1 - u1;
@@ -2001,6 +2013,7 @@ static int srcu_module_coming(struct module *mod)
ssp->sda = alloc_percpu(struct srcu_data);
if (WARN_ON_ONCE(!ssp->sda))
return -ENOMEM;
+ ssp->srcu_ctrp = &ssp->sda->srcu_ctrs[0];
}
return 0;
}