diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index 34079466de0d..b2ed73291219 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -74,6 +74,7 @@ var retiredSettings = map[string]struct{}{ // removed as of 21.1. "sql.distsql.interleaved_joins.enabled": {}, "sql.testing.vectorize.batch_size": {}, + "sql.testing.mutations.max_batch_size": {}, } // register adds a setting to the registry. diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 1e097412f530..95f785b19c45 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -24,7 +24,6 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/mutations", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/rowenc", diff --git a/pkg/sql/flowinfra/flow.go b/pkg/sql/flowinfra/flow.go index eb8e3de901eb..584369a963be 100644 --- a/pkg/sql/flowinfra/flow.go +++ b/pkg/sql/flowinfra/flow.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/mutations" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -179,17 +178,6 @@ func (f *FlowBase) Setup( ctx, f.ctxCancel = contextutil.WithCancel(ctx) f.ctxDone = ctx.Done() f.spec = spec - - mutationsTestingMaxBatchSize := int64(0) - if f.FlowCtx.Cfg.Settings != nil { - mutationsTestingMaxBatchSize = mutations.MutationsTestingMaxBatchSize.Get(&f.FlowCtx.Cfg.Settings.SV) - } - if mutationsTestingMaxBatchSize != 0 { - mutations.SetMaxBatchSizeForTests(int(mutationsTestingMaxBatchSize)) - } else { - mutations.ResetMaxBatchSizeForTests() - } - return ctx, nil } diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 2c000871bd06..42b664a7bbfd 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1110,12 +1110,6 @@ type logicTest struct { curPath string curLineNo int - - // randomizedMutationsMaxBatchSize stores the randomized max batch size for - // the mutation operations. The max batch size will randomly be set to 1 - // with 25% probability, a random value in [2, 100] range with 25% - // probability, or default max batch size with 50% probability. - randomizedMutationsMaxBatchSize int } func (t *logicTest) t() *testing.T { @@ -1459,12 +1453,6 @@ func (t *logicTest) setup(cfg testClusterConfig, serverArgs TestServerArgs) { t.Fatal(err) } - if _, err := conn.Exec( - "SET CLUSTER SETTING sql.testing.mutations.max_batch_size = $1", t.randomizedMutationsMaxBatchSize, - ); err != nil { - t.Fatal(err) - } - if cfg.overrideAutoStats != "" { if _, err := conn.Exec( "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = $1::bool", cfg.overrideAutoStats, @@ -2950,11 +2938,6 @@ type TestServerArgs struct { // actually in-memory). If it is unset, then the default limit of 100MB // will be used. tempStorageDiskLimit int64 - // DisableMutationsMaxBatchSizeRandomization determines whether the test - // runner should randomize the max batch size for mutation operations. This - // should only be set to 'true' when the tests expect to return different - // output when the KV batches of writes have different boundaries. - DisableMutationsMaxBatchSizeRandomization bool } // RunLogicTest is the main entry point for the logic test. The globs parameter @@ -3057,18 +3040,6 @@ func RunLogicTestWithDefaultConfig( } } - rng, _ := randutil.NewPseudoRand() - randomizedMutationsMaxBatchSize := mutations.MaxBatchSize() - // Temporarily disable this randomization because of #54948. - // TODO(yuzefovich): re-enable it once the issue is figured out. - serverArgs.DisableMutationsMaxBatchSizeRandomization = true - if !serverArgs.DisableMutationsMaxBatchSizeRandomization { - randomizedMutationsMaxBatchSize = randomValue(rng, []int{1, 2 + rng.Intn(99)}, []float64{0.25, 0.25}, mutations.MaxBatchSize()) - if randomizedMutationsMaxBatchSize != mutations.MaxBatchSize() { - t.Log(fmt.Sprintf("randomize mutations.MaxBatchSize to %d", randomizedMutationsMaxBatchSize)) - } - } - // The tests below are likely to run concurrently; `log` is shared // between all the goroutines and thus all tests, so it doesn't make // sense to try to use separate `log.Scope` instances for each test. @@ -3122,11 +3093,10 @@ func RunLogicTestWithDefaultConfig( } rng, _ := randutil.NewPseudoRand() lt := logicTest{ - rootT: t, - verbose: verbose, - perErrorSummary: make(map[string][]string), - rng: rng, - randomizedMutationsMaxBatchSize: randomizedMutationsMaxBatchSize, + rootT: t, + verbose: verbose, + perErrorSummary: make(map[string][]string), + rng: rng, } if *printErrorSummary { defer lt.printErrorSummary() @@ -3425,29 +3395,3 @@ func (t *logicTest) printCompletion(path string, config testClusterConfig) { t.outf("--- done: %s with config %s: %d tests, %d failures%s", path, config.name, t.progress, t.failures, unsupportedMsg) } - -// randomValue randomly chooses one element from values according to -// probabilities (the sum of which must not exceed 1.0). If the sum of -// probabilities is less than 1.0, then defaultValue will be chosen in 1.0-sum -// proportion of cases. -func randomValue(rng *rand.Rand, values []int, probabilities []float64, defaultValue int) int { - if len(values) != len(probabilities) { - panic(errors.AssertionFailedf("mismatched number of values %d and probabilities %d", len(values), len(probabilities))) - } - probabilitiesSum := 0.0 - for _, p := range probabilities { - probabilitiesSum += p - } - if probabilitiesSum > 1.0 { - panic(errors.AssertionFailedf("sum of probabilities %v is larger than 1.0", probabilities)) - } - randVal := rng.Float64() - probabilitiesSum = 0 - for i, p := range probabilities { - if randVal < probabilitiesSum+p { - return values[i] - } - probabilitiesSum += p - } - return defaultValue -} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_automatic_stats b/pkg/sql/logictest/testdata/logic_test/distsql_automatic_stats similarity index 93% rename from pkg/sql/opt/exec/execbuilder/testdata/distsql_automatic_stats rename to pkg/sql/logictest/testdata/logic_test/distsql_automatic_stats index 1600fbc5dea5..42d17da1a17a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_automatic_stats +++ b/pkg/sql/logictest/testdata/logic_test/distsql_automatic_stats @@ -1,17 +1,5 @@ # LogicTest: !metamorphic -# TODO(yuzefovich): move the file back. - -# Note that this file was moved from the regular logic tests as a part of -# introducing the randomization of the max batch size of mutations. That -# randomization needs to be disabled for these queries because they become -# non-deterministic. However, there is no easy to do so just for a single file, -# so the file was moved into execbuilder tests which disable that -# randomization. -# The file didn't explicitly specify the configs to run with, and it wasn't -# changed, but if later it becomes a concern, it should be ok to restrict the -# number of configs here. - # Disable automatic stats statement ok SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 77caee8707ff..41a1c61b0e9a 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -581,8 +581,7 @@ FROM system.settings WHERE name NOT IN ('version', 'sql.defaults.distsql', 'cluster.secret', 'sql.stats.automatic_collection.enabled', 'sql.defaults.vectorize', 'sql.defaults.vectorize_row_count_threshold', - 'sql.defaults.experimental_distsql_planning', - 'sql.testing.mutations.max_batch_size') + 'sql.defaults.experimental_distsql_planning') ORDER BY name ---- diagnostics.reporting.enabled true diff --git a/pkg/sql/mutations/BUILD.bazel b/pkg/sql/mutations/BUILD.bazel index ac60b38d62cd..1674c452bdcd 100644 --- a/pkg/sql/mutations/BUILD.bazel +++ b/pkg/sql/mutations/BUILD.bazel @@ -9,7 +9,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/mutations", visibility = ["//visibility:public"], deps = [ - "//pkg/settings", "//pkg/sql/catalog/colinfo", "//pkg/sql/parser", "//pkg/sql/rowenc", diff --git a/pkg/sql/mutations/mutations_util.go b/pkg/sql/mutations/mutations_util.go index 5522fcc48b78..847d584abbc2 100644 --- a/pkg/sql/mutations/mutations_util.go +++ b/pkg/sql/mutations/mutations_util.go @@ -13,7 +13,6 @@ package mutations import ( "sync/atomic" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util" ) @@ -44,12 +43,3 @@ func SetMaxBatchSizeForTests(newMaxBatchSize int) { func ResetMaxBatchSizeForTests() { atomic.SwapInt64(&maxBatchSize, defaultMaxBatchSize) } - -// MutationsTestingMaxBatchSize is a testing cluster setting that sets the -// default max mutation batch size. A low max batch size is useful to test -// batching logic of the mutations. -var MutationsTestingMaxBatchSize = settings.RegisterNonNegativeIntSetting( - "sql.testing.mutations.max_batch_size", - "the max number of rows that are processed by a single KV batch when performing a mutation operation (0=default)", - 0, -) diff --git a/pkg/sql/opt/exec/execbuilder/builder_test.go b/pkg/sql/opt/exec/execbuilder/builder_test.go index 3c62d0bbf1a8..e044dcb9699b 100644 --- a/pkg/sql/opt/exec/execbuilder/builder_test.go +++ b/pkg/sql/opt/exec/execbuilder/builder_test.go @@ -27,11 +27,5 @@ import ( func TestExecBuild(t *testing.T) { defer leaktest.AfterTest(t)() defer sql.TestingOverrideExplainEnvVersion("CockroachDB execbuilder test version")() - logictest.RunLogicTest(t, logictest.TestServerArgs{ - // Several test files in execbuilder verify that mutations behave as - // expected; however, if we add the randomization of the mutations max - // batch size, then the output becomes non-deterministic, so we disable - // that randomization. - DisableMutationsMaxBatchSizeRandomization: true, - }, "testdata/[^.]*") + logictest.RunLogicTest(t, logictest.TestServerArgs{}, "testdata/[^.]*") }