From b43124c7adc8bd5e9d49d6f275e8c52f251dc8ff Mon Sep 17 00:00:00 2001 From: Alex Lunev Date: Thu, 25 Jun 2020 10:22:32 -0700 Subject: [PATCH] kvserver: Replace multiTestContext with TestCluster in client_replica_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 --- pkg/kv/kvserver/client_replica_gc_test.go | 93 ++++++++++++++--------- 1 file changed, 58 insertions(+), 35 deletions(-) diff --git a/pkg/kv/kvserver/client_replica_gc_test.go b/pkg/kv/kvserver/client_replica_gc_test.go index 57d187c1840a..2724bfbc5a8f 100644 --- a/pkg/kv/kvserver/client_replica_gc_test.go +++ b/pkg/kv/kvserver/client_replica_gc_test.go @@ -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" ) @@ -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, @@ -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 { @@ -54,11 +55,12 @@ 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 @@ -66,17 +68,32 @@ func TestReplicaGCQueueDropReplicaDirect(t *testing.T) { 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 @@ -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 @@ -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