Skip to content

Commit f7a300a

Browse files
committed
kvserver: change forTesting suffixes to Testing prefixes for replica.go
Addresses TODO message by @wenyihu6 to do as titled. Standardize function names to use `Testing` prefix. Updated test-related functions in `kvserver/replica.go` and related tests to follow the common `Testing` prefix convention, replacing suffix-based naming. Fixes: #144119 Release note: None
1 parent 161d59e commit f7a300a

16 files changed

+45
-48
lines changed

pkg/ccl/changefeedccl/protected_timestamps_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -637,7 +637,7 @@ func TestPTSRecordProtectsTargetsAndSystemTables(t *testing.T) {
637637
t,
638638
spanconfigptsreader.TestingRefreshPTSState(ctx, ptsReader, asOf),
639639
)
640-
require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx))
640+
require.NoError(t, repl.TestingReadProtectedTimestamps(ctx))
641641
}
642642
gcTestTableRange := func(tableName, databaseName string) {
643643
row := sqlDB.QueryRow(t, fmt.Sprintf("SELECT range_id FROM [SHOW RANGES FROM TABLE %s.%s]", tableName, databaseName))

pkg/crosscluster/physical/stream_ingestion_manager_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -195,7 +195,7 @@ func TestRevertTenantToTimestampPTS(t *testing.T) {
195195
t,
196196
spanconfigptsreader.TestingRefreshPTSState(ctx, ptsReader, asOf),
197197
)
198-
require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx))
198+
require.NoError(t, repl.TestingReadProtectedTimestamps(ctx))
199199

200200
t.Logf("enqueuing range %d for mvccGC", rangeID)
201201
systemSQL.Exec(t, `SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, rangeID)

pkg/kv/kvclient/kvcoord/dist_sender_circuit_breaker_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -103,7 +103,7 @@ func TestDistSenderReplicaStall(t *testing.T) {
103103
// Deadlock n3.
104104
repl3, err := tc.GetFirstStoreFromServer(t, 2).GetReplica(desc.RangeID)
105105
require.NoError(t, err)
106-
mu := repl3.GetMutexForTesting()
106+
mu := repl3.TestingGetMutex()
107107
mu.Lock()
108108
defer mu.Unlock()
109109
t.Log("deadlocked n3")
@@ -211,7 +211,7 @@ func TestDistSenderCircuitBreakerModes(t *testing.T) {
211211
// Deadlock either liveness or the scratch range.
212212
repl, err := tc.GetFirstStoreFromServer(t, 2).GetReplica(desc.RangeID)
213213
require.NoError(t, err)
214-
mu := repl.GetMutexForTesting()
214+
mu := repl.TestingGetMutex()
215215
mu.Lock()
216216
defer mu.Unlock()
217217
t.Logf("deadlocked range on n3 - %v", desc)

pkg/kv/kvclient/kvtenant/tenant_kv_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ func TestTenantRangeQPSStat(t *testing.T) {
7070
require.NoError(t, err)
7171
// NB: We call directly into the load tracking struct, in order to avoid
7272
// flakes due to timing differences affecting the result
73-
loadStats := repl.GetLoadStatsForTesting()
73+
loadStats := repl.TestingGetLoadStats()
7474
qpsBefore := loadStats.TestingGetSum(load.Queries)
7575
for i := 0; i < 110; i++ {
7676
r.Exec(t, `SELECT k FROM foo.qps_test`)

pkg/kv/kvclient/rangefeed/rangefeed_external_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1009,7 +1009,7 @@ func TestUnrecoverableErrors(t *testing.T) {
10091009
if conf, err := repl.LoadSpanConfig(ctx); err != nil || conf.GCPolicy.IgnoreStrictEnforcement {
10101010
return errors.New("waiting for span config to apply")
10111011
}
1012-
require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx))
1012+
require.NoError(t, repl.TestingReadProtectedTimestamps(ctx))
10131013
return nil
10141014
})
10151015

pkg/kv/kvserver/client_mvcc_gc_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -73,7 +73,7 @@ func TestMVCCGCCorrectStats(t *testing.T) {
7373
ms.ValBytes = 32 * (1 << 20) // 16mb
7474
ms.GCBytesAge = 48 * (1 << 20) * 100 * int64(time.Hour.Seconds())
7575

76-
repl.SetMVCCStatsForTesting(&ms)
76+
repl.TestingSetMVCCStats(&ms)
7777
require.NoError(t, store.ManualMVCCGC(repl))
7878

7979
// Verify that the mvcc gc queue restored the stats.
@@ -201,7 +201,7 @@ SELECT count(*)
201201
if len(cfg.GCPolicy.ProtectionPolicies) == 0 {
202202
return errors.New("waiting for span config to apply")
203203
}
204-
require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx))
204+
require.NoError(t, repl.TestingReadProtectedTimestamps(ctx))
205205
return nil
206206
})
207207

pkg/kv/kvserver/client_replica_test.go

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -3133,15 +3133,15 @@ func TestLeaderlessWatcherErrorRefreshedOnUnavailabilityTransition(t *testing.T)
31333133
// The leaderlessWatcher starts off as available.
31343134
require.False(t, repl.LeaderlessWatcher.IsUnavailable())
31353135
// Let it know it's leaderless.
3136-
repl.RefreshLeaderlessWatcherUnavailableStateForTesting(ctx, raft.None, manual.Now(), st)
3136+
repl.TestingRefreshLeaderlessWatcherUnavailableState(ctx, raft.None, manual.Now(), st)
31373137
// Even though the replica is leaderless, enough time hasn't passed for it to
31383138
// be considered unavailable.
31393139
require.False(t, repl.LeaderlessWatcher.IsUnavailable())
31403140
// The error should be nil as we're not considered leaderless at this point.
31413141
require.NoError(t, repl.LeaderlessWatcher.Err())
31423142
// Let enough time pass.
31433143
manual.Increment(10 * time.Second.Nanoseconds())
3144-
repl.RefreshLeaderlessWatcherUnavailableStateForTesting(ctx, raft.None, manual.Now(), st)
3144+
repl.TestingRefreshLeaderlessWatcherUnavailableState(ctx, raft.None, manual.Now(), st)
31453145
// Now the replica is considered unavailable.
31463146
require.True(t, repl.LeaderlessWatcher.IsUnavailable())
31473147
require.Error(t, repl.LeaderlessWatcher.Err())
@@ -3151,14 +3151,14 @@ func TestLeaderlessWatcherErrorRefreshedOnUnavailabilityTransition(t *testing.T)
31513151

31523152
// Next up, let the replica know there's a leader. This should make it
31533153
// available again.
3154-
repl.RefreshLeaderlessWatcherUnavailableStateForTesting(ctx, 1, manual.Now(), st)
3154+
repl.TestingRefreshLeaderlessWatcherUnavailableState(ctx, 1, manual.Now(), st)
31553155
require.False(t, repl.LeaderlessWatcher.IsUnavailable())
31563156
// Change the range descriptor. Mark it leaderless and let enough time pass
31573157
// for it to be considered unavailable again.
31583158
tc.AddVotersOrFatal(t, key, tc.Targets(2)...)
3159-
repl.RefreshLeaderlessWatcherUnavailableStateForTesting(ctx, raft.None, manual.Now(), st)
3159+
repl.TestingRefreshLeaderlessWatcherUnavailableState(ctx, raft.None, manual.Now(), st)
31603160
manual.Increment(10 * time.Second.Nanoseconds())
3161-
repl.RefreshLeaderlessWatcherUnavailableStateForTesting(ctx, raft.None, manual.Now(), st)
3161+
repl.TestingRefreshLeaderlessWatcherUnavailableState(ctx, raft.None, manual.Now(), st)
31623162
// The replica should now be considered unavailable again.
31633163
require.True(t, repl.LeaderlessWatcher.IsUnavailable())
31643164
require.Error(t, repl.LeaderlessWatcher.Err())
@@ -4707,7 +4707,7 @@ func TestStrictGCEnforcement(t *testing.T) {
47074707
t,
47084708
spanconfigptsreader.TestingRefreshPTSState(ctx, ptsReader, l.Start.ToTimestamp().Next()),
47094709
)
4710-
require.NoError(t, r.ReadProtectedTimestampsForTesting(ctx))
4710+
require.NoError(t, r.TestingReadProtectedTimestamps(ctx))
47114711
}
47124712
}
47134713
refreshTo = func(t *testing.T, asOf hlc.Timestamp) {
@@ -4718,7 +4718,7 @@ func TestStrictGCEnforcement(t *testing.T) {
47184718
t,
47194719
spanconfigptsreader.TestingRefreshPTSState(ctx, ptsReader, asOf),
47204720
)
4721-
require.NoError(t, r.ReadProtectedTimestampsForTesting(ctx))
4721+
require.NoError(t, r.TestingReadProtectedTimestamps(ctx))
47224722
}
47234723
}
47244724
// waitForProtectionAndReadProtectedTimestamps waits until the
@@ -4735,7 +4735,7 @@ func TestStrictGCEnforcement(t *testing.T) {
47354735
ptutil.TestingWaitForProtectedTimestampToExistOnSpans(ctx, t, tc.Server(i),
47364736
ptsReader, protectionTimestamp,
47374737
[]roachpb.Span{span})
4738-
require.NoError(t, r.ReadProtectedTimestampsForTesting(ctx))
4738+
require.NoError(t, r.TestingReadProtectedTimestamps(ctx))
47394739
}
47404740
}
47414741
insqlDB = tc.Server(0).InternalDB().(isql.DB)

pkg/kv/kvserver/replica.go

Lines changed: 18 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -1721,9 +1721,9 @@ func (r *Replica) GetMVCCStats() enginepb.MVCCStats {
17211721
return *r.shMu.state.Stats
17221722
}
17231723

1724-
// SetMVCCStatsForTesting updates the MVCC stats on the repl object only, it does
1724+
// TestingSetMVCCStats updates the MVCC stats on the repl object only, it does
17251725
// not affect the on disk state and is only safe to use for testing purposes.
1726-
func (r *Replica) SetMVCCStatsForTesting(stats *enginepb.MVCCStats) {
1726+
func (r *Replica) TestingSetMVCCStats(stats *enginepb.MVCCStats) {
17271727
r.raftMu.Lock()
17281728
defer r.raftMu.Unlock()
17291729
r.mu.Lock()
@@ -2749,9 +2749,9 @@ func (r *Replica) GetLeaseHistory() []roachpb.Lease {
27492749
return r.leaseHistory.get()
27502750
}
27512751

2752-
// EnableLeaseHistoryForTesting turns on the lease history for testing purposes.
2752+
// TestingEnableLeaseHistory turns on the lease history for testing purposes.
27532753
// Returns a function to return it to its original state that can be deferred.
2754-
func EnableLeaseHistoryForTesting(maxEntries int) func() {
2754+
func TestingEnableLeaseHistory(maxEntries int) func() {
27552755
originalValue := leaseHistoryMaxEntries
27562756
leaseHistoryMaxEntries = maxEntries
27572757
return func() {
@@ -2848,21 +2848,21 @@ func (r *Replica) measureNanosRunning(start time.Duration, f func(float64)) {
28482848
f(float64(dur))
28492849
}
28502850

2851-
// GetLoadStatsForTesting is for use only by tests to read the Replicas' load
2851+
// TestingGetLoadStats is for use only by tests to read the Replicas' load
28522852
// tracker state.
2853-
func (r *Replica) GetLoadStatsForTesting() *load.ReplicaLoad {
2853+
func (r *Replica) TestingGetLoadStats() *load.ReplicaLoad {
28542854
return r.loadStats
28552855
}
28562856

2857-
// HasOutstandingLearnerSnapshotInFlightForTesting is for use only by tests to
2857+
// TestingHasOutstandingLearnerSnapshotInFlight is for use only by tests to
28582858
// gather whether there are in-flight snapshots to learner replcas.
2859-
func (r *Replica) HasOutstandingLearnerSnapshotInFlightForTesting() bool {
2859+
func (r *Replica) TestingHasOutstandingLearnerSnapshotInFlight() bool {
28602860
return r.errOnOutstandingLearnerSnapshotInflight() != nil
28612861
}
28622862

2863-
// ReadProtectedTimestampsForTesting is for use only by tests to read and update
2863+
// TestingReadProtectedTimestamps is for use only by tests to read and update
28642864
// the Replicas' cached protected timestamp state.
2865-
func (r *Replica) ReadProtectedTimestampsForTesting(ctx context.Context) (err error) {
2865+
func (r *Replica) TestingReadProtectedTimestamps(ctx context.Context) (err error) {
28662866
var ts cachedProtectedTimestampState
28672867
defer r.maybeUpdateCachedProtectedTS(&ts)
28682868
r.mu.RLock()
@@ -2871,29 +2871,26 @@ func (r *Replica) ReadProtectedTimestampsForTesting(ctx context.Context) (err er
28712871
return err
28722872
}
28732873

2874-
// GetMutexForTesting returns the replica's mutex, for use in tests.
2875-
func (r *Replica) GetMutexForTesting() *ReplicaMutex {
2874+
// TestingGetMutex returns the replica's mutex, for use in tests.
2875+
func (r *Replica) TestingGetMutex() *ReplicaMutex {
28762876
return &r.mu.ReplicaMutex
28772877
}
28782878

2879-
// TODO(wenyihu6): rename the *ForTesting functions to be Testing* (see
2880-
// #144119 for more details).
2881-
2882-
// SetCachedClosedTimestampPolicyForTesting sets the closed timestamp policy on r
2879+
// TestingSetCachedClosedTimestampPolicy sets the closed timestamp policy on r
28832880
// to be the given policy. It is a test-only helper method.
2884-
func (r *Replica) SetCachedClosedTimestampPolicyForTesting(policy ctpb.RangeClosedTimestampPolicy) {
2881+
func (r *Replica) TestingSetCachedClosedTimestampPolicy(policy ctpb.RangeClosedTimestampPolicy) {
28852882
r.cachedClosedTimestampPolicy.Store(&policy)
28862883
}
28872884

2888-
// GetCachedClosedTimestampPolicyForTesting returns the closed timestamp policy on r.
2885+
// TestingGetCachedClosedTimestampPolicy returns the closed timestamp policy on r.
28892886
// It is a test-only helper method.
2890-
func (r *Replica) GetCachedClosedTimestampPolicyForTesting() ctpb.RangeClosedTimestampPolicy {
2887+
func (r *Replica) TestingGetCachedClosedTimestampPolicy() ctpb.RangeClosedTimestampPolicy {
28912888
return *r.cachedClosedTimestampPolicy.Load()
28922889
}
28932890

2894-
// RefreshLeaderlessWatcherUnavailableStateForTesting refreshes the replica's
2891+
// TestingRefreshLeaderlessWatcherUnavailableState refreshes the replica's
28952892
// leaderlessWatcher's unavailable state. Intended for tests.
2896-
func (r *Replica) RefreshLeaderlessWatcherUnavailableStateForTesting(
2893+
func (r *Replica) TestingRefreshLeaderlessWatcherUnavailableState(
28972894
ctx context.Context, postTickLead raftpb.PeerID, nowPhysicalTime time.Time, st *cluster.Settings,
28982895
) {
28992896
r.mu.Lock()

pkg/kv/kvserver/replica_closedts_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1112,7 +1112,7 @@ func TestClosedTimestampPolicyRefreshIntervalOnLeaseTransfers(t *testing.T) {
11121112
})
11131113

11141114
// Force repl2 policy to be LAG_BY_CLUSTER_SETTING.
1115-
repl2.SetCachedClosedTimestampPolicyForTesting(ctpb.LAG_BY_CLUSTER_SETTING)
1115+
repl2.TestingSetCachedClosedTimestampPolicy(ctpb.LAG_BY_CLUSTER_SETTING)
11161116
require.Equal(t, roachpb.LAG_BY_CLUSTER_SETTING, repl2.GetRangeInfo(ctx).ClosedTimestampPolicy)
11171117

11181118
// Ensure that transferring the lease to repl2 does trigger a lease refresh.
@@ -1227,7 +1227,7 @@ func TestRefreshPolicyWithVariousLatencies(t *testing.T) {
12271227
repl.RefreshPolicy(tc.latencies)
12281228

12291229
// Verify the policy is set correctly.
1230-
actualPolicy := repl.GetCachedClosedTimestampPolicyForTesting()
1230+
actualPolicy := repl.TestingGetCachedClosedTimestampPolicy()
12311231
require.Equal(t, tc.expectedPolicy, actualPolicy, "expected policy %v, got %v", tc.expectedPolicy, actualPolicy)
12321232
})
12331233
}

pkg/kv/kvserver/replica_learner_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1741,7 +1741,7 @@ func TestLearnerOrJointConfigAdminRelocateRange(t *testing.T) {
17411741
desc := tc.LookupRangeOrFatal(t, scratchStartKey)
17421742
repl, err := tc.GetFirstStoreFromServer(t, 0).GetReplica(desc.RangeID)
17431743
require.NoError(t, err)
1744-
if repl.HasOutstandingLearnerSnapshotInFlightForTesting() {
1744+
if repl.TestingHasOutstandingLearnerSnapshotInFlight() {
17451745
return errors.Errorf("outstanding learner snapshot in flight %s", desc)
17461746
}
17471747
return nil

0 commit comments

Comments
 (0)