Skip to content

Commit

Permalink
logictest: improve non-metamorphic test handling
Browse files Browse the repository at this point in the history
This commit refactors how non-metamorphic logic tests are handled.
Previously, if a logic test file had `!metamorphic` directive, then they
would be skipped when the build happened to be metamorphic (which occurs
in 80% of the time). However, this led to multiple flakes when PRs were
merged with green CI, but they didn't update the corresponding tests
(this happened because those tests were skipped).

Looking over all of those issues, we see that they are due to a couple
of randomizations - of `mutations.MaxBatchSize` and `row.kvBatchSize`,
so this commit adds a testing knob to override both of them. Now, for
every config and for every logic test file path we remember whether
non-metamorphic directive was specified and possibly disable the
randomizations of `mutations.MaxBatchSize` and `row.kvBatchSize`. This
required a bit of plumbing, but it looks acceptable.

Release note: None
  • Loading branch information
yuzefovich committed Jan 20, 2021
1 parent dd1976d commit 8f1bb31
Show file tree
Hide file tree
Showing 25 changed files with 146 additions and 64 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/ambiguous_commit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ func TestAmbiguousCommit(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

if mutations.MaxBatchSize() == 1 {
if mutations.MaxBatchSize(false /* forceProductionMaxBatchSize */) == 1 {
// This test relies on the fact that the mutation batch consisting of a
// single row also contains an EndTxn which is the case only when the
// max batch size is at least 2, so we'll skip it.
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -288,7 +288,8 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk(
// populated and deleted by the OLTP commands but not otherwise
// read or used
if err := cb.fetcher.StartScan(
ctx, txn, []roachpb.Span{sp}, true /* limitBatches */, chunkSize, traceKV,
ctx, txn, []roachpb.Span{sp}, true /* limitBatches */, chunkSize,
traceKV, false, /* forceProductionKVBatchSize */
); err != nil {
log.Errorf(ctx, "scan error: %s", err)
return roachpb.Key{}, err
Expand Down Expand Up @@ -688,7 +689,8 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
// populated and deleted by the OLTP commands but not otherwise
// read or used
if err := ib.fetcher.StartScan(
ctx, txn, []roachpb.Span{sp}, true /* limitBatches */, initBufferSize, traceKV,
ctx, txn, []roachpb.Span{sp}, true /* limitBatches */, initBufferSize,
traceKV, false, /* forceProductionKVBatchSize */
); err != nil {
log.Errorf(ctx, "scan error: %s", err)
return nil, nil, 0, err
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/colfetcher/cfetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -613,6 +613,7 @@ func (rf *cFetcher) StartScan(
limitBatches bool,
limitHint int64,
traceKV bool,
forceProductionKVBatchSize bool,
) error {
if len(spans) == 0 {
return errors.AssertionFailedf("no spans")
Expand Down Expand Up @@ -645,6 +646,7 @@ func (rf *cFetcher) StartScan(
rf.lockStrength,
rf.lockWaitPolicy,
nil, /* memMonitor */
forceProductionKVBatchSize,
)
if err != nil {
return err
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colfetcher/colbatch_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,8 +71,8 @@ func (s *ColBatchScan) Init() {
s.init = true
limitBatches := !s.parallelize
if err := s.rf.StartScan(
s.ctx, s.flowCtx.Txn, s.spans,
limitBatches, s.limitHint, s.flowCtx.TraceKV,
s.ctx, s.flowCtx.Txn, s.spans, limitBatches, s.limitHint, s.flowCtx.TraceKV,
s.flowCtx.EvalCtx.TestingKnobs.ForceProductionBatchSizes,
); err != nil {
colexecerror.InternalError(err)
}
Expand Down
38 changes: 25 additions & 13 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -1326,6 +1326,7 @@ func (t *logicTest) newCluster(serverArgs TestServerArgs) {
AssertFuncExprReturnTypes: true,
DisableOptimizerRuleProbability: *disableOptRuleProbability,
OptimizerCostPerturbation: *optimizerCostPerturbation,
ForceProductionBatchSizes: serverArgs.forceProductionBatchSizes,
},
SQLExecutor: &sql.ExecutorTestingKnobs{
DeterministicExplainAnalyze: true,
Expand Down Expand Up @@ -1666,8 +1667,11 @@ func getBlocklistIssueNo(blocklistDirective string) (string, int) {
}

// processConfigs, given a list of configNames, returns the list of
// corresponding logicTestConfigIdxs.
func processConfigs(t *testing.T, path string, defaults configSet, configNames []string) configSet {
// corresponding logicTestConfigIdxs as well as a boolean indicating whether
// the test works only in non-metamorphic setting.
func processConfigs(
t *testing.T, path string, defaults configSet, configNames []string,
) (_ configSet, onlyNonMetamorphic bool) {
const blocklistChar = '!'
// blocklist is a map from a blocked config to a corresponding issue number.
// If 0, there is no associated issue.
Expand All @@ -1687,15 +1691,11 @@ func processConfigs(t *testing.T, path string, defaults configSet, configNames [
}

if _, ok := blocklist["metamorphic"]; ok && util.IsMetamorphicBuild() {
// We have a metamorphic build and the file has !metamorphic
// blocklist directive which effectively skips the file, so we
// simply return empty configSet.
t.Logf("will skip test %s because it cannot run in a metamorphic build; pass TAGS=crdb_test_off to disable metamorphic builds", path)
return configSet{}
onlyNonMetamorphic = true
}
if len(blocklist) != 0 && allConfigNamesAreBlocklistDirectives {
// No configs specified, this blocklist applies to the default configs.
return applyBlocklistToConfigs(defaults, blocklist)
return applyBlocklistToConfigs(defaults, blocklist), onlyNonMetamorphic
}

var configs configSet
Expand All @@ -1722,7 +1722,7 @@ func processConfigs(t *testing.T, path string, defaults configSet, configNames [
}
}

return configs
return configs, onlyNonMetamorphic
}

// readTestFileConfigs reads any LogicTest directive at the beginning of a
Expand All @@ -1734,7 +1734,9 @@ func processConfigs(t *testing.T, path string, defaults configSet, configNames [
// # LogicTest: default distsql
//
// If the file doesn't contain a directive, the default config is returned.
func readTestFileConfigs(t *testing.T, path string, defaults configSet) configSet {
func readTestFileConfigs(
t *testing.T, path string, defaults configSet,
) (_ configSet, onlyNonMetamorphic bool) {
file, err := os.Open(path)
if err != nil {
t.Fatal(err)
Expand Down Expand Up @@ -1762,7 +1764,7 @@ func readTestFileConfigs(t *testing.T, path string, defaults configSet) configSe
}
}
// No directive found, return the default config.
return defaults
return defaults, false
}

type subtestDetails struct {
Expand Down Expand Up @@ -3101,6 +3103,9 @@ type TestServerArgs struct {
// actually in-memory). If it is unset, then the default limit of 100MB
// will be used.
tempStorageDiskLimit int64
// If set, mutations.MaxBatchSize and row.getKVBatchSize will be overridden
// to use the non-test value.
forceProductionBatchSizes bool
}

// RunLogicTest is the main entry point for the logic test. The globs parameter
Expand Down Expand Up @@ -3167,6 +3172,9 @@ func RunLogicTestWithDefaultConfig(
// Read the configuration directives from all the files and accumulate a list
// of paths per config.
configPaths := make([][]string, len(logicTestConfigs))
// nonMetamorphic mirrors configPaths and indicates whether a particular
// config on a particular path can only run in non-metamorphic setting.
nonMetamorphic := make([][]bool, len(logicTestConfigs))
configDefaults := defaultConfig
var configFilter map[string]struct{}
if configOverride != "" {
Expand All @@ -3186,7 +3194,7 @@ func RunLogicTestWithDefaultConfig(
}
}
for _, path := range paths {
configs := readTestFileConfigs(t, path, configDefaults)
configs, onlyNonMetamorphic := readTestFileConfigs(t, path, configDefaults)
for _, idx := range configs {
config := logicTestConfigs[idx]
configName := config.name
Expand All @@ -3200,6 +3208,7 @@ func RunLogicTestWithDefaultConfig(
continue
}
configPaths[idx] = append(configPaths[idx], path)
nonMetamorphic[idx] = append(nonMetamorphic[idx], onlyNonMetamorphic)
}
}

Expand All @@ -3216,6 +3225,7 @@ func RunLogicTestWithDefaultConfig(
seenPaths := make(map[string]struct{})
for idx, cfg := range logicTestConfigs {
paths := configPaths[idx]
nonMetamorphic := nonMetamorphic[idx]
if len(paths) == 0 {
continue
}
Expand All @@ -3230,8 +3240,9 @@ func RunLogicTestWithDefaultConfig(
if logicTestsConfigFilter != "" && cfg.name != logicTestsConfigFilter {
skip.IgnoreLint(t, "config does not match env var")
}
for _, path := range paths {
for i, path := range paths {
path := path // Rebind range variable.
onlyNonMetamorphic := nonMetamorphic[i]
// Inner test: one per file path.
t.Run(filepath.Base(path), func(t *testing.T) {
if *rewriteResultsInTestfiles {
Expand Down Expand Up @@ -3269,6 +3280,7 @@ func RunLogicTestWithDefaultConfig(
if *printErrorSummary {
defer lt.printErrorSummary()
}
serverArgs.forceProductionBatchSizes = onlyNonMetamorphic
lt.setup(cfg, serverArgs)
lt.runFile(path, cfg)

Expand Down
18 changes: 14 additions & 4 deletions pkg/sql/mutations/mutations_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,29 @@ import (
"github.com/cockroachdb/cockroach/pkg/util"
)

const productionMaxBatchSize = 10000

var maxBatchSize = defaultMaxBatchSize

var defaultMaxBatchSize = int64(util.ConstantWithMetamorphicTestRange(
10000, /* defaultValue */
1, /* min */
10000, /* max */
productionMaxBatchSize, /* defaultValue */
1, /* min */
productionMaxBatchSize, /* max */
))

// MaxBatchSize returns the max number of entries in the KV batch for a
// mutation operation (delete, insert, update, upsert) - including secondary
// index updates, FK cascading updates, etc - before the current KV batch is
// executed and a new batch is started.
func MaxBatchSize() int {
//
// If forceProductionMaxBatchSize is true, then the "production" value will be
// returned regardless of whether the build is metamorphic or not. This should
// only be used by tests the output of which differs if maxBatchSize is
// randomized.
func MaxBatchSize(forceProductionMaxBatchSize bool) int {
if forceProductionMaxBatchSize {
return productionMaxBatchSize
}
return int(atomic.LoadInt64(&maxBatchSize))
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b
// that we send, not a number of rows. We use this as a guideline only,
// and there is no guarantee that we won't produce a bigger batch.)
values, ok := ins.Input.(*memo.ValuesExpr)
if !ok || values.ChildCount() > mutations.MaxBatchSize() || values.Relational().HasSubquery {
if !ok || values.ChildCount() > mutations.MaxBatchSize(false /* forceProductionMaxBatchSize */) || values.Relational().HasSubquery {
return execPlan{}, false, nil
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/row/fetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -569,6 +569,7 @@ func (rf *Fetcher) StartScan(
limitBatches bool,
limitHint int64,
traceKV bool,
forceProductionKVBatchSize bool,
) error {
if len(spans) == 0 {
return errors.AssertionFailedf("no spans")
Expand All @@ -584,6 +585,7 @@ func (rf *Fetcher) StartScan(
rf.lockStrength,
rf.lockWaitPolicy,
rf.mon,
forceProductionKVBatchSize,
)
if err != nil {
return err
Expand All @@ -609,6 +611,7 @@ func (rf *Fetcher) StartInconsistentScan(
limitBatches bool,
limitHint int64,
traceKV bool,
forceProductionKVBatchSize bool,
) error {
if len(spans) == 0 {
return errors.AssertionFailedf("no spans")
Expand Down Expand Up @@ -665,6 +668,7 @@ func (rf *Fetcher) StartInconsistentScan(
rf.lockStrength,
rf.lockWaitPolicy,
rf.mon,
forceProductionKVBatchSize,
)
if err != nil {
return err
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/row/fetcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,6 +173,7 @@ func TestNextRowSingle(t *testing.T) {
false, /*limitBatches*/
0, /*limitHint*/
false, /*traceKV*/
false, /*forceProductionKVBatchSize*/
); err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -293,6 +294,7 @@ func TestNextRowBatchLimiting(t *testing.T) {
true, /*limitBatches*/
10, /*limitHint*/
false, /*traceKV*/
false, /*forceProductionKVBatchSize*/
); err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -403,6 +405,7 @@ func TestRowFetcherMemoryLimits(t *testing.T) {
false, /*limitBatches*/
0, /*limitHint*/
false, /*traceKV*/
false, /*forceProductionKVBatchSize*/
)
assert.Error(t, err)
assert.Equal(t, pgerror.GetPGCode(err), pgcode.OutOfMemory)
Expand Down Expand Up @@ -490,6 +493,7 @@ INDEX(c)
// batch that ends between rows.
1, /*limitHint*/
false, /*traceKV*/
false, /*forceProductionKVBatchSize*/
); err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -650,6 +654,7 @@ func TestNextRowSecondaryIndex(t *testing.T) {
false, /*limitBatches*/
0, /*limitHint*/
false, /*traceKV*/
false, /*forceProductionKVBatchSize*/
); err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -1007,6 +1012,7 @@ func TestNextRowInterleaved(t *testing.T) {
false, /*limitBatches*/
0, /*limitHint*/
false, /*traceKV*/
false, /*forceProductionKVBatchSize*/
); err != nil {
t.Fatal(err)
}
Expand Down
Loading

0 comments on commit 8f1bb31

Please sign in to comment.