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 #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 29, 2020
1 parent 74d7021 commit b43124c
Showing 1 changed file with 58 additions and 35 deletions.
93 changes: 58 additions & 35 deletions pkg/kv/kvserver/client_replica_gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,13 @@ import (
"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 +32,10 @@ import (
// immediately cleaned up.
func TestReplicaGCQueueDropReplicaDirect(t *testing.T) {
defer leaktest.AfterTest(t)()
mtc := &multiTestContext{}
const numStores = 3
rangeID := roachpb.RangeID(1)

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

// 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 +44,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 +55,45 @@ 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,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &testKnobs,
},
},
},
)
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 +129,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 +144,46 @@ 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,
},
},
},
},
)
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

0 comments on commit b43124c

Please sign in to comment.