Skip to content

Commit

Permalink
kvserver: Replace multiTestContext with TestCluster in client_replica…
Browse files Browse the repository at this point in the history
…_gc_test

Makes progress on cockroachdb#8299

multiTestContext is legacy construct that is deprecated in favor of running
tests via TestCluster. This is one PR out of many to remove the usage of
multiTestContext in the client_replica_gc test cases.

Release note: none
  • Loading branch information
lunevalex committed Jun 26, 2020
1 parent e733a95 commit 4438167
Show file tree
Hide file tree
Showing 3 changed files with 93 additions and 39 deletions.
115 changes: 80 additions & 35 deletions pkg/kv/kvserver/client_replica_gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,17 @@ import (
"context"
"os"
"path/filepath"
"strconv"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
)
Expand All @@ -30,9 +33,32 @@ import (
// immediately cleaned up.
func TestReplicaGCQueueDropReplicaDirect(t *testing.T) {
defer leaktest.AfterTest(t)()
mtc := &multiTestContext{}
const numStores = 3
rangeID := roachpb.RangeID(1)

// Use actual engines (not in memory) because the in-mem ones don't write
// to disk. The test would still pass if we didn't do this except it
// would probably look at an empty sideloaded directory and fail.
tempDir, cleanup := testutils.TempDir(t)
defer cleanup()

testKnobs := kvserver.StoreTestingKnobs{}
var tc *testcluster.TestCluster

serverArgsPerNode := make(map[int]base.TestServerArgs)
for i := 0; i < numStores; i++ {
testServerArgs := base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &testKnobs,
},
StoreSpecs: []base.StoreSpec{
{
Path: filepath.Join(tempDir, strconv.Itoa(i)),
InMemory: false,
},
},
}
serverArgsPerNode[i] = testServerArgs
}

// In this test, the Replica on the second Node is removed, and the test
// verifies that that Node adds this Replica to its RangeGCQueue. However,
Expand All @@ -41,9 +67,7 @@ func TestReplicaGCQueueDropReplicaDirect(t *testing.T) {
// no GC will take place since the consistent RangeLookup hits the first
// Node. We use the TestingEvalFilter to make sure that the second Node
// waits for the first.
cfg := kvserver.TestStoreConfig(nil)
mtc.storeConfig = &cfg
mtc.storeConfig.TestingKnobs.EvalKnobs.TestingEvalFilter =
testKnobs.EvalKnobs.TestingEvalFilter =
func(filterArgs kvserverbase.FilterArgs) *roachpb.Error {
et, ok := filterArgs.Req.(*roachpb.EndTxnRequest)
if !ok || filterArgs.Sid != 2 {
Expand All @@ -54,29 +78,41 @@ func TestReplicaGCQueueDropReplicaDirect(t *testing.T) {
return nil
}
testutils.SucceedsSoon(t, func() error {
r, err := mtc.stores[0].GetReplica(rangeID)
k := tc.ScratchRange(t)
desc, err := tc.LookupRange(k)
if err != nil {
return err
}
if _, ok := r.Desc().GetReplicaDescriptor(2); ok {
if _, ok := desc.GetReplicaDescriptor(2); ok {
return errors.New("expected second node gone from first node's known replicas")
}
return nil
})
return nil
}

defer mtc.Stop()
mtc.Start(t, numStores)

mtc.replicateRange(rangeID, 1, 2)
tc = testcluster.StartTestCluster(t, numStores,
base.TestClusterArgs{
ReplicationMode: base.ReplicationAuto,
ServerArgsPerNode: serverArgsPerNode,
},
)
defer tc.Stopper().Stop(context.Background())

k := tc.ScratchRange(t)
desc := tc.LookupRangeOrFatal(t, k)
ts := tc.Servers[1]
store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID())
if pErr != nil {
t.Fatal(pErr)
}

{
repl1, err := mtc.stores[1].GetReplica(rangeID)
repl1, err := store.GetReplica(desc.RangeID)
if err != nil {
t.Fatal(err)
}
eng := mtc.engines[1]
eng := store.Engine()

// Put some bogus sideloaded data on the replica which we're about to
// remove. Then, at the end of the test, check that that sideloaded
Expand Down Expand Up @@ -112,11 +148,11 @@ func TestReplicaGCQueueDropReplicaDirect(t *testing.T) {
}()
}

mtc.unreplicateRange(rangeID, 1)
desc = tc.RemoveReplicasOrFatal(t, k, tc.Target(1))

// Make sure the range is removed from the store.
testutils.SucceedsSoon(t, func() error {
if _, err := mtc.stores[1].GetReplica(rangeID); !testutils.IsError(err, "r[0-9]+ was not found") {
if _, err := store.GetReplica(desc.RangeID); !testutils.IsError(err, "r[0-9]+ was not found") {
return errors.Errorf("expected range removal: %v", err) // NB: errors.Wrapf(nil, ...) returns nil.
}
return nil
Expand All @@ -127,40 +163,49 @@ func TestReplicaGCQueueDropReplicaDirect(t *testing.T) {
// removes a range from a store that no longer should have a replica.
func TestReplicaGCQueueDropReplicaGCOnScan(t *testing.T) {
defer leaktest.AfterTest(t)()
mtc := &multiTestContext{}
cfg := kvserver.TestStoreConfig(nil)
cfg.TestingKnobs.DisableEagerReplicaRemoval = true
cfg.Clock = nil // manual clock
mtc.storeConfig = &cfg

defer mtc.Stop()
mtc.Start(t, 3)

tc := testcluster.StartTestCluster(t, 3,
base.TestClusterArgs{
ReplicationMode: base.ReplicationAuto,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
DisableEagerReplicaRemoval: true,
// Override the garbage collection threshold to something small,
// so this test can trigger the GC without relying on moving time.
ReplicaGCQueueInactivityThreshold: time.Millisecond * 100,
},
},
},
},
)
defer tc.Stopper().Stop(context.Background())

ts := tc.Servers[1]
store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID())
if pErr != nil {
t.Fatal(pErr)
}
// Disable the replica gc queue to prevent direct removal of replica.
mtc.stores[1].SetReplicaGCQueueActive(false)
store.SetReplicaGCQueueActive(false)

rangeID := roachpb.RangeID(1)
mtc.replicateRange(rangeID, 1, 2)
mtc.unreplicateRange(rangeID, 1)
k := tc.ScratchRange(t)
desc := tc.RemoveReplicasOrFatal(t, k, tc.Target(1))

// Wait long enough for the direct replica GC to have had a chance and been
// discarded because the queue is disabled.
time.Sleep(10 * time.Millisecond)
if _, err := mtc.stores[1].GetReplica(rangeID); err != nil {
if _, err := store.GetReplica(desc.RangeID); err != nil {
t.Error("unexpected range removal")
}

// Enable the queue.
mtc.stores[1].SetReplicaGCQueueActive(true)

// Increment the clock's timestamp to make the replica GC queue process the range.
mtc.advanceClock(context.Background())
mtc.manualClock.Increment(int64(kvserver.ReplicaGCQueueInactivityThreshold + 1))
store.SetReplicaGCQueueActive(true)

// Make sure the range is removed from the store.
testutils.SucceedsSoon(t, func() error {
store := mtc.stores[1]
store.MustForceReplicaGCScanAndProcess()
if _, err := store.GetReplica(rangeID); !testutils.IsError(err, "r[0-9]+ was not found") {
if _, err := store.GetReplica(desc.RangeID); !testutils.IsError(err, "r[0-9]+ was not found") {
return errors.Errorf("expected range removal: %v", err) // NB: errors.Wrapf(nil, ...) returns nil.
}
return nil
Expand Down
14 changes: 10 additions & 4 deletions pkg/kv/kvserver/replica_gc_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,15 +87,21 @@ func makeReplicaGCQueueMetrics() ReplicaGCQueueMetrics {
// ranges that have been rebalanced away from this store.
type replicaGCQueue struct {
*baseQueue
metrics ReplicaGCQueueMetrics
db *kv.DB
metrics ReplicaGCQueueMetrics
db *kv.DB
replicaGCQueueInactivityThreshold time.Duration
}

// newReplicaGCQueue returns a new instance of replicaGCQueue.
func newReplicaGCQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *replicaGCQueue {
replicaGCQueueInactivityThreshold := ReplicaGCQueueInactivityThreshold
if store.TestingKnobs().ReplicaGCQueueInactivityThreshold != 0 {
replicaGCQueueInactivityThreshold = store.TestingKnobs().ReplicaGCQueueInactivityThreshold
}
rgcq := &replicaGCQueue{
metrics: makeReplicaGCQueueMetrics(),
db: db,
metrics: makeReplicaGCQueueMetrics(),
db: db,
replicaGCQueueInactivityThreshold: replicaGCQueueInactivityThreshold,
}
store.metrics.registry.AddMetricStruct(&rgcq.metrics)
rgcq.baseQueue = newBaseQueue(
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,9 @@ type StoreTestingKnobs struct {
// RangeFeedPushTxnsAge overrides the default value for
// rangefeed.Config.PushTxnsAge.
RangeFeedPushTxnsAge time.Duration
// ReplicaGCQueueInactivityThreshold overrides the default value for
// replica_gc_queue.ReplicaGCQueueInactivityThreshold.
ReplicaGCQueueInactivityThreshold time.Duration
}

// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
Expand Down

0 comments on commit 4438167

Please sign in to comment.