From cf83835038791a3d24115eec5da5bd31788e8363 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Tue, 28 Aug 2018 23:40:07 -0400 Subject: [PATCH] *: disable merge queue in more tests Take a more aggressive approach to disabling the merge queue in tests. This commit disables the merge queue in all tests that send a manual AdminSplit request, unless those tests were explicitly verified to be safe to run with the merge queue on. Release note: None --- pkg/ccl/storageccl/import_test.go | 2 + pkg/kv/dist_sender_server_test.go | 39 ++++++++------- pkg/kv/split_test.go | 6 ++- pkg/kv/txn_coord_sender_test.go | 6 ++- pkg/kv/txn_test.go | 7 ++- pkg/server/intent_test.go | 5 ++ pkg/server/node_test.go | 6 +++ pkg/server/server_test.go | 18 ++++++- pkg/sql/distsqlplan/span_resolver_test.go | 5 ++ pkg/storage/client_lease_test.go | 3 ++ pkg/storage/client_merge_test.go | 30 +++++++++-- pkg/storage/client_metrics_test.go | 6 ++- pkg/storage/client_raft_test.go | 29 +++++++++-- pkg/storage/client_replica_test.go | 9 +++- pkg/storage/client_split_test.go | 50 +++++++++++++++++-- pkg/storage/client_status_test.go | 7 ++- pkg/storage/consistency_queue_test.go | 5 ++ pkg/storage/ts_maintenance_queue_test.go | 1 + pkg/testutils/testcluster/testcluster_test.go | 7 +++ 19 files changed, 201 insertions(+), 40 deletions(-) diff --git a/pkg/ccl/storageccl/import_test.go b/pkg/ccl/storageccl/import_test.go index 6110d971d882..7ebfb302ce27 100644 --- a/pkg/ccl/storageccl/import_test.go +++ b/pkg/ccl/storageccl/import_test.go @@ -216,6 +216,8 @@ func runTestImport(t *testing.T, init func(*cluster.Settings)) { return roachpb.NewError(roachpb.NewAmbiguousResultError(strconv.Itoa(int(r)))) }, }, + // Prevent the merge queue from immediately discarding our splits. + DisableMergeQueue: true, }} ctx := context.Background() diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index 7416fe138fa2..1a5e683155ed 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -50,11 +50,12 @@ import ( // starting a TestServer, which creates a "real" node and employs a // distributed sender server-side. -func startNoSplitServer(t *testing.T) (serverutils.TestServerInterface, *client.DB) { +func startNoSplitMergeServer(t *testing.T) (serverutils.TestServerInterface, *client.DB) { s, _, db := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &storage.StoreTestingKnobs{ DisableSplitQueue: true, + DisableMergeQueue: true, }, }, }) @@ -67,7 +68,7 @@ func startNoSplitServer(t *testing.T) (serverutils.TestServerInterface, *client. // index record being read. func TestRangeLookupWithOpenTransaction(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) // Create an intent on the meta1 record by writing directly to the @@ -367,7 +368,7 @@ func checkReverseScanResults( // bounds. func TestMultiRangeBoundedBatchScan(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) ctx := context.TODO() @@ -558,7 +559,7 @@ func TestMultiRangeBoundedBatchScan(t *testing.T) { // contain two partial responses. func TestMultiRangeBoundedBatchScanUnsortedOrder(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) @@ -597,7 +598,7 @@ func TestMultiRangeBoundedBatchScanUnsortedOrder(t *testing.T) { // contain two partial responses. func TestMultiRangeBoundedBatchScanSortedOverlapping(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) @@ -666,7 +667,7 @@ func checkResumeSpanDelRangeResults( // Tests a batch of bounded DelRange() requests. func TestMultiRangeBoundedBatchDelRange(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) @@ -734,7 +735,7 @@ func TestMultiRangeBoundedBatchDelRange(t *testing.T) { // ResumeSpan. func TestMultiRangeBoundedBatchDelRangeBoundary(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) @@ -780,7 +781,7 @@ func TestMultiRangeBoundedBatchDelRangeBoundary(t *testing.T) { // overlap. func TestMultiRangeBoundedBatchDelRangeOverlappingKeys(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) @@ -846,7 +847,7 @@ func TestMultiRangeBoundedBatchDelRangeOverlappingKeys(t *testing.T) { // truncation. In that case, the request is skipped. func TestMultiRangeEmptyAfterTruncate(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) db := s.DB() @@ -869,7 +870,7 @@ func TestMultiRangeEmptyAfterTruncate(t *testing.T) { // TestMultiRequestBatchWithFwdAndReverseRequests are disallowed. func TestMultiRequestBatchWithFwdAndReverseRequests(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) db := s.DB() @@ -891,7 +892,7 @@ func TestMultiRequestBatchWithFwdAndReverseRequests(t *testing.T) { // DeleteRange and ResolveIntentRange work across ranges. func TestMultiRangeScanReverseScanDeleteResolve(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) db := s.DB() @@ -955,7 +956,7 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { roachpb.INCONSISTENT, } { t.Run(rc.String(), func(t *testing.T) { - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) db := s.DB() @@ -1041,7 +1042,7 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { // dist sender. func TestParallelSender(t *testing.T) { defer leaktest.AfterTest(t)() - s, db := startNoSplitServer(t) + s, db := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) ctx := context.TODO() @@ -1110,7 +1111,7 @@ func initReverseScanTestEnv(s serverutils.TestServerInterface, t *testing.T) *cl // on a single range. func TestSingleRangeReverseScan(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) db := initReverseScanTestEnv(s, t) ctx := context.TODO() @@ -1155,7 +1156,7 @@ func TestSingleRangeReverseScan(t *testing.T) { // across multiple ranges. func TestMultiRangeReverseScan(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) db := initReverseScanTestEnv(s, t) ctx := context.TODO() @@ -1181,7 +1182,7 @@ func TestMultiRangeReverseScan(t *testing.T) { func TestStopAtRangeBoundary(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) ctx := context.TODO() defer s.Stopper().Stop(ctx) @@ -1526,7 +1527,7 @@ func TestStopAtRangeBoundary(t *testing.T) { // #12603 for more details. func TestBatchPutWithConcurrentSplit(t *testing.T) { defer leaktest.AfterTest(t)() - s, db := startNoSplitServer(t) + s, db := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) // Split first using the default client and scan to make sure that @@ -1579,7 +1580,7 @@ func TestBatchPutWithConcurrentSplit(t *testing.T) { // across multiple ranges while range splits and merges happen. func TestReverseScanWithSplitAndMerge(t *testing.T) { defer leaktest.AfterTest(t)() - s, _ := startNoSplitServer(t) + s, _ := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) db := initReverseScanTestEnv(s, t) @@ -1610,7 +1611,7 @@ func TestReverseScanWithSplitAndMerge(t *testing.T) { func TestBadRequest(t *testing.T) { defer leaktest.AfterTest(t)() - s, db := startNoSplitServer(t) + s, db := startNoSplitMergeServer(t) defer s.Stopper().Stop(context.TODO()) ctx := context.TODO() diff --git a/pkg/kv/split_test.go b/pkg/kv/split_test.go index 328beba4a6df..d617ba4c27b2 100644 --- a/pkg/kv/split_test.go +++ b/pkg/kv/split_test.go @@ -238,7 +238,11 @@ func TestRangeSplitsWithWritePressure(t *testing.T) { // on the same splitKey succeeds. func TestRangeSplitsWithSameKeyTwice(t *testing.T) { defer leaktest.AfterTest(t)() - s := createTestDB(t) + s := createTestDBWithContextAndKnobs(t, client.DefaultDBContext(), &storage.StoreTestingKnobs{ + DisableScanner: true, + DisableSplitQueue: true, + DisableMergeQueue: true, + }) defer s.Stop() ctx := context.TODO() diff --git a/pkg/kv/txn_coord_sender_test.go b/pkg/kv/txn_coord_sender_test.go index aa9dfcf88169..c977d2c33c5d 100644 --- a/pkg/kv/txn_coord_sender_test.go +++ b/pkg/kv/txn_coord_sender_test.go @@ -289,7 +289,11 @@ func TestTxnCoordSenderCondenseIntentSpans(t *testing.T) { // Test that the theartbeat loop detects aborted transactions and stops. func TestTxnCoordSenderHeartbeat(t *testing.T) { defer leaktest.AfterTest(t)() - s := createTestDB(t) + s := createTestDBWithContextAndKnobs(t, client.DefaultDBContext(), &storage.StoreTestingKnobs{ + DisableScanner: true, + DisableSplitQueue: true, + DisableMergeQueue: true, + }) defer s.Stop() ctx := context.Background() diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index d6fd2e60c031..9ff4bbab5145 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/tscache" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -473,7 +474,11 @@ func TestTxnLongDelayBetweenWritesWithConcurrentRead(t *testing.T) { // See issue #676 for full details about original bug. func TestTxnRepeatGetWithRangeSplit(t *testing.T) { defer leaktest.AfterTest(t)() - s := createTestDB(t) + s := createTestDBWithContextAndKnobs(t, client.DefaultDBContext(), &storage.StoreTestingKnobs{ + DisableScanner: true, + DisableSplitQueue: true, + DisableMergeQueue: true, + }) defer s.Stop() keyA := roachpb.Key("a") diff --git a/pkg/server/intent_test.go b/pkg/server/intent_test.go index d0209c153b58..0ec6af63371c 100644 --- a/pkg/server/intent_test.go +++ b/pkg/server/intent_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) +// TODO(benesch): move this test to somewhere more specific than package server. func TestIntentResolution(t *testing.T) { defer leaktest.AfterTest(t)() @@ -111,7 +112,11 @@ func TestIntentResolution(t *testing.T) { } return nil } + // Prevent the merge queue from immediately discarding our splits. + storeKnobs.DisableMergeQueue = true + // TODO(benesch): starting a test server for every test case is needlessly + // inefficient. s, _, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{Store: &storeKnobs}}) defer s.Stopper().Stop(context.TODO()) diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 22dde3a8d84d..d084cdc67e24 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -546,6 +546,12 @@ func TestNodeStatusWritten(t *testing.T) { // ======================================== srv, _, kvDB := serverutils.StartServer(t, base.TestServerArgs{ DisableEventLog: true, + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{ + // Prevent the merge queue from immediately discarding our splits. + DisableMergeQueue: true, + }, + }, }) defer srv.Stopper().Stop(context.TODO()) ts := srv.(*TestServer) diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index ef15a2516489..d74bfbfd9a60 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -302,7 +302,14 @@ func TestAcceptEncoding(t *testing.T) { // ranges are carried out properly. func TestMultiRangeScanDeleteRange(t *testing.T) { defer leaktest.AfterTest(t)() - s, _, db := serverutils.StartServer(t, base.TestServerArgs{}) + s, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{ + // Prevent the merge queue from immediately discarding our splits. + DisableMergeQueue: true, + }, + }, + }) defer s.Stopper().Stop(context.TODO()) ts := s.(*TestServer) tds := db.NonTransactionalSender() @@ -398,7 +405,14 @@ func TestMultiRangeScanWithMaxResults(t *testing.T) { for i, tc := range testCases { t.Run("", func(t *testing.T) { ctx := context.Background() - s, _, db := serverutils.StartServer(t, base.TestServerArgs{}) + s, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{ + // Prevent the merge queue from immediately discarding our splits. + DisableMergeQueue: true, + }, + }, + }) defer s.Stopper().Stop(ctx) ts := s.(*TestServer) tds := db.NonTransactionalSender() diff --git a/pkg/sql/distsqlplan/span_resolver_test.go b/pkg/sql/distsqlplan/span_resolver_test.go index d3be6f3e2099..54611398fd84 100644 --- a/pkg/sql/distsqlplan/span_resolver_test.go +++ b/pkg/sql/distsqlplan/span_resolver_test.go @@ -331,6 +331,11 @@ func setupRanges( } } + // Prevent the merge queue from immediately discarding our splits. + if _, err := db.Exec("SET CLUSTER SETTING kv.range_merge.queue_enabled = false"); err != nil { + t.Fatal(err) + } + tableDesc := sqlbase.GetTableDescriptor(cdb, "t", "test") // Split every SQL row to its own range. rowRanges := make([]roachpb.RangeDescriptor, len(values)) diff --git a/pkg/storage/client_lease_test.go b/pkg/storage/client_lease_test.go index 0fc827c72fd6..26eddd9f5fef 100644 --- a/pkg/storage/client_lease_test.go +++ b/pkg/storage/client_lease_test.go @@ -38,6 +38,7 @@ func TestStoreRangeLease(t *testing.T) { testutils.RunTrueAndFalse(t, "enableEpoch", func(t *testing.T, enableEpoch bool) { sc := storage.TestStoreConfig(nil) + sc.TestingKnobs.DisableMergeQueue = true sc.EnableEpochRangeLeases = enableEpoch mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() @@ -104,6 +105,7 @@ func TestStoreRangeLease(t *testing.T) { func TestStoreRangeLeaseSwitcheroo(t *testing.T) { defer leaktest.AfterTest(t)() sc := storage.TestStoreConfig(nil) + sc.TestingKnobs.DisableMergeQueue = true sc.EnableEpochRangeLeases = true mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() @@ -169,6 +171,7 @@ func TestStoreRangeLeaseSwitcheroo(t *testing.T) { func TestStoreGossipSystemData(t *testing.T) { defer leaktest.AfterTest(t)() sc := storage.TestStoreConfig(nil) + sc.TestingKnobs.DisableMergeQueue = true sc.EnableEpochRangeLeases = true mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() diff --git a/pkg/storage/client_merge_test.go b/pkg/storage/client_merge_test.go index d535cc0701da..9e1d5531096a 100644 --- a/pkg/storage/client_merge_test.go +++ b/pkg/storage/client_merge_test.go @@ -85,7 +85,9 @@ func TestStoreRangeMergeTwoEmptyRanges(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - var mtc multiTestContext + storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 1) defer mtc.Stop() store := mtc.Store(0) @@ -136,7 +138,9 @@ func TestStoreRangeMergeMetadataCleanup(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - var mtc multiTestContext + storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 1) defer mtc.Stop() store := mtc.Store(0) @@ -220,6 +224,7 @@ func mergeWithData(t *testing.T, retries int64) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableReplicateQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true // Maybe inject some retryable errors when the merge transaction commits. var mtc *multiTestContext @@ -388,7 +393,9 @@ func TestStoreRangeMergeTimestampCache(t *testing.T) { func mergeCheckingTimestampCaches(t *testing.T, disjointLeaseholders bool) { ctx := context.Background() - var mtc multiTestContext + storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{storeConfig: &storeCfg} var lhsStore, rhsStore *storage.Store if disjointLeaseholders { mtc.Start(t, 2) @@ -504,6 +511,7 @@ func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} var readTS hlc.Timestamp rhsKey := roachpb.Key("c") @@ -624,6 +632,7 @@ func TestStoreRangeMergeTxnFailure(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true // Install a store filter that maybe injects retryable errors into a merge // transaction before ultimately aborting the merge. @@ -711,7 +720,9 @@ func TestStoreRangeMergeStats(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 1) defer mtc.Stop() store := mtc.Store(0) @@ -807,6 +818,7 @@ func TestStoreRangeMergeInFlightTxns(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableReplicateQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 1) defer mtc.Stop() @@ -989,6 +1001,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableReplicateQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true // The synchronization in this test is tricky. The merge transaction is // controlled by the AdminMerge function and normally commits quite quickly, @@ -1143,6 +1156,7 @@ func TestStoreRangeMergeConcurrentRequests(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.DisableReplicateQueue = true var mtc *multiTestContext @@ -1282,6 +1296,7 @@ func TestStoreReplicaGCAfterMerge(t *testing.T) { storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableReplicateQueue = true storeCfg.TestingKnobs.DisableReplicaGCQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 2) defer mtc.Stop() @@ -1439,6 +1454,7 @@ func TestStoreRangeMergeAddReplicaRace(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.DisableReplicateQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} @@ -1538,6 +1554,7 @@ func TestStoreRangeMergeSlowAbandonedFollower(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.DisableReplicateQueue = true storeCfg.TestingKnobs.DisableReplicaGCQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} @@ -1608,6 +1625,7 @@ func TestStoreRangeMergeAbandonedFollowers(t *testing.T) { storeCfg.TestingKnobs.DisableReplicateQueue = true storeCfg.TestingKnobs.DisableReplicaGCQueue = true storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 3) defer mtc.Stop() @@ -1697,6 +1715,7 @@ func TestStoreRangeMergeDeadFollower(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 3) defer mtc.Stop() @@ -1725,6 +1744,7 @@ func TestStoreRangeMergeReadoptedBothFollowers(t *testing.T) { storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableReplicateQueue = true storeCfg.TestingKnobs.DisableReplicaGCQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 3) defer mtc.Stop() @@ -1833,6 +1853,7 @@ func TestStoreRangeMergeReadoptedLHSFollower(t *testing.T) { storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableReplicateQueue = true storeCfg.TestingKnobs.DisableReplicaGCQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &storeCfg} mtc.Start(t, 3) defer mtc.Stop() @@ -2016,6 +2037,7 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { ctx := context.Background() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.DisableReplicateQueue = true // Install a filter that triggers a shutdown when stop is non-zero and the diff --git a/pkg/storage/client_metrics_test.go b/pkg/storage/client_metrics_test.go index 351800108347..115d26e066db 100644 --- a/pkg/storage/client_metrics_test.go +++ b/pkg/storage/client_metrics_test.go @@ -237,7 +237,11 @@ func TestStoreResolveMetrics(t *testing.T) { func TestStoreMetrics(t *testing.T) { defer leaktest.AfterTest(t)() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 3) diff --git a/pkg/storage/client_raft_test.go b/pkg/storage/client_raft_test.go index 0599ccf30f33..caf8106e8b6c 100644 --- a/pkg/storage/client_raft_test.go +++ b/pkg/storage/client_raft_test.go @@ -71,6 +71,7 @@ func TestStoreRecoverFromEngine(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true const rangeID = roachpb.RangeID(1) splitKey := roachpb.Key("m") @@ -917,6 +918,7 @@ func TestReplicateAfterRemoveAndSplit(t *testing.T) { defer leaktest.AfterTest(t)() sc := storage.TestStoreConfig(nil) + sc.TestingKnobs.DisableMergeQueue = true // Disable the replica GC queue so that it doesn't accidentally pick up the // removed replica and GC it. We'll explicitly enable it later in the test. sc.TestingKnobs.DisableReplicaGCQueue = true @@ -2278,7 +2280,11 @@ outer: // is not KeyMin replicating to a fresh store can apply snapshots correctly. func TestReplicateAfterSplit(t *testing.T) { defer leaktest.AfterTest(t)() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 2) @@ -2351,7 +2357,11 @@ func TestReplicaRemovalCampaign(t *testing.T) { for i, td := range testData { func() { - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 2) @@ -2428,7 +2438,11 @@ func TestReplicaRemovalCampaign(t *testing.T) { // a remote node correctly after the Replica was removed from the Store. func TestRaftAfterRemoveRange(t *testing.T) { defer leaktest.AfterTest(t)() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 3) @@ -3256,6 +3270,7 @@ func TestReplicaLazyLoad(t *testing.T) { sc.RaftTickInterval = 10 * time.Millisecond // safe because there is only a single node sc.TestingKnobs.DisableScanner = true sc.TestingKnobs.DisablePeriodicGossips = true + sc.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() mtc.Start(t, 1) @@ -3484,6 +3499,7 @@ func TestTransferRaftLeadership(t *testing.T) { const numStores = 3 sc := storage.TestStoreConfig(nil) + sc.TestingKnobs.DisableMergeQueue = true // Suppress timeout-based elections (which also includes a previous // leader stepping down due to a quorum check). Running tests on a // heavily loaded CPU is enough to reach the raft election timeout @@ -3613,6 +3629,7 @@ func TestRaftBlockedReplica(t *testing.T) { defer leaktest.AfterTest(t)() sc := storage.TestStoreConfig(nil) + sc.TestingKnobs.DisableMergeQueue = true sc.TestingKnobs.DisableScanner = true mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() @@ -3737,7 +3754,11 @@ func TestRangeQuiescence(t *testing.T) { // lease points to a different replica. func TestInitRaftGroupOnRequest(t *testing.T) { defer leaktest.AfterTest(t)() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 2) diff --git a/pkg/storage/client_replica_test.go b/pkg/storage/client_replica_test.go index b1a4effd9e82..37100d2a4abc 100644 --- a/pkg/storage/client_replica_test.go +++ b/pkg/storage/client_replica_test.go @@ -359,6 +359,7 @@ func TestRangeLookupUseReverse(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -953,6 +954,7 @@ func TestLeaseMetricsOnSplitAndTransfer(t *testing.T) { var injectLeaseTransferError atomic.Value sc := storage.TestStoreConfig(nil) sc.TestingKnobs.DisableSplitQueue = true + sc.TestingKnobs.DisableMergeQueue = true sc.TestingKnobs.EvalKnobs.TestingEvalFilter = func(filterArgs storagebase.FilterArgs) *roachpb.Error { if args, ok := filterArgs.Req.(*roachpb.TransferLeaseRequest); ok { @@ -1379,7 +1381,11 @@ func TestErrorHandlingForNonKVCommand(t *testing.T) { func TestRangeInfo(t *testing.T) { defer leaktest.AfterTest(t)() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 2) @@ -1562,6 +1568,7 @@ func TestCampaignOnLazyRaftGroupInitialization(t *testing.T) { }{} sc := storage.TestStoreConfig(nil) sc.EnableEpochRangeLeases = false // simpler to test with + sc.TestingKnobs.DisableMergeQueue = true sc.TestingKnobs.DontPreventUseOfOldLeaseOnStart = true sc.TestingKnobs.OnCampaign = func(r *storage.Replica) { if !r.DescLocked().StartKey.Equal(keys.UserTableDataMin) { diff --git a/pkg/storage/client_split_test.go b/pkg/storage/client_split_test.go index 89dd62172651..993a24d31bf8 100644 --- a/pkg/storage/client_split_test.go +++ b/pkg/storage/client_split_test.go @@ -81,6 +81,7 @@ func TestStoreRangeSplitAtIllegalKeys(t *testing.T) { cfg := storage.TestStoreConfig(nil) cfg.TestingKnobs.DisableSplitQueue = true + cfg.TestingKnobs.DisableMergeQueue = true store := createTestStoreWithConfig(t, stopper, cfg) for _, key := range []roachpb.Key{ @@ -108,6 +109,7 @@ func TestStoreSplitAbortSpan(t *testing.T) { clock := hlc.NewClock(manualClock.UnixNano, time.Millisecond) storeCfg := storage.TestStoreConfig(clock) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) @@ -226,6 +228,7 @@ func TestStoreRangeSplitAtTablePrefix(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -281,6 +284,7 @@ func TestStoreRangeSplitInsideRow(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -340,6 +344,7 @@ func TestStoreRangeSplitIntents(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -409,6 +414,7 @@ func TestStoreRangeSplitAtRangeBounds(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -451,6 +457,7 @@ func TestStoreRangeSplitConcurrent(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -505,6 +512,7 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -657,6 +665,7 @@ func TestStoreRangeSplitStats(t *testing.T) { manual := hlc.NewManualClock(123) storeCfg := storage.TestStoreConfig(hlc.NewClock(manual.UnixNano, time.Nanosecond)) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -783,11 +792,12 @@ func TestStoreEmptyRangeSnapshotSize(t *testing.T) { ctx := context.Background() - // Disable the replicate queue and the split queue, as we want to control both - // rebalancing and splits ourselves. + // Disable the replicate queue, the split queue, and the merge queue as we + // want to control both rebalancing, splits, and merges ourselves. sc := storage.TestStoreConfig(nil) sc.TestingKnobs.DisableReplicateQueue = true sc.TestingKnobs.DisableSplitQueue = true + sc.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() @@ -967,7 +977,9 @@ func TestStoreZoneUpdateAndRangeSplit(t *testing.T) { defer leaktest.AfterTest(t)() stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - store, _ := createTestStore(t, stopper) + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + store := createTestStoreWithConfig(t, stopper, storeCfg) config.TestingSetupZoneConfigHook(stopper) const maxBytes = 1 << 16 @@ -1023,7 +1035,9 @@ func TestStoreRangeSplitWithMaxBytesUpdate(t *testing.T) { defer leaktest.AfterTest(t)() stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) - store, _ := createTestStore(t, stopper) + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + store := createTestStoreWithConfig(t, stopper, storeCfg) config.TestingSetupZoneConfigHook(stopper) origRng := store.LookupReplica(roachpb.RKeyMin) @@ -1097,6 +1111,7 @@ func TestStoreRangeSplitBackpressureWrites(t *testing.T) { splitPending, blockSplits := make(chan struct{}), make(chan struct{}) storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableGCQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.TestingRequestFilter = func(ba roachpb.BatchRequest) *roachpb.Error { for _, req := range ba.Requests { @@ -1207,6 +1222,8 @@ func TestStoreRangeSystemSplits(t *testing.T) { defer leaktest.AfterTest(t)() stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) + // Intentionally leave the merge queue enabled. This indrectly tests that the + // merge queue respects these split points. store, _ := createTestStore(t, stopper) userTableMax := keys.MinUserDescID + 4 @@ -1329,6 +1346,8 @@ func runSetupSplitSnapshotRace( // Async intent resolution can sometimes lead to hangs when we stop // most of the stores at the end of this function. sc.TestingKnobs.DisableAsyncIntentResolution = true + // Avoid fighting with the merge queue while trying to reproduce this race. + sc.TestingKnobs.DisableMergeQueue = true mtc := &multiTestContext{storeConfig: &sc} defer mtc.Stop() mtc.Start(t, 6) @@ -1559,6 +1578,7 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { EvalKnobs: batcheval.TestingKnobs{ TestingEvalFilter: filter, }, + DisableMergeQueue: true, } tc := testcluster.StartTestCluster(t, 2, args) @@ -1740,6 +1760,7 @@ func TestStoreSplitOnRemovedReplica(t *testing.T) { args.ReplicationMode = base.ReplicationManual args.ServerArgs.Knobs.Store = &storage.StoreTestingKnobs{ TestingRequestFilter: filter, + DisableMergeQueue: true, } tc := testcluster.StartTestCluster(t, 3, args) @@ -1829,6 +1850,7 @@ func TestStoreSplitFailsAfterMaxRetries(t *testing.T) { args.ReplicationMode = base.ReplicationManual args.ServerArgs.Knobs.Store = &storage.StoreTestingKnobs{ TestingRequestFilter: filter, + DisableMergeQueue: true, } tc := testcluster.StartTestCluster(t, 1, args) @@ -1859,6 +1881,7 @@ func TestStoreSplitGCThreshold(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -1922,6 +1945,7 @@ func TestStoreRangeSplitRaceUninitializedRHS(t *testing.T) { defer leaktest.AfterTest(t)() mtc := &multiTestContext{} storeCfg := storage.TestStoreConfig(nil) + storeCfg.TestingKnobs.DisableMergeQueue = true // An aggressive tick interval lets groups communicate more and thus // triggers test failures much more reliably. We can't go too aggressive // or race tests never make any progress. @@ -2051,6 +2075,7 @@ func TestStoreRangeSplitRaceUninitializedRHS(t *testing.T) { func TestLeaderAfterSplit(t *testing.T) { defer leaktest.AfterTest(t)() storeConfig := storage.TestStoreConfig(nil) + storeConfig.TestingKnobs.DisableMergeQueue = true storeConfig.RaftElectionTimeoutTicks = 1000000 mtc := &multiTestContext{ storeConfig: &storeConfig, @@ -2198,6 +2223,7 @@ func TestStoreSplitBeginTxnPushMetaIntentRace(t *testing.T) { manual := hlc.NewManualClock(123) storeCfg := storage.TestStoreConfig(hlc.NewClock(manual.UnixNano, time.Nanosecond)) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.EvalKnobs.TestingEvalFilter = func(filterArgs storagebase.FilterArgs) *roachpb.Error { startMu.Lock() start := startMu.time @@ -2305,6 +2331,7 @@ func TestStoreRangeGossipOnSplits(t *testing.T) { // changes in the number of leases also triggering store gossip. storeCfg.TestingKnobs.DisableLeaseCapacityGossip = true storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.DisableScanner = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) @@ -2379,6 +2406,7 @@ func TestStoreTxnWaitQueueEnabledOnSplit(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -2401,6 +2429,7 @@ func TestDistributedTxnCleanup(t *testing.T) { defer leaktest.AfterTest(t)() storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) store := createTestStoreWithConfig(t, stopper, storeCfg) @@ -2516,6 +2545,7 @@ func TestUnsplittableRange(t *testing.T) { splitQueuePurgatoryChan := make(chan time.Time, 1) cfg := storage.TestStoreConfig(hlc.NewClock(manual.UnixNano, time.Nanosecond)) cfg.TestingKnobs.SplitQueuePurgatoryChan = splitQueuePurgatoryChan + cfg.TestingKnobs.DisableMergeQueue = true store := createTestStoreWithConfig(t, stopper, cfg) if err := server.WaitForInitialSplits(store.DB()); err != nil { t.Fatal(err) @@ -2589,6 +2619,7 @@ func TestTxnWaitQueueDependencyCycleWithRangeSplit(t *testing.T) { storeCfg := storage.TestStoreConfig(nil) storeCfg.TestingKnobs.DisableSplitQueue = true + storeCfg.TestingKnobs.DisableMergeQueue = true storeCfg.TestingKnobs.EvalKnobs.TestingEvalFilter = func(filterArgs storagebase.FilterArgs) *roachpb.Error { if _, ok := filterArgs.Req.(*roachpb.PushTxnRequest); ok { @@ -2696,6 +2727,7 @@ func TestStoreCapacityAfterSplit(t *testing.T) { manualClock := hlc.NewManualClock(123) cfg := storage.TestStoreConfig(hlc.NewClock(manualClock.UnixNano, time.Nanosecond)) cfg.TestingKnobs.DisableSplitQueue = true + cfg.TestingKnobs.DisableMergeQueue = true s := createTestStoreWithConfig(t, stopper, cfg) cap, err := s.Capacity(false /* useCached */) @@ -2798,7 +2830,13 @@ func TestRangeLookupAfterMeta2Split(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{ + DisableMergeQueue: true, + }, + }, + }) s := srv.(*server.TestServer) defer s.Stopper().Stop(ctx) @@ -2922,6 +2960,7 @@ func TestStoreSplitRangeLookupRace(t *testing.T) { Knobs: base.TestingKnobs{ Store: &storage.StoreTestingKnobs{ DisableSplitQueue: true, + DisableMergeQueue: true, ForceSyncIntentResolution: true, TestingResponseFilter: respFilter, }, @@ -2991,6 +3030,7 @@ func TestRangeLookupAsyncResolveIntent(t *testing.T) { cfg := storage.TestStoreConfig(nil) cfg.TestingKnobs.ForceSyncIntentResolution = true cfg.TestingKnobs.DisableSplitQueue = true + cfg.TestingKnobs.DisableMergeQueue = true cfg.TestingKnobs.TestingProposalFilter = func(args storagebase.ProposalFilterArgs) *roachpb.Error { for _, union := range args.Req.Requests { diff --git a/pkg/storage/client_status_test.go b/pkg/storage/client_status_test.go index e8b7c69910e5..6ec1d6c493fb 100644 --- a/pkg/storage/client_status_test.go +++ b/pkg/storage/client_status_test.go @@ -22,13 +22,18 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) func TestComputeStatsForKeySpan(t *testing.T) { defer leaktest.AfterTest(t)() - mtc := &multiTestContext{} + storeCfg := storage.TestStoreConfig(nil /* clock */) + storeCfg.TestingKnobs.DisableMergeQueue = true + mtc := &multiTestContext{ + storeConfig: &storeCfg, + } defer mtc.Stop() mtc.Start(t, 3) diff --git a/pkg/storage/consistency_queue_test.go b/pkg/storage/consistency_queue_test.go index a7c67b3c2f18..08bd294c7207 100644 --- a/pkg/storage/consistency_queue_test.go +++ b/pkg/storage/consistency_queue_test.go @@ -300,6 +300,11 @@ func TestConsistencyQueueRecomputeStats(t *testing.T) { ScanInterval: time.Second, ScanMinIdleTime: 0, ScanMaxIdleTime: 100 * time.Millisecond, + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{ + DisableMergeQueue: true, + }, + }, } nodeZeroArgs := tsArgs nodeZeroArgs.StoreSpecs = []base.StoreSpec{{ diff --git a/pkg/storage/ts_maintenance_queue_test.go b/pkg/storage/ts_maintenance_queue_test.go index 5f2a6b2b4799..cc2658a7058f 100644 --- a/pkg/storage/ts_maintenance_queue_test.go +++ b/pkg/storage/ts_maintenance_queue_test.go @@ -105,6 +105,7 @@ func TestTimeSeriesMaintenanceQueue(t *testing.T) { cfg.TimeSeriesDataStore = model cfg.TestingKnobs.DisableScanner = true cfg.TestingKnobs.DisableSplitQueue = true + cfg.TestingKnobs.DisableMergeQueue = true stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) diff --git a/pkg/testutils/testcluster/testcluster_test.go b/pkg/testutils/testcluster/testcluster_test.go index d9b043020dc0..10e21f3261aa 100644 --- a/pkg/testutils/testcluster/testcluster_test.go +++ b/pkg/testutils/testcluster/testcluster_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/httputil" @@ -40,6 +41,12 @@ func TestManualReplication(t *testing.T) { ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ UseDatabase: "t", + Knobs: base.TestingKnobs{ + Store: &storage.StoreTestingKnobs{ + // Prevent the merge queue from immediately discarding our splits. + DisableMergeQueue: true, + }, + }, }, }) defer tc.Stopper().Stop(context.TODO())