Skip to content

Commit

Permalink
sql,logictest: use deterministic descriptor ID generation in logctests
Browse files Browse the repository at this point in the history
This commit adds a testing knob which results in descriptor ID generation
being handled transactionally. The logictests now use this knob.

Fixes #37751
Fixes #69226

Release note: None
  • Loading branch information
ajwerner committed Aug 1, 2022
1 parent ad31fc7 commit 85ce24d
Show file tree
Hide file tree
Showing 5 changed files with 19 additions and 2 deletions.
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -312,6 +312,7 @@ go_library(
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descbuilder",
"//pkg/sql/catalog/descidgen",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/funcdesc",
Expand Down
11 changes: 10 additions & 1 deletion pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
Expand Down Expand Up @@ -2715,7 +2716,7 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo
StmtDiagnosticsRequestInserter: ex.server.cfg.StmtDiagnosticsRecorder.InsertRequest,
CatalogBuiltins: &p.evalCatalogBuiltins,
QueryCancelKey: ex.queryCancelKey,
DescIDGenerator: ex.server.cfg.DescIDGenerator,
DescIDGenerator: ex.getDescIDGenerator(),
},
Tracing: &ex.sessionTracing,
MemMetrics: &ex.memMetrics,
Expand Down Expand Up @@ -3273,6 +3274,14 @@ func (ex *connExecutor) runPreCommitStages(ctx context.Context) error {
return nil
}

func (ex *connExecutor) getDescIDGenerator() eval.DescIDGenerator {
if ex.server.cfg.TestingKnobs.UseTransactionalDescIDGenerator &&
ex.state.mu.txn != nil {
return descidgen.NewTransactionalGenerator(ex.server.cfg.Codec, ex.state.mu.txn)
}
return ex.server.cfg.DescIDGenerator
}

// StatementCounters groups metrics for counting different types of
// statements.
type StatementCounters struct {
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1470,6 +1470,11 @@ type ExecutorTestingKnobs struct {
// OnRecordTxnFinish, if set, will be called as we record a transaction
// finishing.
OnRecordTxnFinish func(isInternal bool, phaseTimes *sessionphase.Times, stmt string)

// UseTransactionDescIDGenerator is used to force descriptor ID generation
// to use a transaction, and, in doing so, more deterministically allocate
// descriptor IDs at the cost of decreased parallelism.
UseTransactionalDescIDGenerator bool
}

// PGWireTestingKnobs contains knobs for the pgwire module.
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -1171,7 +1171,8 @@ func (t *logicTest) newCluster(
ForceProductionValues: serverArgs.ForceProductionValues,
},
SQLExecutor: &sql.ExecutorTestingKnobs{
DeterministicExplain: true,
DeterministicExplain: true,
UseTransactionalDescIDGenerator: true,
},
SQLStatsKnobs: &sqlstats.TestingKnobs{
AOSTClause: "AS OF SYSTEM TIME '-1us'",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/mvcc_backfiller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ func TestIndexBackfillMergeRetry(t *testing.T) {
defer log.Scope(t).Close(t)

skip.UnderStressRace(t, "TODO(ssd) test times outs under race")
skip.UnderRace(t, "TODO(ssd) test times outs under race")

params, _ := tests.CreateTestServerParams()

Expand Down

0 comments on commit 85ce24d

Please sign in to comment.