From 40625298cb80161c30e70ccbea4ec5bbad8705f5 Mon Sep 17 00:00:00 2001 From: Pavel Kalinnikov Date: Fri, 30 Sep 2022 14:30:07 +0100 Subject: [PATCH 1/5] kvserver: use GetFirstStoreFromServer in test Release note: None --- pkg/kv/kvserver/consistency_queue_test.go | 48 +++++------------------ 1 file changed, 10 insertions(+), 38 deletions(-) diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index 3d73aab0a400..81a752760162 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -113,11 +113,7 @@ func TestCheckConsistencyMultiStore(t *testing.T) { ) defer tc.Stopper().Stop(context.Background()) - ts := tc.Servers[0] - store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } + store := tc.GetFirstStoreFromServer(t, 0) // Write something to the DB. putArgs := putArgs([]byte("a"), []byte("b")) @@ -199,11 +195,7 @@ func TestCheckConsistencyReplay(t *testing.T) { defer tc.Stopper().Stop(context.Background()) - ts := tc.Servers[0] - store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } + store := tc.GetFirstStoreFromServer(t, 0) checkArgs := roachpb.CheckConsistencyRequest{ RequestHeader: roachpb.RequestHeader{ Key: []byte("a"), @@ -306,11 +298,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // s2 (index 1) will panic. notifyFatal := make(chan struct{}, 1) testKnobs.ConsistencyTestingKnobs.OnBadChecksumFatal = func(s roachpb.StoreIdent) { - ts := tc.Servers[1] - store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } + store := tc.GetFirstStoreFromServer(t, 1) if s != *store.Ident { t.Errorf("OnBadChecksumFatal called from %v", s) return @@ -345,11 +333,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { ) defer tc.Stopper().Stop(context.Background()) - ts := tc.Servers[0] - store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } + store := tc.GetFirstStoreFromServer(t, 0) // Write something to the DB. pArgs := putArgs([]byte("a"), []byte("b")) if _, err := kv.SendWrapped(context.Background(), store.DB().NonTransactionalSender(), pArgs); err != nil { @@ -377,17 +361,13 @@ func TestCheckConsistencyInconsistent(t *testing.T) { } onDiskCheckpointPaths := func(nodeIdx int) []string { - testServer := tc.Servers[nodeIdx] fs, pErr := stickyEngineRegistry.GetUnderlyingFS( base.StoreSpec{StickyInMemoryEngineID: strconv.FormatInt(int64(nodeIdx), 10)}) if pErr != nil { t.Fatal(pErr) } - testStore, pErr := testServer.Stores().GetStore(testServer.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } - checkpointPath := filepath.Join(testStore.Engine().GetAuxiliaryDir(), "checkpoints") + store := tc.GetFirstStoreFromServer(t, nodeIdx) + checkpointPath := filepath.Join(store.Engine().GetAuxiliaryDir(), "checkpoints") checkpoints, _ := fs.List(checkpointPath) var checkpointPaths []string for _, cpDirName := range checkpoints { @@ -414,14 +394,10 @@ func TestCheckConsistencyInconsistent(t *testing.T) { } // Write some arbitrary data only to store 1. Inconsistent key "e"! - ts1 := tc.Servers[1] - store1, pErr := ts1.Stores().GetStore(ts1.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } + store1 := tc.GetFirstStoreFromServer(t, 1) var val roachpb.Value val.SetInt(42) - diffTimestamp = ts.Clock().Now() + diffTimestamp = tc.Server(0).Clock().Now() if err := storage.MVCCPut( context.Background(), store1.Engine(), nil, diffKey, diffTimestamp, hlc.ClockTimestamp{}, val, nil, ); err != nil { @@ -674,18 +650,14 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { } // Force a run of the consistency queue, otherwise it might take a while. - ts := tc.Servers[0] - store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) - if pErr != nil { - t.Fatal(pErr) - } + store := tc.GetFirstStoreFromServer(t, 0) if err := store.ForceConsistencyQueueProcess(); err != nil { t.Fatal(err) } // The stats should magically repair themselves. We'll first do a quick check // and then a full recomputation. - repl, _, err := ts.Stores().GetReplicaForRangeID(ctx, rangeID) + repl, _, err := tc.Servers[0].Stores().GetReplicaForRangeID(ctx, rangeID) if err != nil { t.Fatal(err) } From 47a0843fb80f8bd000a55dc2aba4df54eba29747 Mon Sep 17 00:00:00 2001 From: Pavel Kalinnikov Date: Fri, 30 Sep 2022 14:32:45 +0100 Subject: [PATCH 2/5] kvserver: use close(channel) idiom for signaling Release note: None --- pkg/kv/kvserver/consistency_queue_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index 81a752760162..ff2bed94cc00 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -250,7 +250,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // write (s3 agrees with s1). diffKey := []byte("e") var diffTimestamp hlc.Timestamp - notifyReportDiff := make(chan struct{}, 1) + notifyReportDiff := make(chan struct{}) testKnobs.ConsistencyTestingKnobs.BadChecksumReportDiff = func(s roachpb.StoreIdent, diff kvserver.ReplicaSnapshotDiffSlice) { rangeDesc := tc.LookupRangeOrFatal(t, diffKey) @@ -293,17 +293,17 @@ func TestCheckConsistencyInconsistent(t *testing.T) { t.Errorf("expected:\n%s\ngot:\n%s", exp, act) } - notifyReportDiff <- struct{}{} + close(notifyReportDiff) } // s2 (index 1) will panic. - notifyFatal := make(chan struct{}, 1) + notifyFatal := make(chan struct{}) testKnobs.ConsistencyTestingKnobs.OnBadChecksumFatal = func(s roachpb.StoreIdent) { store := tc.GetFirstStoreFromServer(t, 1) if s != *store.Ident { t.Errorf("OnBadChecksumFatal called from %v", s) return } - notifyFatal <- struct{}{} + close(notifyFatal) } serverArgsPerNode := make(map[int]base.TestServerArgs) From 37e05b0efa0d7eb5928909bc5a6d620c6dda4edc Mon Sep 17 00:00:00 2001 From: Pavel Kalinnikov Date: Fri, 30 Sep 2022 14:39:46 +0100 Subject: [PATCH 3/5] kvserver: inline and unindent test setup code Release note: None --- pkg/kv/kvserver/consistency_queue_test.go | 31 +++++++++-------------- 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index ff2bed94cc00..ca3a6e74753e 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -306,31 +306,24 @@ func TestCheckConsistencyInconsistent(t *testing.T) { close(notifyFatal) } - serverArgsPerNode := make(map[int]base.TestServerArgs) + serverArgsPerNode := make(map[int]base.TestServerArgs, numStores) for i := 0; i < numStores; i++ { - testServerArgs := base.TestServerArgs{ + serverArgsPerNode[i] = base.TestServerArgs{ Knobs: base.TestingKnobs{ - Store: &testKnobs, - Server: &server.TestingKnobs{ - StickyEngineRegistry: stickyEngineRegistry, - }, - }, - StoreSpecs: []base.StoreSpec{ - { - InMemory: true, - StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), - }, + Store: &testKnobs, + Server: &server.TestingKnobs{StickyEngineRegistry: stickyEngineRegistry}, }, + StoreSpecs: []base.StoreSpec{{ + InMemory: true, + StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + }}, } - serverArgsPerNode[i] = testServerArgs } - tc = testcluster.StartTestCluster(t, numStores, - base.TestClusterArgs{ - ReplicationMode: base.ReplicationAuto, - ServerArgsPerNode: serverArgsPerNode, - }, - ) + tc = testcluster.StartTestCluster(t, numStores, base.TestClusterArgs{ + ReplicationMode: base.ReplicationAuto, + ServerArgsPerNode: serverArgsPerNode, + }) defer tc.Stopper().Stop(context.Background()) store := tc.GetFirstStoreFromServer(t, 0) From 339f165ea2ddb13d887ab104bb5ecd61d03e0025 Mon Sep 17 00:00:00 2001 From: Pavel Kalinnikov Date: Fri, 30 Sep 2022 14:47:39 +0100 Subject: [PATCH 4/5] kvserver: use require package in tests Release note: None --- pkg/kv/kvserver/consistency_queue_test.go | 66 +++++++---------------- 1 file changed, 18 insertions(+), 48 deletions(-) diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index ca3a6e74753e..35c63dc1cac0 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -254,15 +254,11 @@ func TestCheckConsistencyInconsistent(t *testing.T) { testKnobs.ConsistencyTestingKnobs.BadChecksumReportDiff = func(s roachpb.StoreIdent, diff kvserver.ReplicaSnapshotDiffSlice) { rangeDesc := tc.LookupRangeOrFatal(t, diffKey) - repl, pErr := tc.FindRangeLeaseHolder(rangeDesc, nil) - if pErr != nil { - t.Fatal(pErr) - } + repl, err := tc.FindRangeLeaseHolder(rangeDesc, nil) + require.NoError(t, err) // Servers start at 0, but NodeID starts at 1. - store, pErr := tc.Servers[repl.NodeID-1].Stores().GetStore(repl.StoreID) - if pErr != nil { - t.Fatal(pErr) - } + store, err := tc.Servers[repl.NodeID-1].Stores().GetStore(repl.StoreID) + require.NoError(t, err) if s != *store.Ident { t.Errorf("BadChecksumReportDiff called from follower (StoreIdent = %v)", s) return @@ -347,18 +343,14 @@ func TestCheckConsistencyInconsistent(t *testing.T) { Mode: roachpb.ChecksumMode_CHECK_VIA_QUEUE, } resp, err := kv.SendWrapped(context.Background(), store.DB().NonTransactionalSender(), &checkArgs) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err.GoError()) return resp.(*roachpb.CheckConsistencyResponse) } onDiskCheckpointPaths := func(nodeIdx int) []string { - fs, pErr := stickyEngineRegistry.GetUnderlyingFS( + fs, err := stickyEngineRegistry.GetUnderlyingFS( base.StoreSpec{StickyInMemoryEngineID: strconv.FormatInt(int64(nodeIdx), 10)}) - if pErr != nil { - t.Fatal(pErr) - } + require.NoError(t, err) store := tc.GetFirstStoreFromServer(t, nodeIdx) checkpointPath := filepath.Join(store.Engine().GetAuxiliaryDir(), "checkpoints") checkpoints, _ := fs.List(checkpointPath) @@ -391,11 +383,8 @@ func TestCheckConsistencyInconsistent(t *testing.T) { var val roachpb.Value val.SetInt(42) diffTimestamp = tc.Server(0).Clock().Now() - if err := storage.MVCCPut( - context.Background(), store1.Engine(), nil, diffKey, diffTimestamp, hlc.ClockTimestamp{}, val, nil, - ); err != nil { - t.Fatal(err) - } + require.NoError(t, storage.MVCCPut(context.Background(), store1.Engine(), nil, + diffKey, diffTimestamp, hlc.ClockTimestamp{}, val, nil)) // Run consistency check again, this time it should find something. resp := runConsistencyCheck() @@ -518,9 +507,7 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { RequestHeader: roachpb.RequestHeader{Key: key}, DryRun: true, }) - if err := db.Run(ctx, &b); err != nil { - t.Fatal(err) - } + require.NoError(t, db.Run(ctx, &b)) resp := b.RawResponse().Responses[0].GetInner().(*roachpb.RecomputeStatsResponse) delta := enginepb.MVCCStats(resp.AddedDelta) delta.AgeTo(0) @@ -536,9 +523,7 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { // Split off a range so that we get away from the timeseries writes, which // pollute the stats with ContainsEstimates=true. Note that the split clears // the right hand side (which is what we operate on) from that flag. - if err := db0.AdminSplit(ctx, key, hlc.MaxTimestamp /* expirationTime */); err != nil { - t.Fatal(err) - } + require.NoError(t, db0.AdminSplit(ctx, key, hlc.MaxTimestamp /* expirationTime */)) delta := computeDelta(db0) @@ -547,9 +532,7 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { } rangeDesc, err := tc.LookupRange(key) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) return rangeDesc.RangeID }() @@ -561,16 +544,12 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { storage.Filesystem(path), storage.CacheSize(1<<20 /* 1 MiB */), storage.MustExist) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) defer eng.Close() rsl := stateloader.Make(rangeID) ms, err := rsl.LoadMVCCStats(ctx, eng) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) // Put some garbage in the stats that we're hoping the consistency queue will // trigger a removal of via RecomputeStats. SysCount was chosen because it is @@ -585,9 +564,7 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { // Overwrite with the new stats; remember that this range hasn't upreplicated, // so the consistency checker won't see any replica divergence when it runs, // but it should definitely see that its recomputed stats mismatch. - if err := rsl.SetMVCCStats(ctx, eng, &ms); err != nil { - t.Fatal(err) - } + require.NoError(t, rsl.SetMVCCStats(ctx, eng, &ms)) }() // Now that we've tampered with the stats, restart the cluster and extend it @@ -618,10 +595,7 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { // can starve the actual work to be done. done := time.After(5 * time.Second) for { - if err := db0.Put(ctx, fmt.Sprintf("%s%d", key, rand.Int63()), "ballast"); err != nil { - t.Error(err) - } - + require.NoError(t, db0.Put(ctx, fmt.Sprintf("%s%d", key, rand.Int63()), "ballast")) select { case <-ctx.Done(): return @@ -644,16 +618,12 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { // Force a run of the consistency queue, otherwise it might take a while. store := tc.GetFirstStoreFromServer(t, 0) - if err := store.ForceConsistencyQueueProcess(); err != nil { - t.Fatal(err) - } + require.NoError(t, store.ForceConsistencyQueueProcess()) // The stats should magically repair themselves. We'll first do a quick check // and then a full recomputation. repl, _, err := tc.Servers[0].Stores().GetReplicaForRangeID(ctx, rangeID) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) ms := repl.GetMVCCStats() if ms.SysCount >= sysCountGarbage { t.Fatalf("still have a SysCount of %d", ms.SysCount) From 8f5b18afafd0b765daf99b1f9349188a0d439d85 Mon Sep 17 00:00:00 2001 From: Pavel Kalinnikov Date: Fri, 30 Sep 2022 15:18:41 +0100 Subject: [PATCH 5/5] kvserver: use assert package in tests Release note: None --- pkg/kv/kvserver/consistency_queue_test.go | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index 35c63dc1cac0..325b94dfa618 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -259,12 +259,10 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // Servers start at 0, but NodeID starts at 1. store, err := tc.Servers[repl.NodeID-1].Stores().GetStore(repl.StoreID) require.NoError(t, err) - if s != *store.Ident { - t.Errorf("BadChecksumReportDiff called from follower (StoreIdent = %v)", s) + if !assert.Equal(t, *store.Ident, s) { return } - if len(diff) != 1 { - t.Errorf("diff length = %d, diff = %v", len(diff), diff) + if !assert.Len(t, diff, 1) { return } d := diff[0] @@ -284,19 +282,14 @@ func TestCheckConsistencyInconsistent(t *testing.T) { + value:"\x00\x00\x00\x00\x01T" + raw mvcc_key/value: 6500000000000000007b000003db0d 000000000154 ` - if act != exp { - // We already logged the actual one above. - t.Errorf("expected:\n%s\ngot:\n%s", exp, act) - } - + assert.Equal(t, exp, act) close(notifyReportDiff) } // s2 (index 1) will panic. notifyFatal := make(chan struct{}) testKnobs.ConsistencyTestingKnobs.OnBadChecksumFatal = func(s roachpb.StoreIdent) { store := tc.GetFirstStoreFromServer(t, 1) - if s != *store.Ident { - t.Errorf("OnBadChecksumFatal called from %v", s) + if !assert.Equal(t, *store.Ident, s) { return } close(notifyFatal)