From 1b0e1979baaa58e68f52126d19fce17157de7bc0 Mon Sep 17 00:00:00 2001 From: Stan Rosenberg Date: Thu, 29 Sep 2022 20:42:57 +0000 Subject: [PATCH 1/7] codeowners: add test-eng to owners of pkg/workload Add test-eng as a code owner/watcher for pkg/workload. In light of recent and future improvements [1], [2], TestEng would prefer to be in sync with all changes to the workload code. Over time, the team plans to build expertise in this area. [1] https://github.com/cockroachdb/cockroach/pull/88362 [2] https://github.com/cockroachdb/cockroach/issues/88566 Release note: None Release justification: test only change --- .github/CODEOWNERS | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 947c188b21ba..62650205f686 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -223,7 +223,7 @@ /pkg/ccl/testccl/workload/schemachange/ @cockroachdb/sql-schema /pkg/ccl/testutilsccl/ @cockroachdb/test-eng-noreview /pkg/ccl/utilccl/ @cockroachdb/unowned @cockroachdb/server-prs -/pkg/ccl/workloadccl/ @cockroachdb/sql-experience-noreview +/pkg/ccl/workloadccl/ @cockroachdb/sql-experience-noreview @cockroachdb/test-eng /pkg/ccl/benchccl/rttanalysisccl/ @cockroachdb/sql-experience /pkg/clusterversion/ @cockroachdb/kv-prs-noreview /pkg/cmd/allocsim/ @cockroachdb/kv-prs @@ -286,7 +286,7 @@ /pkg/cmd/uptodate/ @cockroachdb/dev-inf /pkg/cmd/urlcheck/ @cockroachdb/docs /pkg/cmd/whoownsit/ @cockroachdb/test-eng -/pkg/cmd/workload/ @cockroachdb/sql-experience-noreview +/pkg/cmd/workload/ @cockroachdb/sql-experience-noreview @cockroachdb/test-eng /pkg/cmd/wraprules/ @cockroachdb/obs-inf-prs-noreview /pkg/cmd/zerosum/ @cockroachdb/kv-noreview /pkg/col/ @cockroachdb/sql-queries @@ -360,7 +360,7 @@ /pkg/util/admission/ @cockroachdb/admission-control /pkg/util/schedulerlatency/ @cockroachdb/admission-control /pkg/util/tracing @cockroachdb/obs-inf-prs -/pkg/workload/ @cockroachdb/sql-experience-noreview +/pkg/workload/ @cockroachdb/sql-experience-noreview @cockroachdb/test-eng /pkg/obs/ @cockroachdb/obs-inf-prs /pkg/obsservice/ @cockroachdb/obs-inf-prs From ee3fe1df334c4913d33dc29d4c80016ba41f7491 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 19:35:15 +0000 Subject: [PATCH 2/7] kvserver: add `SmallEngineBlocks` testing knob This patch adds a store testing knob `SmallEngineBlocks` which configures Pebble with a block size of 1 byte. This will store every key in a separate block, which can provoke bugs in time-bound iterators. Release note: None --- pkg/kv/kvserver/testing_knobs.go | 5 +++++ pkg/server/config.go | 21 ++++++++++++++++----- pkg/server/sticky_engine.go | 5 +++++ pkg/storage/mvcc_history_test.go | 6 +----- pkg/storage/open.go | 19 +++++++++++++++++++ 5 files changed, 46 insertions(+), 10 deletions(-) diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 3b38d0f655ef..49dc2cbbe434 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -442,6 +442,11 @@ type StoreTestingKnobs struct { // MVCCGCQueueLeaseCheckInterceptor intercepts calls to Replica.LeaseStatusAt when // making high priority replica scans. MVCCGCQueueLeaseCheckInterceptor func(ctx context.Context, replica *Replica, now hlc.ClockTimestamp) bool + + // SmallEngineBlocks will configure the engine with a very small block size of + // 1 byte, resulting in each key having its own block. This can provoke bugs + // in time-bound iterators. + SmallEngineBlocks bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/server/config.go b/pkg/server/config.go index 9f22c7b988f8..f11b1df2da21 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -606,8 +606,11 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { tableCache = pebble.NewTableCache(pebbleCache, runtime.GOMAXPROCS(0), totalFileLimit) } - skipSizeCheck := cfg.TestingKnobs.Store != nil && - cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs).SkipMinSizeCheck + var storeKnobs kvserver.StoreTestingKnobs + if s := cfg.TestingKnobs.Store; s != nil { + storeKnobs = *s.(*kvserver.StoreTestingKnobs) + } + for i, spec := range cfg.Stores.Specs { log.Eventf(ctx, "initializing %+v", spec) var sizeInBytes = spec.Size.InBytes @@ -619,7 +622,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { } sizeInBytes = int64(float64(sysMem) * spec.Size.Percent / 100) } - if sizeInBytes != 0 && !skipSizeCheck && sizeInBytes < base.MinimumStoreSize { + if sizeInBytes != 0 && !storeKnobs.SkipMinSizeCheck && sizeInBytes < base.MinimumStoreSize { return Engines{}, errors.Errorf("%f%% of memory is only %s bytes, which is below the minimum requirement of %s", spec.Size.Percent, humanizeutil.IBytes(sizeInBytes), humanizeutil.IBytes(base.MinimumStoreSize)) } @@ -650,7 +653,9 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { storage.CacheSize(cfg.CacheSize), storage.MaxSize(sizeInBytes), storage.EncryptionAtRest(spec.EncryptionOptions), - storage.Settings(cfg.Settings)) + storage.Settings(cfg.Settings), + storage.If(storeKnobs.SmallEngineBlocks, storage.BlockSize(1)), + ) if err != nil { return Engines{}, err } @@ -667,7 +672,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { if spec.Size.Percent > 0 { sizeInBytes = int64(float64(du.TotalBytes) * spec.Size.Percent / 100) } - if sizeInBytes != 0 && !skipSizeCheck && sizeInBytes < base.MinimumStoreSize { + if sizeInBytes != 0 && !storeKnobs.SkipMinSizeCheck && sizeInBytes < base.MinimumStoreSize { return Engines{}, errors.Errorf("%f%% of %s's total free space is only %s bytes, which is below the minimum requirement of %s", spec.Size.Percent, spec.Path, humanizeutil.IBytes(sizeInBytes), humanizeutil.IBytes(base.MinimumStoreSize)) } @@ -695,6 +700,12 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { pebbleConfig.Opts.Experimental.CPUWorkPermissionGranter = &cpuWorkPermissionGranter{ cfg.SoftSlotGranter, } + if storeKnobs.SmallEngineBlocks { + for i := range pebbleConfig.Opts.Levels { + pebbleConfig.Opts.Levels[i].BlockSize = 1 + pebbleConfig.Opts.Levels[i].IndexBlockSize = 1 + } + } // If the spec contains Pebble options, set those too. if len(spec.PebbleOptions) > 0 { err := pebbleConfig.Opts.Parse(spec.PebbleOptions, &pebble.ParseHooks{ diff --git a/pkg/server/sticky_engine.go b/pkg/server/sticky_engine.go index 41e78ffe7a1a..968a413d5d4b 100644 --- a/pkg/server/sticky_engine.go +++ b/pkg/server/sticky_engine.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -138,6 +139,10 @@ func (registry *stickyInMemEnginesRegistryImpl) GetOrCreateStickyInMemEngine( storage.ForStickyEngineTesting, } + if s := cfg.TestingKnobs.Store; s != nil && s.(*kvserver.StoreTestingKnobs).SmallEngineBlocks { + options = append(options, storage.BlockSize(1)) + } + log.Infof(ctx, "creating new sticky in-mem engine %s", spec.StickyInMemoryEngineID) engine := storage.InMemFromFS(ctx, fs, "", options...) diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index b9ac1f8e5636..08d015121637 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -167,12 +167,8 @@ func TestMVCCHistories(t *testing.T) { // If enabled by metamorphic parameter, use very small blocks to provoke TBI // optimization. We'll also flush after each command. if separateEngineBlocks && !disableSeparateEngineBlocks { - engineOpts = append(engineOpts, func(cfg *engineConfig) error { + engineOpts = append(engineOpts, BlockSize(1), func(cfg *engineConfig) error { cfg.Opts.DisableAutomaticCompactions = true - for i := range cfg.Opts.Levels { - cfg.Opts.Levels[i].BlockSize = 1 - cfg.Opts.Levels[i].IndexBlockSize = 1 - } return nil }) } diff --git a/pkg/storage/open.go b/pkg/storage/open.go index 3d6da78711d1..fbc62ed7646b 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -101,6 +101,17 @@ func MaxSize(size int64) ConfigOption { } } +// BlockSize sets the engine block size, primarily for testing purposes. +func BlockSize(size int) ConfigOption { + return func(cfg *engineConfig) error { + for i := range cfg.Opts.Levels { + cfg.Opts.Levels[i].BlockSize = size + cfg.Opts.Levels[i].IndexBlockSize = size + } + return nil + } +} + // MaxWriterConcurrency sets the concurrency of the sstable Writers. A concurrency // of 0 implies no parallelism in the Writer, and a concurrency of 1 or more implies // parallelism in the Writer. Currently, there's no difference between a concurrency @@ -170,6 +181,14 @@ func Hook(hookFunc func(*base.StorageConfig) error) ConfigOption { } } +// If enables the given option if enable is true. +func If(enable bool, opt ConfigOption) ConfigOption { + if enable { + return opt + } + return func(cfg *engineConfig) error { return nil } +} + // A Location describes where the storage engine's data will be written. A // Location may be in-memory or on the filesystem. type Location struct { From e0e59112d60b9657600fc9c4641763536282018f Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 19:39:51 +0000 Subject: [PATCH 3/7] sql/logictest: add metamorphic test param for small engine blocks Uses a Pebble block size of 1 byte, to provoke bugs in time-bound iterators. Release note: None --- pkg/sql/logictest/logic.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 5ca2a4217b86..23d04afec8b1 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -551,6 +551,10 @@ var ( useMVCCRangeTombstonesForPointDeletes = util.ConstantWithMetamorphicTestBool( "logictest-use-mvcc-range-tombstones-for-point-deletes", false) + // smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke + // bugs in time-bound iterators. + smallEngineBlocks = util.ConstantWithMetamorphicTestBool("logictest-small-engine-blocks", false) + // BackupRestoreProbability is the environment variable for `3node-backup` config. backupRestoreProbability = envutil.EnvOrDefaultFloat64("COCKROACH_LOGIC_TEST_BACKUP_RESTORE_PROBABILITY", 0.0) ) @@ -1243,6 +1247,7 @@ func (t *logicTest) newCluster( UseRangeTombstonesForPointDeletes: supportsMVCCRangeTombstones && useMVCCRangeTombstonesForPointDeletes, }, + SmallEngineBlocks: smallEngineBlocks, }, SQLEvalContext: &eval.TestingKnobs{ AssertBinaryExprReturnTypes: true, From 33d3c21fe7881c0620444f5c123200f8708d67ea Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 19:50:59 +0000 Subject: [PATCH 4/7] kvserver/rangefeed: add metamorphic test param for small engine blocks Uses a Pebble block size of 1 byte, to provoke bugs in time-bound iterators. Release note: None --- pkg/kv/kvclient/rangefeed/BUILD.bazel | 1 + .../rangefeed/rangefeed_external_test.go | 25 +++++++++++++++++-- pkg/kv/kvserver/rangefeed/BUILD.bazel | 1 + .../kvserver/rangefeed/catchup_scan_test.go | 11 +++++--- 4 files changed, 33 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvclient/rangefeed/BUILD.bazel b/pkg/kv/kvclient/rangefeed/BUILD.bazel index 7103f774f392..459ddc79f8a5 100644 --- a/pkg/kv/kvclient/rangefeed/BUILD.bazel +++ b/pkg/kv/kvclient/rangefeed/BUILD.bazel @@ -82,6 +82,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/storageutils", "//pkg/testutils/testcluster", + "//pkg/util", "//pkg/util/ctxgroup", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index a33a50528b55..2f0034ed8f2c 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -41,6 +42,10 @@ import ( var ( pointKV = storageutils.PointKV rangeKV = storageutils.RangeKV + + // smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke + // bugs in time-bound iterators. + smallEngineBlocks = util.ConstantWithMetamorphicTestBool("small-engine-blocks", false) ) type kvs = storageutils.KVs @@ -564,7 +569,15 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, + }, + }, + }, + }) defer tc.Stopper().Stop(ctx) srv := tc.Server(0) db := srv.DB() @@ -653,7 +666,15 @@ func TestWithOnDeleteRange(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, + }, + }, + }, + }) defer tc.Stopper().Stop(ctx) srv := tc.Server(0) db := srv.DB() diff --git a/pkg/kv/kvserver/rangefeed/BUILD.bazel b/pkg/kv/kvserver/rangefeed/BUILD.bazel index 2ecebc5d8d1f..69c07b22f184 100644 --- a/pkg/kv/kvserver/rangefeed/BUILD.bazel +++ b/pkg/kv/kvserver/rangefeed/BUILD.bazel @@ -62,6 +62,7 @@ go_test( "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/testutils/skip", + "//pkg/util", "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/leaktest", diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index 95acf1917c7c..fde52ff241e4 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -26,6 +27,10 @@ import ( "github.com/stretchr/testify/require" ) +// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke +// bugs in time-bound iterators. +var smallEngineBlocks = util.ConstantWithMetamorphicTestBool("small-engine-blocks", false) + // TODO(erikgrinaker): This should be migrated to a data-driven test harness for // end-to-end rangefeed testing, with more exhaustive test cases. See: // https://github.com/cockroachdb/cockroach/issues/82715 @@ -91,7 +96,7 @@ func TestCatchupScan(t *testing.T) { kv2_2_2 := makeKTV(testKey2, ts2, testValue2) kv2_5_3 := makeKTV(testKey2, ts5, testValue3) - eng := storage.NewDefaultInMemForTesting() + eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1))) defer eng.Close() // Put with no intent. for _, kv := range []storage.MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1, kv2_2_2, kv2_5_3} { @@ -141,7 +146,7 @@ func TestCatchupScanInlineError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - eng := storage.NewDefaultInMemForTesting() + eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1))) defer eng.Close() // Write an inline value. @@ -167,7 +172,7 @@ func TestCatchupScanSeesOldIntent(t *testing.T) { // [#85886]: https://github.com/cockroachdb/cockroach/issues/85886 ctx := context.Background() - eng := storage.NewDefaultInMemForTesting() + eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1))) defer eng.Close() // b -> version @ 1100 (visible) From b2a6b80920324bd6b31cba9a6f622961979de600 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Thu, 29 Sep 2022 13:50:27 -0400 Subject: [PATCH 5/7] jobs: Clear out claim info when pausing Clear out job claim information when job is paused. Clearing out claim information is beneficial since it allows operator to pause/resume job if they want to try to move job coordinator to another node. Release note: none --- pkg/jobs/adopt.go | 4 +++- pkg/jobs/jobs_test.go | 25 +++++++++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index 57057c9370f3..da5da78e548a 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -432,7 +432,9 @@ const pauseAndCancelUpdate = ` ELSE status END, num_runs = 0, - last_run = NULL + last_run = NULL, + claim_session_id = NULL, + claim_instance_id = NULL WHERE (status IN ('` + string(StatusPauseRequested) + `', '` + string(StatusCancelRequested) + `')) AND ((claim_session_id = $1) AND (claim_instance_id = $2)) RETURNING id, status diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 9783c1e91e63..782d75c26c3a 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -3138,9 +3138,31 @@ func TestPauseReason(t *testing.T) { _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */) require.NoError(t, err) + countRowsWithClaimInfo := func() int { + t.Helper() + n := 0 + tdb.QueryRow(t, + "SELECT count(*) FROM system.jobs "+ + "WHERE id = $1 AND (claim_session_id IS NOT NULL OR claim_instance_id IS NOT NULL)", + jobID).Scan(&n) + return n + } + mustNotHaveClaim := func() { + require.Equal(t, 0, countRowsWithClaimInfo()) + } + mustHaveClaim := func() { + testutils.SucceedsSoon(t, func() error { + if countRowsWithClaimInfo() == 1 { + return nil + } + return errors.New("still waiting for claim info") + }) + } + // First wait until the job is running q := fmt.Sprintf("SELECT status FROM system.jobs WHERE id = %d", jobID) tdb.CheckQueryResultsRetry(t, q, [][]string{{"running"}}) + mustHaveClaim() getStatusAndPayload := func(t *testing.T, id jobspb.JobID) (string, jobspb.Payload) { var payloadBytes []byte @@ -3164,6 +3186,7 @@ func TestPauseReason(t *testing.T) { require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "for testing")) tdb.CheckQueryResultsRetry(t, q, [][]string{{"paused"}}) checkStatusAndPauseReason(t, jobID, "paused", "for testing") + mustNotHaveClaim() } { @@ -3172,12 +3195,14 @@ func TestPauseReason(t *testing.T) { tdb.CheckQueryResultsRetry(t, q, [][]string{{"running"}}) checkStatusAndPauseReason(t, jobID, "running", "for testing") + mustHaveClaim() } { // Pause the job again with a different reason. Verify that the job is paused with the reason. require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "second time")) tdb.CheckQueryResultsRetry(t, q, [][]string{{"paused"}}) checkStatusAndPauseReason(t, jobID, "paused", "second time") + mustNotHaveClaim() } } From 248cabddd0ea30efa1e952fc578a566b215cea9a Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 19:51:57 +0000 Subject: [PATCH 6/7] kvserver/gc: add metamorphic test param for small engine blocks Uses a Pebble block size of 1 byte, to provoke bugs in time-bound iterators. Release note: None --- pkg/kv/kvserver/gc/BUILD.bazel | 2 ++ pkg/kv/kvserver/gc/gc_int_test.go | 10 ++++++++++ pkg/kv/kvserver/gc/gc_random_test.go | 9 +++++++-- 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/pkg/kv/kvserver/gc/BUILD.bazel b/pkg/kv/kvserver/gc/BUILD.bazel index 53bcb80bd59c..25ac6661cecb 100644 --- a/pkg/kv/kvserver/gc/BUILD.bazel +++ b/pkg/kv/kvserver/gc/BUILD.bazel @@ -48,6 +48,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvserver", "//pkg/kv/kvserver/rditer", "//pkg/roachpb", "//pkg/security/securityassets", @@ -59,6 +60,7 @@ go_test( "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", + "//pkg/util", "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/leaktest", diff --git a/pkg/kv/kvserver/gc/gc_int_test.go b/pkg/kv/kvserver/gc/gc_int_test.go index 368b7ccb78f3..10ae6cba80e5 100644 --- a/pkg/kv/kvserver/gc/gc_int_test.go +++ b/pkg/kv/kvserver/gc/gc_int_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/securityassets" "github.com/cockroachdb/cockroach/pkg/security/securitytest" @@ -29,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -37,6 +39,11 @@ import ( "github.com/stretchr/testify/require" ) +// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke +// bugs in time-bound iterators. We disable this under race, due to the slowdown. +var smallEngineBlocks = !util.RaceEnabled && + util.ConstantWithMetamorphicTestBool("small-engine-blocks", false) + func init() { randutil.SeedForTests() securityassets.SetLoader(securitytest.EmbeddedAssets) @@ -54,6 +61,9 @@ func TestEndToEndGC(t *testing.T) { tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, + }, Server: &server.TestingKnobs{ WallClock: manualClock, }, diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index c2bd0fe485d0..0302a7f5a5ca 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -87,6 +88,10 @@ var ( oldIntentFrac: .1, rangeKeyFrac: .1, } + + // smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke + // bugs in time-bound iterators. + smallEngineBlocks = util.ConstantWithMetamorphicTestBool("small-engine-blocks", false) ) const intentAgeThreshold = 2 * time.Hour @@ -121,7 +126,7 @@ func TestRunNewVsOld(t *testing.T) { rng, seed := randutil.NewTestRand() t.Logf("Using subtest seed: %d", seed) - eng := storage.NewDefaultInMemForTesting() + eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1))) defer eng.Close() tc.ds.dist(N, rng).setupTest(t, eng, *tc.ds.desc()) @@ -273,7 +278,7 @@ func TestNewVsInvariants(t *testing.T) { t.Logf("Using subtest seed: %d", seed) desc := tc.ds.desc() - eng := storage.NewDefaultInMemForTesting() + eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1))) defer eng.Close() sortedDistribution(tc.ds.dist(N, rng)).setupTest(t, eng, *desc) From 06b840537386e5ea2a126a58a52b44265ff79cbe Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 20:06:27 +0000 Subject: [PATCH 7/7] backupccl: add metamorphic test param for small engine blocks Uses a Pebble block size of 1 byte, to provoke bugs in time-bound iterators. Release note: None --- pkg/ccl/backupccl/utils_test.go | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 50f7f57cae89..6aaf9760fc78 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" @@ -60,6 +61,11 @@ const ( localFoo = "nodelocal://0/foo" ) +// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke +// bugs in time-bound iterators. We disable this in race builds, which can +// be too slow. +var smallEngineBlocks = !util.RaceEnabled && util.ConstantWithMetamorphicTestBool("small-engine-blocks", false) + func backupRestoreTestSetupWithParams( t testing.TB, clusterSize int, @@ -90,6 +96,13 @@ func backupRestoreTestSetupWithParams( } } + if smallEngineBlocks { + if params.ServerArgs.Knobs.Store == nil { + params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{} + } + params.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs).SmallEngineBlocks = true + } + tc = testcluster.StartTestCluster(t, clusterSize, params) init(tc) @@ -168,6 +181,14 @@ func backupRestoreTestSetupEmptyWithParams( params.ServerArgsPerNode[i] = param } } + + if smallEngineBlocks { + if params.ServerArgs.Knobs.Store == nil { + params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{} + } + params.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs).SmallEngineBlocks = true + } + tc = testcluster.StartTestCluster(t, clusterSize, params) init(tc) @@ -192,6 +213,9 @@ func createEmptyCluster( // Disabling the default test tenant due to test failures. More // investigation is required. Tracked with #76378. params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, + } tc := testcluster.StartTestCluster(t, clusterSize, params) sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0])