Skip to content

Commit

Permalink
kvserver: refactor testing utilities dealing with range splits
Browse files Browse the repository at this point in the history
This commit extracts some existing testing methods to be more general.
so they can be reused for testing merge queue behavior with non-voting
replicas.

Release note: None
  • Loading branch information
aayushshah15 committed Jan 7, 2021
1 parent 82448b8 commit 867340b
Show file tree
Hide file tree
Showing 8 changed files with 206 additions and 122 deletions.
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -525,8 +525,7 @@ SELECT count(*), sum(value) FROM crdb_internal.node_metrics WHERE
numSnapsBefore := numRaftSnaps("before")

doSplit := func(ctx context.Context, _ int) error {
_, _, err := tc.SplitRange(
[]byte(fmt.Sprintf("key-%d", perm[atomic.AddInt32(&idx, 1)])))
_, _, err := tc.SplitRange([]byte(fmt.Sprintf("key-%d", perm[atomic.AddInt32(&idx, 1)])))
return err
}

Expand Down
236 changes: 127 additions & 109 deletions pkg/kv/kvserver/closed_timestamp_test.go

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion pkg/kv/kvserver/constraint/analyzer.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func AnalyzeConstraints(
// matches the conjunction if it matches all of them.
func ConjunctionsCheck(store roachpb.StoreDescriptor, constraints []zonepb.Constraint) bool {
for _, constraint := range constraints {
// StoreSatisfiesConstraint returns whether a store matches the given constraint.
// StoreMatchesConstraint returns whether a store matches the given constraint.
hasConstraint := zonepb.StoreMatchesConstraint(store, constraint)
if (constraint.Type == zonepb.Constraint_REQUIRED && !hasConstraint) ||
(constraint.Type == zonepb.Constraint_PROHIBITED && hasConstraint) {
Expand Down
8 changes: 6 additions & 2 deletions pkg/kv/kvserver/replica_rangefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -772,8 +772,10 @@ func TestReplicaRangefeedPushesTransactions(t *testing.T) {
defer log.Scope(t).Close(t)

ctx := context.Background()
tc, db, _, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, testingCloseFraction, aggressiveResolvedTimestampClusterArgs)
tc, db, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration,
testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv")
defer tc.Stopper().Stop(ctx)
repls := replsForRange(ctx, t, tc, desc, numNodes)

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`)
Expand Down Expand Up @@ -884,8 +886,10 @@ func TestReplicaRangefeedNudgeSlowClosedTimestamp(t *testing.T) {
defer log.Scope(t).Close(t)

ctx := context.Background()
tc, db, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, testingCloseFraction, aggressiveResolvedTimestampClusterArgs)
tc, db, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration,
testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv")
defer tc.Stopper().Stop(ctx)
repls := replsForRange(ctx, t, tc, desc, numNodes)

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`)
Expand Down
17 changes: 13 additions & 4 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -1126,15 +1126,16 @@ func (ts *TestServer) MergeRanges(leftKey roachpb.Key) (roachpb.RangeDescriptor,
return ts.LookupRange(leftKey)
}

// SplitRange splits the range containing splitKey.
// SplitRangeWithExpiration splits the range containing splitKey with a sticky
// bit expiring at expirationTime.
// The right range created by the split starts at the split key and extends to the
// original range's end key.
// Returns the new descriptors of the left and right ranges.
//
// splitKey must correspond to a SQL table key (it must end with a family ID /
// col ID).
func (ts *TestServer) SplitRange(
splitKey roachpb.Key,
func (ts *TestServer) SplitRangeWithExpiration(
splitKey roachpb.Key, expirationTime hlc.Timestamp,
) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) {
ctx := context.Background()
splitRKey, err := keys.Addr(splitKey)
Expand All @@ -1146,7 +1147,7 @@ func (ts *TestServer) SplitRange(
Key: splitKey,
},
SplitKey: splitKey,
ExpirationTime: hlc.MaxTimestamp,
ExpirationTime: expirationTime,
}
_, pErr := kv.SendWrapped(ctx, ts.DB().NonTransactionalSender(), &splitReq)
if pErr != nil {
Expand Down Expand Up @@ -1220,6 +1221,14 @@ func (ts *TestServer) SplitRange(
return leftRangeDesc, rightRangeDesc, nil
}

// SplitRange is exactly like SplitRangeWithExpiration, except that it creates a
// split with a sticky bit that never expires.
func (ts *TestServer) SplitRange(
splitKey roachpb.Key,
) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) {
return ts.SplitRangeWithExpiration(splitKey, hlc.MaxTimestamp)
}

// GetRangeLease returns the current lease for the range containing key, and a
// timestamp taken from the node.
//
Expand Down
17 changes: 16 additions & 1 deletion pkg/testutils/serverutils/test_cluster_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,11 +87,22 @@ type TestClusterInterface interface {
startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) (roachpb.RangeDescriptor, error)

// RemoveNonVoters removes one or more learners from a range.
// AddNonVotersOrFatal is the same as AddNonVoters but will fatal if it fails.
AddNonVotersOrFatal(
t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) roachpb.RangeDescriptor

// RemoveNonVoters removes one or more non-voters from a range.
RemoveNonVoters(
startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) (roachpb.RangeDescriptor, error)

// RemoveNonVotersOrFatal is the same as RemoveNonVoters but will fatal if it
// fails.
RemoveNonVotersOrFatal(
t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) roachpb.RangeDescriptor

// FindRangeLeaseHolder returns the current lease holder for the given range.
// In particular, it returns one particular node's (the hint, if specified) view
// of the current lease.
Expand Down Expand Up @@ -136,6 +147,10 @@ type TestClusterInterface interface {
// as kv scratch space (it doesn't overlap system spans or SQL tables). The
// range is lazily split off on the first call to ScratchRange.
ScratchRange(t testing.TB) roachpb.Key

// WaitForFullReplication waits until all stores in the cluster
// have no ranges with replication pending.
WaitForFullReplication() error
}

// TestClusterFactory encompasses the actual implementation of the shim
Expand Down
4 changes: 1 addition & 3 deletions pkg/testutils/serverutils/test_server_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,9 +175,7 @@ type TestServerInterface interface {
ClusterSettings() *cluster.Settings

// SplitRange splits the range containing splitKey.
SplitRange(
splitKey roachpb.Key,
) (left roachpb.RangeDescriptor, right roachpb.RangeDescriptor, err error)
SplitRange(splitKey roachpb.Key) (left roachpb.RangeDescriptor, right roachpb.RangeDescriptor, err error)

// MergeRanges merges the range containing leftKey with the following adjacent
// range.
Expand Down
41 changes: 41 additions & 0 deletions pkg/testutils/testcluster/testcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -488,6 +488,20 @@ func (tc *TestCluster) LookupRangeOrFatal(t testing.TB, key roachpb.Key) roachpb
return desc
}

// SplitRangeWithExpiration splits the range containing splitKey with a sticky
// bit expiring at expirationTime.
// The right range created by the split starts at the split key and extends to the
// original range's end key.
// Returns the new descriptors of the left and right ranges.
//
// splitKey must correspond to a SQL table key (it must end with a family ID /
// col ID).
func (tc *TestCluster) SplitRangeWithExpiration(
splitKey roachpb.Key, expirationTime hlc.Timestamp,
) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) {
return tc.Servers[0].SplitRangeWithExpiration(splitKey, expirationTime)
}

// SplitRange splits the range containing splitKey.
// The right range created by the split starts at the split key and extends to the
// original range's end key.
Expand Down Expand Up @@ -583,6 +597,17 @@ func (tc *TestCluster) AddNonVoters(
return tc.addReplica(startKey, roachpb.ADD_NON_VOTER, targets...)
}

// AddNonVotersOrFatal is part of TestClusterInterface.
func (tc *TestCluster) AddNonVotersOrFatal(
t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) roachpb.RangeDescriptor {
desc, err := tc.addReplica(startKey, roachpb.ADD_NON_VOTER, targets...)
if err != nil {
t.Fatal(err)
}
return desc
}

// AddVotersMulti is part of TestClusterInterface.
func (tc *TestCluster) AddVotersMulti(
kts ...serverutils.KeyAndTargets,
Expand Down Expand Up @@ -705,6 +730,18 @@ func (tc *TestCluster) RemoveNonVoters(
return tc.changeReplicas(roachpb.REMOVE_NON_VOTER, keys.MustAddr(startKey), targets...)
}

// RemoveNonVotersOrFatal is part of TestClusterInterface.
func (tc *TestCluster) RemoveNonVotersOrFatal(
t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget,
) roachpb.RangeDescriptor {
desc, err := tc.RemoveNonVoters(startKey, targets...)
if err != nil {
t.Fatalf(`could not remove %v replicas from range containing %s: %+v`,
targets, startKey, err)
}
return desc
}

// TransferRangeLease is part of the TestServerInterface.
func (tc *TestCluster) TransferRangeLease(
rangeDesc roachpb.RangeDescriptor, dest roachpb.ReplicationTarget,
Expand Down Expand Up @@ -901,6 +938,10 @@ func (tc *TestCluster) WaitForFullReplication() error {
log.Infof(context.TODO(), "%s has %d underreplicated ranges", s, n)
notReplicated = true
}
if n := s.Metrics().OverReplicatedRangeCount.Value(); n > 0 {
log.Infof(context.TODO(), "%s has %d overreplicated ranges", s, n)
notReplicated = true
}
return nil
})
if err != nil {
Expand Down

0 comments on commit 867340b

Please sign in to comment.