From ed0c3e6f234f3594eb51a76a09916fcfa7e4da4e Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 9 Feb 2024 18:23:30 -0500 Subject: [PATCH 1/6] opt/bench: add benchmark for execbuilder This commit adds a benchmark that measures only the execbuilder phase of optimization, and includes no other phases. Release note: None --- pkg/sql/opt/bench/bench_test.go | 67 +++++++++++++++++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/pkg/sql/opt/bench/bench_test.go b/pkg/sql/opt/bench/bench_test.go index 4dcbaa5b76aa..1d8f1a81dd02 100644 --- a/pkg/sql/opt/bench/bench_test.go +++ b/pkg/sql/opt/bench/bench_test.go @@ -1584,3 +1584,70 @@ func BenchmarkSlowQueries(b *testing.B) { }) } } + +// BenchmarkExecBuild measures the time that the execbuilder phase takes. It +// does not include any other phases. +func BenchmarkExecBuild(b *testing.B) { + type testCase struct { + query benchQuery + schema []string + } + var testCases []testCase + + // Add the basic queries. + for _, query := range queriesToTest(b) { + testCases = append(testCases, testCase{query, schemas}) + } + + // Add the slow queries. + p := datapathutils.TestDataPath(b, "slow-schemas.sql") + slowSchemas, err := os.ReadFile(p) + if err != nil { + b.Fatalf("%v", err) + } + for _, query := range slowQueries { + testCases = append(testCases, testCase{query, []string{string(slowSchemas)}}) + } + + for _, tc := range testCases { + h := newHarness(b, tc.query, tc.schema) + + stmt, err := parser.ParseOne(tc.query.query) + if err != nil { + b.Fatalf("%v", err) + } + + h.optimizer.Init(context.Background(), &h.evalCtx, h.testCat) + bld := optbuilder.New(h.ctx, &h.semaCtx, &h.evalCtx, h.testCat, h.optimizer.Factory(), stmt.AST) + if err = bld.Build(); err != nil { + b.Fatalf("%v", err) + } + + if _, err := h.optimizer.Optimize(); err != nil { + panic(err) + } + + execMemo := h.optimizer.Memo() + root := execMemo.RootExpr() + + b.Run(tc.query.name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + eb := execbuilder.New( + context.Background(), + explain.NewPlanGistFactory(exec.StubFactory{}), + &h.optimizer, + execMemo, + nil, /* catalog */ + root, + &h.semaCtx, + &h.evalCtx, + true, /* allowAutoCommit */ + false, /* isANSIDML */ + ) + if _, err := eb.Build(); err != nil { + b.Fatalf("%v", err) + } + } + }) + } +} From 485449c98e25ab424a14ef35bc073da64a8d66bd Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 8 Feb 2024 15:37:29 -0500 Subject: [PATCH 2/6] opt/execbuilder: remove column map from execPlan As the execbuilder traverses a relational expression and recursively builds an `execPlan`, it creates mappings from column IDs to their ordinal position in the expression for each `execPlan` node. These mappings are used when building parent nodes to correctly map column IDs to indexed variables. In most cases the mappings are only used when building a parent, and never again. Prior to this commit, the column mappings were a field of `execPlan`, tying the lifetime of `execPlan` nodes and column mappings together. This commit decouples the lifetimes of both by removing the mapping field from `execPlan` and propagating mappings up as return values of recursive function calls. This will enable future optimizations that can reuse memory allocated for mappings that are no longer needed. Release note: None --- pkg/sql/opt/exec/execbuilder/builder.go | 6 +- pkg/sql/opt/exec/execbuilder/mutation.go | 148 +-- pkg/sql/opt/exec/execbuilder/relational.go | 1125 ++++++++++---------- pkg/sql/opt/exec/execbuilder/scalar.go | 14 +- pkg/sql/opt/exec/execbuilder/statement.go | 247 +++-- 5 files changed, 824 insertions(+), 716 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 8e9a5e04141c..284767f0e17d 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -230,7 +230,7 @@ func New( // Build constructs the execution node tree and returns its root node if no // error occurred. func (b *Builder) Build() (_ exec.Plan, err error) { - plan, err := b.build(b.e) + plan, _, err := b.build(b.e) if err != nil { return nil, err } @@ -257,7 +257,7 @@ func (b *Builder) wrapFunction(fnName string) (tree.ResolvableFunctionReference, return tree.WrapFunction(fnName), nil } -func (b *Builder) build(e opt.Expr) (_ execPlan, err error) { +func (b *Builder) build(e opt.Expr) (_ execPlan, outputCols opt.ColMap, err error) { defer func() { if r := recover(); r != nil { // This code allows us to propagate errors without adding lots of checks @@ -274,7 +274,7 @@ func (b *Builder) build(e opt.Expr) (_ execPlan, err error) { rel, ok := e.(memo.RelExpr) if !ok { - return execPlan{}, errors.AssertionFailedf( + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf( "building execution for non-relational operator %s", redact.Safe(e.Op()), ) } diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 54b5cd8b1be1..a4620038ec0f 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -30,10 +30,10 @@ import ( func (b *Builder) buildMutationInput( mutExpr, inputExpr memo.RelExpr, colList opt.ColList, p *memo.MutationPrivate, -) (execPlan, error) { +) (_ execPlan, outputCols opt.ColMap, err error) { shouldApplyImplicitLocking, err := b.shouldApplyImplicitLockingToMutationInput(mutExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if shouldApplyImplicitLocking { // Re-entrance is not possible because mutations are never nested. @@ -41,9 +41,9 @@ func (b *Builder) buildMutationInput( defer func() { b.forceForUpdateLocking = false }() } - input, err := b.buildRelational(inputExpr) + input, inputCols, err := b.buildRelational(inputExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // TODO(mgartner/radu): This can incorrectly append columns in a FK cascade @@ -60,27 +60,27 @@ func (b *Builder) buildMutationInput( } } - input, err = b.ensureColumns(input, inputExpr, colList, inputExpr.ProvidedPhysical().Ordering) + input, inputCols, err = b.ensureColumns(input, inputCols, inputExpr, colList, inputExpr.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if p.WithID != 0 { label := fmt.Sprintf("buffer %d", p.WithID) bufferNode, err := b.factory.ConstructBuffer(input.root, label) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - b.addBuiltWithExpr(p.WithID, input.outputCols, bufferNode) + b.addBuiltWithExpr(p.WithID, inputCols, bufferNode) input.root = bufferNode } - return input, nil + return input, inputCols, nil } -func (b *Builder) buildInsert(ins *memo.InsertExpr) (execPlan, error) { - if ep, ok, err := b.tryBuildFastPathInsert(ins); err != nil || ok { - return ep, err +func (b *Builder) buildInsert(ins *memo.InsertExpr) (_ execPlan, outputCols opt.ColMap, err error) { + if ep, cols, ok, err := b.tryBuildFastPathInsert(ins); err != nil || ok { + return ep, cols, err } // Construct list of columns that only contains columns that need to be // inserted (e.g. delete-only mutation columns don't need to be inserted). @@ -88,9 +88,9 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (execPlan, error) { colList = appendColsWhenPresent(colList, ins.InsertCols) colList = appendColsWhenPresent(colList, ins.CheckCols) colList = appendColsWhenPresent(colList, ins.PartialIndexPutCols) - input, err := b.buildMutationInput(ins, ins.Input, colList, &ins.MutationPrivate) + input, _, err := b.buildMutationInput(ins, ins.Input, colList, &ins.MutationPrivate) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the Insert node. @@ -110,28 +110,30 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (execPlan, error) { len(ins.FKChecks) == 0 && len(ins.FKCascades) == 0, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the output column map. ep := execPlan{root: node} if ins.NeedResults() { - ep.outputCols = mutationOutputColMap(ins) + outputCols = mutationOutputColMap(ins) } if err := b.buildUniqueChecks(ins.UniqueChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKChecks(ins.FKChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } // tryBuildFastPathInsert attempts to construct an insert using the fast path, // checking all required conditions. See exec.Factory.ConstructInsertFastPath. -func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok bool, _ error) { +func (b *Builder) tryBuildFastPathInsert( + ins *memo.InsertExpr, +) (_ execPlan, outputCols opt.ColMap, ok bool, _ error) { // Conditions from ConstructFastPathInsert: // // - there are no other mutations in the statement, and the output of the @@ -140,12 +142,12 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b // // This condition was taken into account in build(). if !b.allowInsertFastPath { - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } // If there are unique checks required, there must be the same number of fast // path unique checks. if len(ins.UniqueChecks) != len(ins.FastPathUniqueChecks) { - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } insInput := ins.Input @@ -153,7 +155,7 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b // Values expressions containing subqueries or UDFs, or having a size larger // than the max mutation batch size are disallowed. if !ok || !memo.ValuesLegalForInsertFastPath(values) { - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } md := b.mem.Metadata() @@ -167,7 +169,7 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b // uniqueness checks during fast-path insert. Even if DatumsFromConstraint // contains no Datums, that case indicates that all values to check come // from the input row. - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } execFastPathCheck := &uniqChecks[i] // Set up the execbuilder structure from the elements built during @@ -177,7 +179,7 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b execFastPathCheck.CheckOrdinal = c.CheckOrdinal locking, err := b.buildLocking(c.Locking) if err != nil { - return execPlan{}, false, err + return execPlan{}, opt.ColMap{}, false, err } execFastPathCheck.Locking = locking execFastPathCheck.InsertCols = make([]exec.TableColumnOrdinal, len(c.InsertCols)) @@ -210,19 +212,19 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b c := &ins.FKChecks[i] if md.Table(c.ReferencedTable).ID() == md.Table(ins.Table).ID() { // Self-referencing FK. - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } fk := tab.OutboundForeignKey(c.FKOrdinal) lookupJoin, isLookupJoin := c.Check.(*memo.LookupJoinExpr) if !isLookupJoin || lookupJoin.JoinType != opt.AntiJoinOp { // Not a lookup anti-join. - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } // TODO(rytaft): see if we can remove the requirement that LookupExpr is // empty. if len(lookupJoin.On) > 0 || len(lookupJoin.LookupExpr) > 0 || len(lookupJoin.KeyCols) != fk.ColumnCount() { - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } inputExpr := lookupJoin.Input // Ignore any select (used to deal with NULLs). @@ -231,15 +233,15 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b } withScan, isWithScan := inputExpr.(*memo.WithScanExpr) if !isWithScan { - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } if withScan.With != ins.WithID { - return execPlan{}, false, nil + return execPlan{}, opt.ColMap{}, false, nil } locking, err := b.buildLocking(lookupJoin.Locking) if err != nil { - return execPlan{}, false, err + return execPlan{}, opt.ColMap{}, false, err } out := &fkChecks[i] @@ -250,12 +252,12 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b var withColOrd, inputColOrd int withColOrd, ok = withScan.OutCols.Find(keyCol) if !ok { - return execPlan{}, false, errors.AssertionFailedf("cannot find column %d", keyCol) + return execPlan{}, opt.ColMap{}, false, errors.AssertionFailedf("cannot find column %d", keyCol) } inputCol := withScan.InCols[withColOrd] inputColOrd, ok = ins.InsertCols.Find(inputCol) if !ok { - return execPlan{}, false, errors.AssertionFailedf("cannot find column %d", inputCol) + return execPlan{}, opt.ColMap{}, false, errors.AssertionFailedf("cannot find column %d", inputCol) } out.InsertCols[j] = exec.TableColumnOrdinal(inputColOrd) } @@ -294,12 +296,12 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b colList = appendColsWhenPresent(colList, ins.PartialIndexPutCols) rows, err := b.buildValuesRows(values) if err != nil { - return execPlan{}, false, err + return execPlan{}, opt.ColMap{}, false, err } // We may need to rearrange the columns. rows, err = rearrangeColumns(values.Cols, rows, colList) if err != nil { - return execPlan{}, false, err + return execPlan{}, opt.ColMap{}, false, err } // Construct the InsertFastPath node. @@ -317,14 +319,14 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b b.allowAutoCommit, ) if err != nil { - return execPlan{}, false, err + return execPlan{}, opt.ColMap{}, false, err } // Construct the output column map. ep := execPlan{root: node} if ins.NeedResults() { - ep.outputCols = mutationOutputColMap(ins) + outputCols = mutationOutputColMap(ins) } - return ep, true, nil + return ep, outputCols, true, nil } // rearrangeColumns rearranges the columns in a matrix of TypedExpr values. @@ -357,7 +359,7 @@ func rearrangeColumns( return outRows, nil } -func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (execPlan, error) { +func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (_ execPlan, outputCols opt.ColMap, err error) { // Currently, the execution engine requires one input column for each fetch // and update expression, so use ensureColumns to map and reorder columns so // that they correspond to target table columns. For example: @@ -385,9 +387,9 @@ func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (execPlan, error) { colList = appendColsWhenPresent(colList, upd.PartialIndexPutCols) colList = appendColsWhenPresent(colList, upd.PartialIndexDelCols) - input, err := b.buildMutationInput(upd, upd.Input, colList, &upd.MutationPrivate) + input, _, err := b.buildMutationInput(upd, upd.Input, colList, &upd.MutationPrivate) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the Update node. @@ -419,30 +421,30 @@ func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (execPlan, error) { len(upd.FKChecks) == 0 && len(upd.FKCascades) == 0, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildUniqueChecks(upd.UniqueChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKChecks(upd.FKChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKCascades(upd.WithID, upd.FKCascades); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the output column map. ep := execPlan{root: node} if upd.NeedResults() { - ep.outputCols = mutationOutputColMap(upd) + outputCols = mutationOutputColMap(upd) } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (execPlan, error) { +func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols opt.ColMap, err error) { // Currently, the execution engine requires one input column for each insert, // fetch, and update expression, so use ensureColumns to map and reorder // columns so that they correspond to target table columns. For example: @@ -474,9 +476,9 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (execPlan, error) { colList = appendColsWhenPresent(colList, ups.PartialIndexPutCols) colList = appendColsWhenPresent(colList, ups.PartialIndexDelCols) - input, err := b.buildMutationInput(ups, ups.Input, colList, &ups.MutationPrivate) + input, inputCols, err := b.buildMutationInput(ups, ups.Input, colList, &ups.MutationPrivate) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the Upsert node. @@ -484,9 +486,9 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (execPlan, error) { tab := md.Table(ups.Table) canaryCol := exec.NodeColumnOrdinal(-1) if ups.CanaryCol != 0 { - canaryCol, err = input.getNodeColumnOrdinal(ups.CanaryCol) + canaryCol, err = getNodeColumnOrdinal(inputCols, ups.CanaryCol) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } insertColOrds := ordinalSetFromColList(ups.InsertCols) @@ -509,19 +511,19 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (execPlan, error) { len(ups.FKChecks) == 0 && len(ups.FKCascades) == 0, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildUniqueChecks(ups.UniqueChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKChecks(ups.FKChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKCascades(ups.WithID, ups.FKCascades); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // If UPSERT returns rows, they contain all non-mutation columns from the @@ -530,15 +532,15 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (execPlan, error) { // result of the UPSERT operation for that row. ep := execPlan{root: node} if ups.NeedResults() { - ep.outputCols = mutationOutputColMap(ups) + outputCols = mutationOutputColMap(ups) } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildDelete(del *memo.DeleteExpr) (execPlan, error) { +func (b *Builder) buildDelete(del *memo.DeleteExpr) (_ execPlan, outputCols opt.ColMap, err error) { // Check for the fast-path delete case that can use a range delete. if ep, ok, err := b.tryBuildDeleteRange(del); err != nil || ok { - return ep, err + return ep, opt.ColMap{}, err } // Ensure that order of input columns matches order of target table columns. @@ -555,9 +557,9 @@ func (b *Builder) buildDelete(del *memo.DeleteExpr) (execPlan, error) { } colList = appendColsWhenPresent(colList, del.PartialIndexDelCols) - input, err := b.buildMutationInput(del, del.Input, colList, &del.MutationPrivate) + input, _, err := b.buildMutationInput(del, del.Input, colList, &del.MutationPrivate) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the Delete node. md := b.mem.Metadata() @@ -584,24 +586,24 @@ func (b *Builder) buildDelete(del *memo.DeleteExpr) (execPlan, error) { b.allowAutoCommit && len(del.FKChecks) == 0 && len(del.FKCascades) == 0, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKChecks(del.FKChecks); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if err := b.buildFKCascades(del.WithID, del.FKCascades); err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Construct the output column map. ep := execPlan{root: node} if del.NeedResults() { - ep.outputCols = mutationOutputColMap(del) + outputCols = mutationOutputColMap(del) } - return ep, nil + return ep, outputCols, nil } // tryBuildDeleteRange attempts to construct a fast DeleteRange execution for a @@ -772,7 +774,7 @@ func (b *Builder) buildUniqueChecks(checks memo.UniqueChecksExpr) error { for i := range checks { c := &checks[i] // Construct the query that returns uniqueness violations. - query, err := b.buildRelational(c.Check) + query, queryCols, err := b.buildRelational(c.Check) if err != nil { return err } @@ -780,7 +782,7 @@ func (b *Builder) buildUniqueChecks(checks memo.UniqueChecksExpr) error { mkErr := func(row tree.Datums) error { keyVals := make(tree.Datums, len(c.KeyCols)) for i, col := range c.KeyCols { - ord, err := query.getNodeColumnOrdinal(col) + ord, err := getNodeColumnOrdinal(queryCols, col) if err != nil { return err } @@ -804,7 +806,7 @@ func (b *Builder) buildFKChecks(checks memo.FKChecksExpr) error { for i := range checks { c := &checks[i] // Construct the query that returns FK violations. - query, err := b.buildRelational(c.Check) + query, queryCols, err := b.buildRelational(c.Check) if err != nil { return err } @@ -812,7 +814,7 @@ func (b *Builder) buildFKChecks(checks memo.FKChecksExpr) error { mkErr := func(row tree.Datums) error { keyVals := make(tree.Datums, len(c.KeyCols)) for i, col := range c.KeyCols { - ord, err := query.getNodeColumnOrdinal(col) + ord, err := getNodeColumnOrdinal(queryCols, col) if err != nil { return err } @@ -1140,11 +1142,11 @@ func unwrapProjectExprs(input memo.RelExpr) memo.RelExpr { return input } -func (b *Builder) buildLock(lock *memo.LockExpr) (execPlan, error) { +func (b *Builder) buildLock(lock *memo.LockExpr) (_ execPlan, outputCols opt.ColMap, err error) { // Don't bother creating the lookup join if we don't need it. locking, err := b.buildLocking(lock.Locking) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if !locking.IsLocking() { return b.buildRelational(lock.Input) diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 7e40ce61bba9..6e69d6d76bc1 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -57,39 +57,6 @@ import ( type execPlan struct { root exec.Node - - // outputCols is a map from opt.ColumnID to exec.NodeColumnOrdinal. It maps - // columns in the output set of a relational expression to indices in the - // result columns of the exec.Node. - // - // The reason we need to keep track of this (instead of using just the - // relational properties) is that the relational properties don't force a - // single "schema": any ordering of the output columns is possible. We choose - // the schema that is most convenient: for scans, we use the table's column - // ordering. Consider: - // SELECT a, b FROM t WHERE a = b - // and the following two cases: - // 1. The table is defined as (k INT PRIMARY KEY, a INT, b INT). The scan will - // return (k, a, b). - // 2. The table is defined as (k INT PRIMARY KEY, b INT, a INT). The scan will - // return (k, b, a). - // In these two cases, the relational properties are effectively the same. - // - // An alternative to this would be to always use a "canonical" schema, for - // example the output columns in increasing index order. However, this would - // require a lot of otherwise unnecessary projections. - // - // Note: conceptually, this could be a ColList; however, the map is more - // convenient when converting VariableOps to IndexedVars. - outputCols opt.ColMap -} - -// numOutputCols returns the number of columns emitted by the execPlan's Node. -// This will typically be equal to ep.outputCols.Len(), but might be different -// if the node outputs the same optimizer ColumnID multiple times. -// TODO(justin): we should keep track of this instead of computing it each time. -func (ep *execPlan) numOutputCols() int { - return numOutputColsInMap(ep.outputCols) } // numOutputColsInMap returns the number of slots required to fill in all of @@ -104,28 +71,30 @@ func numOutputColsInMap(m opt.ColMap) int { // makeBuildScalarCtx returns a buildScalarCtx that can be used with expressions // that refer the output columns of this plan. -func (ep *execPlan) makeBuildScalarCtx() buildScalarCtx { +func makeBuildScalarCtx(cols opt.ColMap) buildScalarCtx { return buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, ep.numOutputCols()), - ivarMap: ep.outputCols, + ivh: tree.MakeIndexedVarHelper(nil /* container */, numOutputColsInMap(cols)), + ivarMap: cols, } } // getNodeColumnOrdinal takes a column that is known to be produced by the execPlan // and returns the ordinal index of that column in the result columns of the // node. -func (ep *execPlan) getNodeColumnOrdinal(col opt.ColumnID) (exec.NodeColumnOrdinal, error) { - ord, ok := ep.outputCols.Get(int(col)) +func getNodeColumnOrdinal(colMap opt.ColMap, col opt.ColumnID) (exec.NodeColumnOrdinal, error) { + ord, ok := colMap.Get(int(col)) if !ok { return 0, errors.AssertionFailedf("column %d not in input", redact.Safe(col)) } return exec.NodeColumnOrdinal(ord), nil } -func (ep *execPlan) getNodeColumnOrdinalSet(cols opt.ColSet) (exec.NodeColumnOrdinalSet, error) { +func getNodeColumnOrdinalSet( + colMap opt.ColMap, cols opt.ColSet, +) (exec.NodeColumnOrdinalSet, error) { var res exec.NodeColumnOrdinalSet for colID, ok := cols.Next(0); ok; colID, ok = cols.Next(colID + 1) { - colOrd, err := ep.getNodeColumnOrdinal(colID) + colOrd, err := getNodeColumnOrdinal(colMap, colID) if err != nil { return exec.NodeColumnOrdinalSet{}, err } @@ -136,20 +105,20 @@ func (ep *execPlan) getNodeColumnOrdinalSet(cols opt.ColSet) (exec.NodeColumnOrd // reqOrdering converts the provided ordering of a relational expression to an // OutputOrdering (according to the outputCols map). -func (ep *execPlan) reqOrdering(expr memo.RelExpr) (exec.OutputOrdering, error) { - ordering, err := ep.sqlOrdering(expr.ProvidedPhysical().Ordering) +func reqOrdering(expr memo.RelExpr, cols opt.ColMap) (exec.OutputOrdering, error) { + ordering, err := sqlOrdering(expr.ProvidedPhysical().Ordering, cols) return exec.OutputOrdering(ordering), err } // sqlOrdering converts an Ordering to a ColumnOrdering (according to the // outputCols map). -func (ep *execPlan) sqlOrdering(ordering opt.Ordering) (colinfo.ColumnOrdering, error) { +func sqlOrdering(ordering opt.Ordering, cols opt.ColMap) (colinfo.ColumnOrdering, error) { if ordering.Empty() { return nil, nil } colOrder := make(colinfo.ColumnOrdering, len(ordering)) for i := range ordering { - ord, err := ep.getNodeColumnOrdinal(ordering[i].ID()) + ord, err := getNodeColumnOrdinal(cols, ordering[i].ID()) if err != nil { return nil, err } @@ -164,9 +133,37 @@ func (ep *execPlan) sqlOrdering(ordering opt.Ordering) (colinfo.ColumnOrdering, return colOrder, nil } -func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { +// builRelational builds a relational expression into an execPlan. +// +// outputCols is a map from opt.ColumnID to exec.NodeColumnOrdinal. It maps +// columns in the output set of a relational expression to indices in the +// result columns of the exec.Node. +// +// The reason we need to keep track of this (instead of using just the +// relational properties) is that the relational properties don't force a +// single "schema": any ordering of the output columns is possible. We choose +// the schema that is most convenient: for scans, we use the table's column +// ordering. Consider: +// +// SELECT a, b FROM t WHERE a = b +// +// and the following two cases: +// 1. The table is defined as (k INT PRIMARY KEY, a INT, b INT). The scan will +// return (k, a, b). +// 2. The table is defined as (k INT PRIMARY KEY, b INT, a INT). The scan will +// return (k, b, a). +// +// In these two cases, the relational properties are effectively the same. +// +// An alternative to this would be to always use a "canonical" schema, for +// example the output columns in increasing index order. However, this would +// require a lot of otherwise unnecessary projections. +// +// Note: conceptually, this could be a ColList; however, the map is more +// convenient when converting VariableOps to IndexedVars. +// outputCols opt.ColMap +func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { var ep execPlan - var err error if opt.IsDDLOp(e) { // Mark the statement as containing DDL for use @@ -178,7 +175,7 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { b.flags.Set(exec.PlanFlagContainsMutation) // Raise error if mutation op is part of a read-only transaction. if b.evalCtx.TxnReadOnly { - return execPlan{}, pgerror.Newf(pgcode.ReadOnlySQLTransaction, + return execPlan{}, opt.ColMap{}, pgerror.Newf(pgcode.ReadOnlySQLTransaction, "cannot execute %s in a read-only transaction", b.statementTag(e)) } } @@ -186,7 +183,7 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { // Raise error if bounded staleness is used incorrectly. if b.boundedStaleness() { if _, ok := boundedStalenessAllowList[e.Op()]; !ok { - return execPlan{}, unimplemented.NewWithIssuef(67562, + return execPlan{}, opt.ColMap{}, unimplemented.NewWithIssuef(67562, "cannot use bounded staleness for %s", b.statementTag(e), ) } @@ -211,176 +208,176 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { switch t := e.(type) { case *memo.ValuesExpr: - ep, err = b.buildValues(t) + ep, outputCols, err = b.buildValues(t) case *memo.LiteralValuesExpr: - ep, err = b.buildLiteralValues(t) + ep, outputCols, err = b.buildLiteralValues(t) case *memo.ScanExpr: - ep, err = b.buildScan(t) + ep, outputCols, err = b.buildScan(t) case *memo.PlaceholderScanExpr: - ep, err = b.buildPlaceholderScan(t) + ep, outputCols, err = b.buildPlaceholderScan(t) case *memo.SelectExpr: - ep, err = b.buildSelect(t) + ep, outputCols, err = b.buildSelect(t) case *memo.ProjectExpr: - ep, err = b.buildProject(t) + ep, outputCols, err = b.buildProject(t) case *memo.GroupByExpr, *memo.ScalarGroupByExpr: - ep, err = b.buildGroupBy(e) + ep, outputCols, err = b.buildGroupBy(e) case *memo.DistinctOnExpr, *memo.EnsureDistinctOnExpr, *memo.UpsertDistinctOnExpr, *memo.EnsureUpsertDistinctOnExpr: - ep, err = b.buildDistinct(t) + ep, outputCols, err = b.buildDistinct(t) case *memo.TopKExpr: - ep, err = b.buildTopK(t) + ep, outputCols, err = b.buildTopK(t) case *memo.LimitExpr, *memo.OffsetExpr: - ep, err = b.buildLimitOffset(e) + ep, outputCols, err = b.buildLimitOffset(e) case *memo.SortExpr: - ep, err = b.buildSort(t) + ep, outputCols, err = b.buildSort(t) case *memo.DistributeExpr: - ep, err = b.buildDistribute(t) + ep, outputCols, err = b.buildDistribute(t) case *memo.IndexJoinExpr: - ep, err = b.buildIndexJoin(t) + ep, outputCols, err = b.buildIndexJoin(t) case *memo.LookupJoinExpr: - ep, err = b.buildLookupJoin(t) + ep, outputCols, err = b.buildLookupJoin(t) case *memo.InvertedJoinExpr: - ep, err = b.buildInvertedJoin(t) + ep, outputCols, err = b.buildInvertedJoin(t) case *memo.ZigzagJoinExpr: - ep, err = b.buildZigzagJoin(t) + ep, outputCols, err = b.buildZigzagJoin(t) case *memo.OrdinalityExpr: - ep, err = b.buildOrdinality(t) + ep, outputCols, err = b.buildOrdinality(t) case *memo.MergeJoinExpr: - ep, err = b.buildMergeJoin(t) + ep, outputCols, err = b.buildMergeJoin(t) case *memo.Max1RowExpr: - ep, err = b.buildMax1Row(t) + ep, outputCols, err = b.buildMax1Row(t) case *memo.ProjectSetExpr: - ep, err = b.buildProjectSet(t) + ep, outputCols, err = b.buildProjectSet(t) case *memo.WindowExpr: - ep, err = b.buildWindow(t) + ep, outputCols, err = b.buildWindow(t) case *memo.SequenceSelectExpr: - ep, err = b.buildSequenceSelect(t) + ep, outputCols, err = b.buildSequenceSelect(t) case *memo.InsertExpr: - ep, err = b.buildInsert(t) + ep, outputCols, err = b.buildInsert(t) case *memo.InvertedFilterExpr: - ep, err = b.buildInvertedFilter(t) + ep, outputCols, err = b.buildInvertedFilter(t) case *memo.UpdateExpr: - ep, err = b.buildUpdate(t) + ep, outputCols, err = b.buildUpdate(t) case *memo.UpsertExpr: - ep, err = b.buildUpsert(t) + ep, outputCols, err = b.buildUpsert(t) case *memo.DeleteExpr: - ep, err = b.buildDelete(t) + ep, outputCols, err = b.buildDelete(t) case *memo.LockExpr: - ep, err = b.buildLock(t) + ep, outputCols, err = b.buildLock(t) case *memo.BarrierExpr: - ep, err = b.buildBarrier(t) + ep, outputCols, err = b.buildBarrier(t) case *memo.CreateTableExpr: - ep, err = b.buildCreateTable(t) + ep, outputCols, err = b.buildCreateTable(t) case *memo.CreateViewExpr: - ep, err = b.buildCreateView(t) + ep, outputCols, err = b.buildCreateView(t) case *memo.CreateFunctionExpr: - ep, err = b.buildCreateFunction(t) + ep, outputCols, err = b.buildCreateFunction(t) case *memo.WithExpr: - ep, err = b.buildWith(t) + ep, outputCols, err = b.buildWith(t) case *memo.WithScanExpr: - ep, err = b.buildWithScan(t) + ep, outputCols, err = b.buildWithScan(t) case *memo.RecursiveCTEExpr: - ep, err = b.buildRecursiveCTE(t) + ep, outputCols, err = b.buildRecursiveCTE(t) case *memo.CallExpr: - ep, err = b.buildCall(t) + ep, outputCols, err = b.buildCall(t) case *memo.ExplainExpr: - ep, err = b.buildExplain(t) + ep, outputCols, err = b.buildExplain(t) case *memo.ShowTraceForSessionExpr: - ep, err = b.buildShowTrace(t) + ep, outputCols, err = b.buildShowTrace(t) case *memo.OpaqueRelExpr, *memo.OpaqueMutationExpr, *memo.OpaqueDDLExpr: - ep, err = b.buildOpaque(t.Private().(*memo.OpaqueRelPrivate)) + ep, outputCols, err = b.buildOpaque(t.Private().(*memo.OpaqueRelPrivate)) case *memo.AlterTableSplitExpr: - ep, err = b.buildAlterTableSplit(t) + ep, outputCols, err = b.buildAlterTableSplit(t) case *memo.AlterTableUnsplitExpr: - ep, err = b.buildAlterTableUnsplit(t) + ep, outputCols, err = b.buildAlterTableUnsplit(t) case *memo.AlterTableUnsplitAllExpr: - ep, err = b.buildAlterTableUnsplitAll(t) + ep, outputCols, err = b.buildAlterTableUnsplitAll(t) case *memo.AlterTableRelocateExpr: - ep, err = b.buildAlterTableRelocate(t) + ep, outputCols, err = b.buildAlterTableRelocate(t) case *memo.AlterRangeRelocateExpr: - ep, err = b.buildAlterRangeRelocate(t) + ep, outputCols, err = b.buildAlterRangeRelocate(t) case *memo.ControlJobsExpr: - ep, err = b.buildControlJobs(t) + ep, outputCols, err = b.buildControlJobs(t) case *memo.ControlSchedulesExpr: - ep, err = b.buildControlSchedules(t) + ep, outputCols, err = b.buildControlSchedules(t) case *memo.ShowCompletionsExpr: - ep, err = b.buildShowCompletions(t) + ep, outputCols, err = b.buildShowCompletions(t) case *memo.CancelQueriesExpr: - ep, err = b.buildCancelQueries(t) + ep, outputCols, err = b.buildCancelQueries(t) case *memo.CancelSessionsExpr: - ep, err = b.buildCancelSessions(t) + ep, outputCols, err = b.buildCancelSessions(t) case *memo.CreateStatisticsExpr: - ep, err = b.buildCreateStatistics(t) + ep, outputCols, err = b.buildCreateStatistics(t) case *memo.ExportExpr: - ep, err = b.buildExport(t) + ep, outputCols, err = b.buildExport(t) default: switch { case opt.IsSetOp(e): - ep, err = b.buildSetOp(e) + ep, outputCols, err = b.buildSetOp(e) case opt.IsJoinNonApplyOp(e): - ep, err = b.buildHashJoin(e) + ep, outputCols, err = b.buildHashJoin(e) case opt.IsJoinApplyOp(e): - ep, err = b.buildApplyJoin(e) + ep, outputCols, err = b.buildApplyJoin(e) default: err = errors.AssertionFailedf("no execbuild for %T", t) } } if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // In test builds, assert that the exec plan output columns match the opt @@ -388,11 +385,11 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { if buildutil.CrdbTestBuild { optCols := e.Relational().OutputCols var execCols opt.ColSet - ep.outputCols.ForEach(func(key, val int) { + outputCols.ForEach(func(key, val int) { execCols.Add(opt.ColumnID(key)) }) if !execCols.Equals(optCols) { - return execPlan{}, errors.AssertionFailedf( + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf( "exec columns do not match opt columns: expected %v, got %v. op: %T", optCols, execCols, e) } } @@ -400,17 +397,18 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { b.maybeAnnotateWithEstimates(ep.root, e) if saveTableName != "" { - ep, err = b.applySaveTable(ep, e, saveTableName) + // The output columns do not change in applySaveTable. + ep, err = b.applySaveTable(ep, outputCols, e, saveTableName) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } // Wrap the expression in a render expression if presentation requires it. if p := e.RequiredPhysical(); !p.Presentation.Any() { - ep, err = b.applyPresentation(ep, p.Presentation) + ep, outputCols, err = b.applyPresentation(ep, outputCols, p.Presentation) } - return ep, err + return ep, outputCols, err } // maybeAnnotateWithEstimates checks if we are building against an @@ -461,10 +459,12 @@ func (b *Builder) maybeAnnotateWithEstimates(node exec.Node, e memo.RelExpr) { } } -func (b *Builder) buildValues(values *memo.ValuesExpr) (execPlan, error) { +func (b *Builder) buildValues( + values *memo.ValuesExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { rows, err := b.buildValuesRows(values) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } return b.constructValues(rows, values.Cols) } @@ -502,7 +502,9 @@ func makeTypedExprMatrix(numRows, numCols int) [][]tree.TypedExpr { return rows } -func (b *Builder) constructValues(rows [][]tree.TypedExpr, cols opt.ColList) (execPlan, error) { +func (b *Builder) constructValues( + rows [][]tree.TypedExpr, cols opt.ColList, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() resultCols := make(colinfo.ResultColumns, len(cols)) for i, col := range cols { @@ -512,17 +514,19 @@ func (b *Builder) constructValues(rows [][]tree.TypedExpr, cols opt.ColList) (ex } node, err := b.factory.ConstructValues(rows, resultCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } ep := execPlan{root: node} for i, col := range cols { - ep.outputCols.Set(int(col), i) + outputCols.Set(int(col), i) } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildLiteralValues(values *memo.LiteralValuesExpr) (execPlan, error) { +func (b *Builder) buildLiteralValues( + values *memo.LiteralValuesExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() resultCols := make(colinfo.ResultColumns, len(values.ColList())) for i, col := range values.ColList() { @@ -532,14 +536,14 @@ func (b *Builder) buildLiteralValues(values *memo.LiteralValuesExpr) (execPlan, } node, err := b.factory.ConstructLiteralValues(values.Rows.Rows, resultCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } ep := execPlan{root: node} for i, col := range values.ColList() { - ep.outputCols.Set(int(col), i) + outputCols.Set(int(col), i) } - return ep, nil + return ep, outputCols, nil } // getColumns returns the set of column ordinals in the table for the set of @@ -737,7 +741,7 @@ func (b *Builder) scanParams( }, outputMap, nil } -func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { +func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() tab := md.Table(scan.Table) @@ -746,7 +750,7 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { } if scan.Flags.ForceZigzag { - return execPlan{}, fmt.Errorf("could not produce a query plan conforming to the FORCE_ZIGZAG hint") + return execPlan{}, opt.ColMap{}, fmt.Errorf("could not produce a query plan conforming to the FORCE_ZIGZAG hint") } isUnfiltered := scan.IsUnfiltered(md) @@ -756,13 +760,13 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { // user has explicitly forced the partial index *and* used NO_FULL_SCAN, we // disallow the full index scan. if isUnfiltered || (scan.Flags.ForceIndex && scan.IsFullIndexScan(md)) { - return execPlan{}, fmt.Errorf("could not produce a query plan conforming to the NO_FULL_SCAN hint") + return execPlan{}, opt.ColMap{}, fmt.Errorf("could not produce a query plan conforming to the NO_FULL_SCAN hint") } } idx := tab.Index(scan.Index) if idx.IsInverted() && len(scan.InvertedConstraint) == 0 { - return execPlan{}, + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("expected inverted index scan to have an inverted constraint") } b.IndexesUsed = util.CombineUnique(b.IndexesUsed, []string{fmt.Sprintf("%d@%d", tab.ID(), idx.ID())}) @@ -840,14 +844,14 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { } } - params, outputCols, err := b.scanParams(tab, &scan.ScanPrivate, scan.Relational(), scan.RequiredPhysical()) + var params exec.ScanParams + params, outputCols, err = b.scanParams(tab, &scan.ScanPrivate, scan.Relational(), scan.RequiredPhysical()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - res := execPlan{outputCols: outputCols} - reqOrdering, err := res.reqOrdering(scan) + reqOrdering, err := reqOrdering(scan, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } root, err := b.factory.ConstructScan( tab, @@ -856,9 +860,10 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { reqOrdering, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root = root if b.evalCtx.SessionData().EnforceHomeRegion && b.IsANSIDML && b.doScanExprCollection { if b.builtScans == nil { @@ -868,12 +873,14 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { } b.builtScans = append(b.builtScans, scan) } - return res, nil + return res, outputCols, nil } -func (b *Builder) buildPlaceholderScan(scan *memo.PlaceholderScanExpr) (execPlan, error) { +func (b *Builder) buildPlaceholderScan( + scan *memo.PlaceholderScanExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { if scan.Constraint != nil || scan.InvertedConstraint != nil { - return execPlan{}, errors.AssertionFailedf("PlaceholderScan cannot have constraints") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("PlaceholderScan cannot have constraints") } md := b.mem.Metadata() @@ -898,7 +905,7 @@ func (b *Builder) buildPlaceholderScan(scan *memo.PlaceholderScanExpr) (execPlan if p, ok := expr.(*memo.PlaceholderExpr); ok { val, err := eval.Expr(b.ctx, b.evalCtx, p.Value) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } values[i] = val } else { @@ -918,14 +925,14 @@ func (b *Builder) buildPlaceholderScan(scan *memo.PlaceholderScanExpr) (execPlan private := scan.ScanPrivate private.SetConstraint(b.evalCtx, &c) - params, outputCols, err := b.scanParams(tab, &private, scan.Relational(), scan.RequiredPhysical()) + var params exec.ScanParams + params, outputCols, err = b.scanParams(tab, &private, scan.Relational(), scan.RequiredPhysical()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - res := execPlan{outputCols: outputCols} - reqOrdering, err := res.reqOrdering(scan) + reqOrdering, err := reqOrdering(scan, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } root, err := b.factory.ConstructScan( tab, @@ -934,45 +941,48 @@ func (b *Builder) buildPlaceholderScan(scan *memo.PlaceholderScanExpr) (execPlan reqOrdering, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root = root - return res, nil + return res, outputCols, nil } -func (b *Builder) buildSelect(sel *memo.SelectExpr) (execPlan, error) { - input, err := b.buildRelational(sel.Input) +func (b *Builder) buildSelect(sel *memo.SelectExpr) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(sel.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - filter, err := b.buildScalarWithMap(input.outputCols, &sel.Filters) + filter, err := b.buildScalarWithMap(inputCols, &sel.Filters) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - // A filtering node does not modify the schema. - res := execPlan{outputCols: input.outputCols} - reqOrder, err := res.reqOrdering(sel) + reqOrder, err := reqOrdering(sel, inputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root, err = b.factory.ConstructFilter(input.root, filter, reqOrder) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return res, nil + // A filtering node does not modify the schema, so we can pass along the + // input's output columns. + return res, inputCols, nil } -func (b *Builder) buildInvertedFilter(invFilter *memo.InvertedFilterExpr) (execPlan, error) { - input, err := b.buildRelational(invFilter.Input) +func (b *Builder) buildInvertedFilter( + invFilter *memo.InvertedFilterExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(invFilter.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - // A filtering node does not modify the schema. - res := execPlan{outputCols: input.outputCols} - invertedCol, err := input.getNodeColumnOrdinal(invFilter.InvertedColumn) + var res execPlan + invertedCol, err := getNodeColumnOrdinal(inputCols, invFilter.InvertedColumn) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } var typedPreFilterExpr tree.TypedExpr var typ *types.T @@ -987,27 +997,31 @@ func (b *Builder) buildInvertedFilter(invFilter *memo.InvertedFilterExpr) (execP } typedPreFilterExpr, err = b.buildScalar(&ctx, invFilter.PreFiltererState.Expr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } typ = invFilter.PreFiltererState.Typ } res.root, err = b.factory.ConstructInvertedFilter( input.root, invFilter.InvertedExpression, typedPreFilterExpr, typ, invertedCol) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Apply a post-projection to remove the inverted column. // // TODO(rytaft): the invertedFilter used to do this post-projection, but we // had difficulty integrating that behavior. Investigate and restore that // original behavior. - return b.applySimpleProject(res, invFilter, invFilter.Relational().OutputCols, invFilter.ProvidedPhysical().Ordering) + return b.applySimpleProject(res, inputCols, invFilter, invFilter.Relational().OutputCols, invFilter.ProvidedPhysical().Ordering) } // applySimpleProject adds a simple projection on top of an existing plan. func (b *Builder) applySimpleProject( - input execPlan, inputExpr memo.RelExpr, cols opt.ColSet, providedOrd opt.Ordering, -) (execPlan, error) { + input execPlan, + inputCols opt.ColMap, + inputExpr memo.RelExpr, + cols opt.ColSet, + providedOrd opt.Ordering, +) (_ execPlan, outputCols opt.ColMap, err error) { // Since we are constructing a simple project on top of the main operator, // we need to explicitly annotate the latter with estimates since the code // in buildRelational() will attach them to the project. @@ -1016,52 +1030,52 @@ func (b *Builder) applySimpleProject( colList := make([]exec.NodeColumnOrdinal, 0, cols.Len()) var res execPlan for i, ok := cols.Next(0); ok; i, ok = cols.Next(i + 1) { - res.outputCols.Set(int(i), len(colList)) - ord, err := input.getNodeColumnOrdinal(i) + outputCols.Set(int(i), len(colList)) + ord, err := getNodeColumnOrdinal(inputCols, i) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } colList = append(colList, ord) } - var err error - sqlOrdering, err := res.sqlOrdering(providedOrd) + sqlOrdering, err := sqlOrdering(providedOrd, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } res.root, err = b.factory.ConstructSimpleProject( input.root, colList, exec.OutputOrdering(sqlOrdering), ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return res, nil + return res, outputCols, nil } -func (b *Builder) buildProject(prj *memo.ProjectExpr) (execPlan, error) { +func (b *Builder) buildProject( + prj *memo.ProjectExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() - input, err := b.buildRelational(prj.Input) + input, inputCols, err := b.buildRelational(prj.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } projections := prj.Projections if len(projections) == 0 { // We have only pass-through columns. - return b.applySimpleProject(input, prj.Input, prj.Passthrough, prj.ProvidedPhysical().Ordering) + return b.applySimpleProject(input, inputCols, prj.Input, prj.Passthrough, prj.ProvidedPhysical().Ordering) } - var res execPlan numExprs := len(projections) + prj.Passthrough.Len() exprs := make(tree.TypedExprs, 0, numExprs) cols := make(colinfo.ResultColumns, 0, numExprs) - ctx := input.makeBuildScalarCtx() + ctx := makeBuildScalarCtx(inputCols) for i := range projections { item := &projections[i] expr, err := b.buildScalar(&ctx, item.Element) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - res.outputCols.Set(int(item.Col), i) + outputCols.Set(int(item.Col), i) exprs = append(exprs, expr) cols = append(cols, colinfo.ResultColumn{ Name: md.ColumnMeta(item.Col).Alias, @@ -1069,10 +1083,10 @@ func (b *Builder) buildProject(prj *memo.ProjectExpr) (execPlan, error) { }) } for colID, ok := prj.Passthrough.Next(0); ok; colID, ok = prj.Passthrough.Next(colID + 1) { - res.outputCols.Set(int(colID), len(exprs)) + outputCols.Set(int(colID), len(exprs)) indexedVar, err := b.indexedVar(&ctx, md, colID) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } exprs = append(exprs, indexedVar) meta := md.ColumnMeta(colID) @@ -1081,26 +1095,27 @@ func (b *Builder) buildProject(prj *memo.ProjectExpr) (execPlan, error) { Typ: meta.Type, }) } - reqOrdering, err := res.reqOrdering(prj) + reqOrdering, err := reqOrdering(prj, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root, err = b.factory.ConstructRender(input.root, cols, exprs, reqOrdering) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return res, nil + return res, outputCols, nil } -func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { +func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { switch join.Op() { case opt.InnerJoinApplyOp, opt.LeftJoinApplyOp, opt.SemiJoinApplyOp, opt.AntiJoinApplyOp: default: - return execPlan{}, fmt.Errorf("couldn't execute correlated subquery with op %s", join.Op()) + return execPlan{}, opt.ColMap{}, fmt.Errorf("couldn't execute correlated subquery with op %s", join.Op()) } joinType, err := joinOpToJoinType(join.Op()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } leftExpr := join.Child(0).(memo.RelExpr) leftProps := leftExpr.Relational() @@ -1112,9 +1127,9 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { withExprs := make([]builtWithExpr, len(b.withExprs)) copy(withExprs, b.withExprs) - leftPlan, err := b.buildRelational(leftExpr) + leftPlan, leftCols, err := b.buildRelational(leftExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Make a copy of the required props for the right side. @@ -1129,9 +1144,9 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { // in the left side that contains the binding. var leftBoundColMap opt.ColMap for col, ok := leftBoundCols.Next(0); ok; col, ok = leftBoundCols.Next(col + 1) { - v, ok := leftPlan.outputCols.Get(int(col)) + v, ok := leftCols.Get(int(col)) if !ok { - return execPlan{}, fmt.Errorf("couldn't find binding column %d in left output columns", col) + return execPlan{}, opt.ColMap{}, fmt.Errorf("couldn't find binding column %d in left output columns", col) } leftBoundColMap.Set(int(col), v) } @@ -1237,7 +1252,7 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { for i := range rightRequiredProps.Presentation { rightOutputCols.Set(int(rightRequiredProps.Presentation[i].ID), i) } - allCols := joinOutputMap(leftPlan.outputCols, rightOutputCols) + allCols := joinOutputMap(leftCols, rightOutputCols) var onExpr tree.TypedExpr if len(*filters) != 0 { @@ -1247,18 +1262,17 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { } onExpr, err = b.buildScalar(&scalarCtx, filters) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } - var outputCols opt.ColMap if !joinType.ShouldIncludeRightColsInOutput() { - outputCols = leftPlan.outputCols + outputCols = leftCols } else { outputCols = allCols } - ep := execPlan{outputCols: outputCols} + var ep execPlan b.recordJoinType(joinType) b.recordJoinAlgorithm(exec.ApplyJoin) @@ -1270,9 +1284,9 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { planRightSideFn, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } // makePresentation creates a Presentation that contains the given columns, in @@ -1303,7 +1317,7 @@ func (b *Builder) presentationToResultColumns(pres physical.Presentation) colinf return result } -func (b *Builder) buildHashJoin(join memo.RelExpr) (execPlan, error) { +func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { if f := join.Private().(*memo.JoinPrivate).Flags; f.Has(memo.DisallowHashJoinStoreRight) { // We need to do a bit of reverse engineering here to determine what the // hint was. @@ -1314,14 +1328,14 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (execPlan, error) { hint = tree.AstInverted } - return execPlan{}, errors.Errorf( + return execPlan{}, opt.ColMap{}, errors.Errorf( "could not produce a query plan conforming to the %s JOIN hint", hint, ) } joinType, err := joinOpToJoinType(join.Op()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } leftExpr := join.Child(0).(memo.RelExpr) rightExpr := join.Child(1).(memo.RelExpr) @@ -1363,29 +1377,28 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (execPlan, error) { telemetry.Inc(opt.JoinTypeToUseCounter(join.Op())) } - left, right, onExpr, outputCols, err := b.initJoinBuild( + left, right, onExpr, leftCols, rightCols, outputCols, err := b.initJoinBuild( leftExpr, rightExpr, memo.ExtractRemainingJoinFilters(*filters, leftEq, rightEq), joinType, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - ep := execPlan{outputCols: outputCols} // Convert leftEq/rightEq to ordinals. eqColsBuf := make([]exec.NodeColumnOrdinal, 2*len(leftEq)) leftEqOrdinals := eqColsBuf[:len(leftEq):len(leftEq)] rightEqOrdinals := eqColsBuf[len(leftEq):] for i := range leftEq { - leftEqOrdinals[i], err = left.getNodeColumnOrdinal(leftEq[i]) + leftEqOrdinals[i], err = getNodeColumnOrdinal(leftCols, leftEq[i]) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - rightEqOrdinals[i], err = right.getNodeColumnOrdinal(rightEq[i]) + rightEqOrdinals[i], err = getNodeColumnOrdinal(rightCols, rightEq[i]) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } @@ -1398,6 +1411,7 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (execPlan, error) { } else { b.recordJoinAlgorithm(exec.HashJoin) } + var ep execPlan ep.root, err = b.factory.ConstructHashJoin( joinType, left.root, right.root, @@ -1406,12 +1420,14 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (execPlan, error) { onExpr, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildMergeJoin(join *memo.MergeJoinExpr) (execPlan, error) { +func (b *Builder) buildMergeJoin( + join *memo.MergeJoinExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { if !b.disableTelemetry { telemetry.Inc(sqltelemetry.JoinAlgoMergeUseCounter) telemetry.Inc(opt.JoinTypeToUseCounter(join.JoinType)) @@ -1419,7 +1435,7 @@ func (b *Builder) buildMergeJoin(join *memo.MergeJoinExpr) (execPlan, error) { joinType, err := joinOpToJoinType(join.JoinType) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } leftExpr, rightExpr := join.Left, join.Right leftEq, rightEq := join.LeftEq, join.RightEq @@ -1443,29 +1459,29 @@ func (b *Builder) buildMergeJoin(join *memo.MergeJoinExpr) (execPlan, error) { } } - left, right, onExpr, outputCols, err := b.initJoinBuild( + left, right, onExpr, leftCols, rightCols, outputCols, err := b.initJoinBuild( leftExpr, rightExpr, join.On, joinType, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - leftOrd, err := left.sqlOrdering(leftEq) + leftOrd, err := sqlOrdering(leftEq, leftCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - rightOrd, err := right.sqlOrdering(rightEq) + rightOrd, err := sqlOrdering(rightEq, rightCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - ep := execPlan{outputCols: outputCols} - reqOrd, err := ep.reqOrdering(join) + reqOrd, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } leftEqColsAreKey := leftExpr.Relational().FuncDeps.ColsAreStrictKey(leftEq.ColSet()) rightEqColsAreKey := rightExpr.Relational().FuncDeps.ColsAreStrictKey(rightEq.ColSet()) b.recordJoinType(joinType) b.recordJoinAlgorithm(exec.MergeJoin) + var ep execPlan ep.root, err = b.factory.ConstructMergeJoin( joinType, left.root, right.root, @@ -1474,9 +1490,9 @@ func (b *Builder) buildMergeJoin(join *memo.MergeJoinExpr) (execPlan, error) { leftEqColsAreKey, rightEqColsAreKey, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } // initJoinBuild builds the inputs to the join as well as the ON expression. @@ -1485,32 +1501,37 @@ func (b *Builder) initJoinBuild( rightChild memo.RelExpr, filters memo.FiltersExpr, joinType descpb.JoinType, -) (leftPlan, rightPlan execPlan, onExpr tree.TypedExpr, outputCols opt.ColMap, _ error) { - leftPlan, err := b.buildRelational(leftChild) +) ( + leftPlan, rightPlan execPlan, + onExpr tree.TypedExpr, + leftCols, rightCols, outputCols opt.ColMap, + _ error, +) { + leftPlan, leftCols, err := b.buildRelational(leftChild) if err != nil { - return execPlan{}, execPlan{}, nil, opt.ColMap{}, err + return execPlan{}, execPlan{}, nil, opt.ColMap{}, opt.ColMap{}, opt.ColMap{}, err } - rightPlan, err = b.buildRelational(rightChild) + rightPlan, rightCols, err = b.buildRelational(rightChild) if err != nil { - return execPlan{}, execPlan{}, nil, opt.ColMap{}, err + return execPlan{}, execPlan{}, nil, opt.ColMap{}, opt.ColMap{}, opt.ColMap{}, err } - allCols := joinOutputMap(leftPlan.outputCols, rightPlan.outputCols) + allCols := joinOutputMap(leftCols, rightCols) if len(filters) != 0 { onExpr, err = b.buildScalarWithMap(allCols, &filters) if err != nil { - return execPlan{}, execPlan{}, nil, opt.ColMap{}, err + return execPlan{}, execPlan{}, nil, opt.ColMap{}, opt.ColMap{}, opt.ColMap{}, err } } if !joinType.ShouldIncludeLeftColsInOutput() { - return leftPlan, rightPlan, onExpr, rightPlan.outputCols, nil + return leftPlan, rightPlan, onExpr, leftCols, rightCols, rightCols, nil } if !joinType.ShouldIncludeRightColsInOutput() { - return leftPlan, rightPlan, onExpr, leftPlan.outputCols, nil + return leftPlan, rightPlan, onExpr, leftCols, rightCols, leftCols, nil } - return leftPlan, rightPlan, onExpr, allCols, nil + return leftPlan, rightPlan, onExpr, leftCols, rightCols, allCols, nil } // joinOutputMap determines the outputCols map for a (non-semi/anti) join, given @@ -1550,20 +1571,21 @@ func joinOpToJoinType(op opt.Operator) (descpb.JoinType, error) { } } -func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (execPlan, error) { - input, err := b.buildGroupByInput(groupBy) +func (b *Builder) buildGroupBy( + groupBy memo.RelExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildGroupByInput(groupBy) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - var ep execPlan groupingCols := groupBy.Private().(*memo.GroupingPrivate).GroupingCols groupingColIdx := make([]exec.NodeColumnOrdinal, 0, groupingCols.Len()) for i, ok := groupingCols.Next(0); ok; i, ok = groupingCols.Next(i + 1) { - ep.outputCols.Set(int(i), len(groupingColIdx)) - ord, err := input.getNodeColumnOrdinal(i) + outputCols.Set(int(i), len(groupingColIdx)) + ord, err := getNodeColumnOrdinal(inputCols, i) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } groupingColIdx = append(groupingColIdx, ord) } @@ -1578,11 +1600,11 @@ func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (execPlan, error) { if aggFilter, ok := agg.(*memo.AggFilterExpr); ok { filter, ok := aggFilter.Filter.(*memo.VariableExpr) if !ok { - return execPlan{}, errors.AssertionFailedf("only VariableOp args supported") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("only VariableOp args supported") } - filterOrd, err = input.getNodeColumnOrdinal(filter.Col) + filterOrd, err = getNodeColumnOrdinal(inputCols, filter.Col) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } agg = aggFilter.Input } @@ -1603,16 +1625,16 @@ func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (execPlan, error) { child := agg.Child(j) if variable, ok := child.(*memo.VariableExpr); ok { if len(constArgs) != 0 { - return execPlan{}, errors.Errorf("constant args must come after variable args") + return execPlan{}, opt.ColMap{}, errors.Errorf("constant args must come after variable args") } - ord, err := input.getNodeColumnOrdinal(variable.Col) + ord, err := getNodeColumnOrdinal(inputCols, variable.Col) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } argCols = append(argCols, ord) } else { if len(argCols) == 0 { - return execPlan{}, errors.Errorf("a constant arg requires at least one variable arg") + return execPlan{}, opt.ColMap{}, errors.Errorf("a constant arg requires at least one variable arg") } constArgs = append(constArgs, memo.ExtractConstDatum(child)) } @@ -1627,24 +1649,25 @@ func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (execPlan, error) { Filter: filterOrd, DistsqlBlocklist: overload.DistsqlBlocklist, } - ep.outputCols.Set(int(item.Col), len(groupingColIdx)+i) + outputCols.Set(int(item.Col), len(groupingColIdx)+i) } + var ep execPlan if groupBy.Op() == opt.ScalarGroupByOp { ep.root, err = b.factory.ConstructScalarGroupBy(input.root, aggInfos) } else { groupBy := groupBy.(*memo.GroupByExpr) var groupingColOrder colinfo.ColumnOrdering - groupingColOrder, err = input.sqlOrdering(ordering.StreamingGroupingColOrdering( + groupingColOrder, err = sqlOrdering(ordering.StreamingGroupingColOrdering( &groupBy.GroupingPrivate, &groupBy.RequiredPhysical().Ordering, - )) + ), inputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - var reqOrdering exec.OutputOrdering - reqOrdering, err = ep.reqOrdering(groupBy) + var reqOrd exec.OutputOrdering + reqOrd, err = reqOrdering(groupBy, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } orderType := exec.GroupingOrderType(groupBy.GroupingOrderType(&groupBy.RequiredPhysical().Ordering)) var rowCount uint64 @@ -1652,70 +1675,74 @@ func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (execPlan, error) { rowCount = uint64(math.Ceil(relProps.Statistics().RowCount)) } ep.root, err = b.factory.ConstructGroupBy( - input.root, groupingColIdx, groupingColOrder, aggInfos, reqOrdering, orderType, rowCount, + input.root, groupingColIdx, groupingColOrder, aggInfos, reqOrd, orderType, rowCount, ) } if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildDistinct(distinct memo.RelExpr) (execPlan, error) { +func (b *Builder) buildDistinct( + distinct memo.RelExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { private := distinct.Private().(*memo.GroupingPrivate) if private.GroupingCols.Empty() { // A DistinctOn with no grouping columns should have been converted to a // LIMIT 1 or Max1Row by normalization rules. - return execPlan{}, fmt.Errorf("cannot execute distinct on no columns") + return execPlan{}, opt.ColMap{}, fmt.Errorf("cannot execute distinct on no columns") } - input, err := b.buildGroupByInput(distinct) + input, inputCols, err := b.buildGroupByInput(distinct) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - distinctCols, err := input.getNodeColumnOrdinalSet(private.GroupingCols) + distinctCols, err := getNodeColumnOrdinalSet(inputCols, private.GroupingCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } var orderedCols exec.NodeColumnOrdinalSet ordering := ordering.StreamingGroupingColOrdering( private, &distinct.RequiredPhysical().Ordering, ) for i := range ordering { - ord, err := input.getNodeColumnOrdinal(ordering[i].ID()) + ord, err := getNodeColumnOrdinal(inputCols, ordering[i].ID()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } orderedCols.Add(int(ord)) } - ep := execPlan{outputCols: input.outputCols} - reqOrdering, err := ep.reqOrdering(distinct) + reqOrdering, err := reqOrdering(distinct, inputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var ep execPlan ep.root, err = b.factory.ConstructDistinct( input.root, distinctCols, orderedCols, reqOrdering, private.NullsAreDistinct, private.ErrorOnDup) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // buildGroupByInput can add extra sort column(s), so discard those if they // are present by using an additional projection. outCols := distinct.Relational().OutputCols - if input.outputCols.Len() == outCols.Len() { - return ep, nil + if inputCols.Len() == outCols.Len() { + return ep, inputCols, nil } - return b.ensureColumns(ep, distinct, outCols.ToList(), distinct.ProvidedPhysical().Ordering) + return b.ensureColumns(ep, inputCols, distinct, outCols.ToList(), distinct.ProvidedPhysical().Ordering) } -func (b *Builder) buildGroupByInput(groupBy memo.RelExpr) (execPlan, error) { +func (b *Builder) buildGroupByInput( + groupBy memo.RelExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { groupByInput := groupBy.Child(0).(memo.RelExpr) - input, err := b.buildRelational(groupByInput) + input, inputCols, err := b.buildRelational(groupByInput) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // TODO(radu): this is a one-off fix for an otherwise bigger gap: we should @@ -1743,44 +1770,42 @@ func (b *Builder) buildGroupByInput(groupBy memo.RelExpr) (execPlan, error) { if neededCols.Equals(groupByInput.Relational().OutputCols) { // All columns produced by the input are used. - return input, nil + return input, inputCols, nil } // The input is producing columns that are not useful; set up a projection. cols := make([]exec.NodeColumnOrdinal, 0, neededCols.Len()) - var newOutputCols opt.ColMap for colID, ok := neededCols.Next(0); ok; colID, ok = neededCols.Next(colID + 1) { - ordinal, ordOk := input.outputCols.Get(int(colID)) + ordinal, ordOk := inputCols.Get(int(colID)) if !ordOk { - return execPlan{}, + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("needed column not produced by group-by input") } - newOutputCols.Set(int(colID), len(cols)) + outputCols.Set(int(colID), len(cols)) cols = append(cols, exec.NodeColumnOrdinal(ordinal)) } - input.outputCols = newOutputCols - reqOrdering, err := input.reqOrdering(groupByInput) + reqOrdering, err := reqOrdering(groupByInput, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } input.root, err = b.factory.ConstructSimpleProject(input.root, cols, reqOrdering) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return input, nil + return input, outputCols, nil } -func (b *Builder) buildSetOp(set memo.RelExpr) (execPlan, error) { +func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { leftExpr := set.Child(0).(memo.RelExpr) - left, err := b.buildRelational(leftExpr) + left, leftCols, err := b.buildRelational(leftExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } rightExpr := set.Child(1).(memo.RelExpr) - right, err := b.buildRelational(rightExpr) + right, rightCols, err := b.buildRelational(rightExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } private := set.Private().(*memo.SetPrivate) @@ -1804,13 +1829,13 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (execPlan, error) { // Note that (unless this is part of a larger query) the presentation property // will ensure that the columns are presented correctly in the output (i.e. in // the order `b, c, a`). - left, err = b.ensureColumns(left, leftExpr, private.LeftCols, leftExpr.ProvidedPhysical().Ordering) + left, leftCols, err = b.ensureColumns(left, leftCols, leftExpr, private.LeftCols, leftExpr.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - right, err = b.ensureColumns(right, rightExpr, private.RightCols, rightExpr.ProvidedPhysical().Ordering) + right, rightCols, err = b.ensureColumns(right, rightCols, rightExpr, private.RightCols, rightExpr.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } var typ tree.UnionType @@ -1829,7 +1854,7 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (execPlan, error) { case opt.ExceptAllOp: typ, all = tree.ExceptOp, true default: - return execPlan{}, errors.AssertionFailedf("invalid operator %s", redact.Safe(set.Op())) + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("invalid operator %s", redact.Safe(set.Op())) } switch typ { @@ -1858,21 +1883,22 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (execPlan, error) { enforceHomeRegion = b.IsANSIDML && b.evalCtx.SessionData().EnforceHomeRegion } - ep := execPlan{} for i, col := range private.OutCols { - ep.outputCols.Set(int(col), i) + outputCols.Set(int(col), i) } - streamingOrdering, err := ep.sqlOrdering( - ordering.StreamingSetOpOrdering(set, &set.RequiredPhysical().Ordering), + streamingOrdering, err := sqlOrdering( + ordering.StreamingSetOpOrdering(set, &set.RequiredPhysical().Ordering), outputCols, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - reqOrdering, err := ep.reqOrdering(set) + + reqOrdering, err := reqOrdering(set, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var ep execPlan if typ == tree.UnionOp && all { ep.root, err = b.factory.ConstructUnionAll(left.root, right.root, reqOrdering, hardLimit, enforceHomeRegion) } else if len(streamingOrdering) > 0 { @@ -1882,7 +1908,7 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (execPlan, error) { ep.root, err = b.factory.ConstructStreamingSetOp(typ, all, left.root, right.root, streamingOrdering, reqOrdering) } else { if len(reqOrdering) > 0 { - return execPlan{}, errors.AssertionFailedf("hash set op is not supported with a required ordering") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("hash set op is not supported with a required ordering") } if typ != tree.UnionOp { b.recordJoinAlgorithm(exec.HashJoin) @@ -1890,73 +1916,74 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (execPlan, error) { ep.root, err = b.factory.ConstructHashSetOp(typ, all, left.root, right.root) } if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } // buildTopK builds a plan for a TopKOp, which is like a combined SortOp and LimitOp. -func (b *Builder) buildTopK(e *memo.TopKExpr) (execPlan, error) { +func (b *Builder) buildTopK(e *memo.TopKExpr) (_ execPlan, outputCols opt.ColMap, err error) { inputExpr := e.Input - input, err := b.buildRelational(inputExpr) + input, inputCols, err := b.buildRelational(inputExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } ordering := e.Ordering.ToOrdering() inputOrdering := e.Input.ProvidedPhysical().Ordering alreadyOrderedPrefix := 0 for i := range inputOrdering { if i == len(ordering) { - return execPlan{}, errors.AssertionFailedf("sort ordering already provided by input") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("sort ordering already provided by input") } if inputOrdering[i] != ordering[i] { break } alreadyOrderedPrefix = i + 1 } - sqlOrdering, err := input.sqlOrdering(ordering) + sqlOrdering, err := sqlOrdering(ordering, inputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructTopK( + var ep execPlan + ep.root, err = b.factory.ConstructTopK( input.root, e.K, exec.OutputOrdering(sqlOrdering), alreadyOrderedPrefix) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return execPlan{root: node, outputCols: input.outputCols}, nil + return ep, inputCols, nil } // buildLimitOffset builds a plan for a LimitOp or OffsetOp -func (b *Builder) buildLimitOffset(e memo.RelExpr) (execPlan, error) { - input, err := b.buildRelational(e.Child(0).(memo.RelExpr)) +func (b *Builder) buildLimitOffset(e memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(e.Child(0).(memo.RelExpr)) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // LIMIT/OFFSET expression should never need buildScalarContext, because it // can't refer to the input expression. expr, err := b.buildScalar(nil, e.Child(1).(opt.ScalarExpr)) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - var node exec.Node + var ep execPlan if e.Op() == opt.LimitOp { - node, err = b.factory.ConstructLimit(input.root, expr, nil) + ep.root, err = b.factory.ConstructLimit(input.root, expr, nil) } else { - node, err = b.factory.ConstructLimit(input.root, nil, expr) + ep.root, err = b.factory.ConstructLimit(input.root, nil, expr) } if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return execPlan{root: node, outputCols: input.outputCols}, nil + return ep, inputCols, nil } -func (b *Builder) buildSort(sort *memo.SortExpr) (execPlan, error) { - input, err := b.buildRelational(sort.Input) +func (b *Builder) buildSort(sort *memo.SortExpr) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(sort.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } ordering := sort.ProvidedPhysical().Ordering @@ -1964,7 +1991,7 @@ func (b *Builder) buildSort(sort *memo.SortExpr) (execPlan, error) { alreadyOrderedPrefix := 0 for i := range inputOrdering { if i == len(ordering) { - return execPlan{}, errors.AssertionFailedf("sort ordering already provided by input") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("sort ordering already provided by input") } if inputOrdering[i] != ordering[i] { break @@ -1972,19 +1999,20 @@ func (b *Builder) buildSort(sort *memo.SortExpr) (execPlan, error) { alreadyOrderedPrefix = i + 1 } - sqlOrdering, err := input.sqlOrdering(ordering) + sqlOrdering, err := sqlOrdering(ordering, inputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructSort( + var ep execPlan + ep.root, err = b.factory.ConstructSort( input.root, exec.OutputOrdering(sqlOrdering), alreadyOrderedPrefix, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return execPlan{root: node, outputCols: input.outputCols}, nil + return ep, inputCols, nil } func (b *Builder) enforceScanWithHomeRegion(skipID cat.StableID) error { @@ -2082,17 +2110,19 @@ func (b *Builder) enforceScanWithHomeRegion(skipID cat.StableID) error { return nil } -func (b *Builder) buildDistribute(distribute *memo.DistributeExpr) (input execPlan, err error) { - input, err = b.buildRelational(distribute.Input) +func (b *Builder) buildDistribute( + distribute *memo.DistributeExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(distribute.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if distribute.NoOpDistribution() { // Don't bother creating a no-op distribution. This likely exists because // the input is a Sort expression, and this is an artifact of how physical // properties are enforced. - return input, err + return input, inputCols, err } if b.evalCtx.SessionData().EnforceHomeRegion && b.IsANSIDML { @@ -2113,14 +2143,15 @@ func (b *Builder) buildDistribute(distribute *memo.DistributeExpr) (input execPl b.doScanExprCollection = true // Traverse the tree again, this time collecting ScanExprs that should // be processed for error handling. - _, err = b.buildRelational(distribute.Input) + // TODO(mgartner): Can we avoid re-traversing the tree? + _, _, err = b.buildRelational(distribute.Input) b.doScanExprCollection = saveDoScanExprCollection if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } err = b.enforceScanWithHomeRegion(mutationStableID) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } homeRegion, ok := distribute.GetInputHomeRegion() @@ -2146,39 +2177,45 @@ func (b *Builder) buildDistribute(distribute *memo.DistributeExpr) (input execPl msgString := errorStringBuilder.String() err = pgerror.Newf(errCode, "%s", msgString) } - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // TODO(rytaft): This is currently a no-op. We should pass this distribution // info to the DistSQL planner. - return input, err + return input, inputCols, err } -func (b *Builder) buildOrdinality(ord *memo.OrdinalityExpr) (execPlan, error) { - input, err := b.buildRelational(ord.Input) +func (b *Builder) buildOrdinality( + ord *memo.OrdinalityExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(ord.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } colName := b.mem.Metadata().ColumnMeta(ord.ColID).Alias - node, err := b.factory.ConstructOrdinality(input.root, colName) + var ep execPlan + ep.root, err = b.factory.ConstructOrdinality(input.root, colName) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // We have one additional ordinality column, which is ordered at the end of // the list. - outputCols := input.outputCols.Copy() + // TODO(mgartner): Is the copy here necessary? + outputCols = inputCols.Copy() outputCols.Set(int(ord.ColID), outputCols.Len()) - return execPlan{root: node, outputCols: outputCols}, nil + return ep, outputCols, nil } -func (b *Builder) buildIndexJoin(join *memo.IndexJoinExpr) (execPlan, error) { - input, err := b.buildRelational(join.Input) +func (b *Builder) buildIndexJoin( + join *memo.IndexJoinExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(join.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } md := b.mem.Metadata() @@ -2190,34 +2227,34 @@ func (b *Builder) buildIndexJoin(join *memo.IndexJoinExpr) (execPlan, error) { b.IndexesUsed = util.CombineUnique(b.IndexesUsed, []string{fmt.Sprintf("%d@%d", tab.ID(), pri.ID())}) keyCols := make([]exec.NodeColumnOrdinal, pri.KeyColumnCount()) for i := range keyCols { - keyCols[i], err = input.getNodeColumnOrdinal(join.Table.ColumnID(pri.Column(i).Ordinal())) + keyCols[i], err = getNodeColumnOrdinal(inputCols, join.Table.ColumnID(pri.Column(i).Ordinal())) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } - cols := join.Cols - needed, output := b.getColumns(cols, join.Table) + var needed exec.TableColumnOrdinalSet + needed, outputCols = b.getColumns(join.Cols, join.Table) locking, err := b.buildLocking(join.Locking) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - res := execPlan{outputCols: output} b.recordJoinAlgorithm(exec.IndexJoin) - reqOrdering, err := res.reqOrdering(join) + reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root, err = b.factory.ConstructIndexJoin( input.root, tab, keyCols, needed, reqOrdering, locking, join.RequiredPhysical().LimitHintInt64(), ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return res, nil + return res, outputCols, nil } func (b *Builder) indexColumnNames( @@ -2428,7 +2465,9 @@ func (b *Builder) handleRemoteLookupJoinError(join *memo.LookupJoinExpr) (err er return nil } -func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { +func (b *Builder) buildLookupJoin( + join *memo.LookupJoinExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() if !b.disableTelemetry { telemetry.Inc(sqltelemetry.JoinAlgoLookupUseCounter) @@ -2451,9 +2490,9 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { b.doScanExprCollection = false } } - input, err := b.buildRelational(join.Input) + input, inputCols, err := b.buildRelational(join.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if enforceHomeRegion { b.doScanExprCollection = saveDoScanExprCollection @@ -2462,26 +2501,26 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { // merged. err = b.handleRemoteLookupJoinError(join) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } keyCols := make([]exec.NodeColumnOrdinal, len(join.KeyCols)) for i, c := range join.KeyCols { - keyCols[i], err = input.getNodeColumnOrdinal(c) + keyCols[i], err = getNodeColumnOrdinal(inputCols, c) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } - inputCols := join.Input.Relational().OutputCols - lookupCols := join.Cols.Difference(inputCols) + joinInputCols := join.Input.Relational().OutputCols + lookupCols := join.Cols.Difference(joinInputCols) if join.IsFirstJoinInPairedJoiner { lookupCols.Remove(join.ContinuationCol) } lookupOrdinals, lookupColMap := b.getColumns(lookupCols, join.Table) // allExprCols are the columns used in expressions evaluated by this join. - allExprCols := joinOutputMap(input.outputCols, lookupColMap) + allExprCols := joinOutputMap(inputCols, lookupColMap) allCols := allExprCols if join.IsFirstJoinInPairedJoiner { // allCols needs to include the continuation column since it will be @@ -2489,15 +2528,15 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { allCols = allExprCols.Copy() maxValue, ok := allCols.MaxValue() if !ok { - return execPlan{}, errors.AssertionFailedf("allCols should not be empty") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("allCols should not be empty") } // Assign the continuation column the next unused value in the map. allCols.Set(int(join.ContinuationCol), maxValue+1) } - res := execPlan{outputCols: allCols} + outputCols = allCols if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { // For semi and anti join, only the left columns are output. - res.outputCols = input.outputCols + outputCols = inputCols } ctx := buildScalarCtx{ @@ -2509,14 +2548,14 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { var err error lookupExpr, err = b.buildScalar(&ctx, &join.LookupExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } if len(join.RemoteLookupExpr) > 0 { var err error remoteLookupExpr, err = b.buildScalar(&ctx, &join.RemoteLookupExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } var onExpr tree.TypedExpr @@ -2524,7 +2563,7 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { var err error onExpr, err = b.buildScalar(&ctx, &join.On) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } @@ -2534,19 +2573,20 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { locking, err := b.buildLocking(join.Locking) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } joinType, err := joinOpToJoinType(join.JoinType) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } b.recordJoinType(joinType) b.recordJoinAlgorithm(exec.LookupJoin) - reqOrdering, err := res.reqOrdering(join) + reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root, err = b.factory.ConstructLookupJoin( joinType, input.root, @@ -2566,7 +2606,7 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { join.RemoteOnlyLookups, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Apply a post-projection if Cols doesn't contain all input columns. @@ -2574,14 +2614,14 @@ func (b *Builder) buildLookupJoin(join *memo.LookupJoinExpr) (execPlan, error) { // NB: For paired-joins, this is where the continuation column and the PK // columns for the right side, which were part of the inputCols, are projected // away. - if !inputCols.SubsetOf(join.Cols) { + if !joinInputCols.SubsetOf(join.Cols) { outCols := join.Cols if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { - outCols = join.Cols.Intersection(inputCols) + outCols = join.Cols.Intersection(joinInputCols) } - return b.applySimpleProject(res, join, outCols, join.ProvidedPhysical().Ordering) + return b.applySimpleProject(res, outputCols, join, outCols, join.ProvidedPhysical().Ordering) } - return res, nil + return res, outputCols, nil } func (b *Builder) handleRemoteInvertedJoinError(join *memo.InvertedJoinExpr) (err error) { @@ -2672,7 +2712,9 @@ func (b *Builder) handleRemoteInvertedJoinError(join *memo.InvertedJoinExpr) (er return nil } -func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, error) { +func (b *Builder) buildInvertedJoin( + join *memo.InvertedJoinExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { enforceHomeRegion := b.evalCtx.SessionData().EnforceHomeRegion && b.IsANSIDML saveDoScanExprCollection := false if enforceHomeRegion { @@ -2686,9 +2728,9 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro b.doScanExprCollection = false } } - input, err := b.buildRelational(join.Input) + input, inputCols, err := b.buildRelational(join.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if enforceHomeRegion { @@ -2698,7 +2740,7 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro // merged. err = b.handleRemoteInvertedJoinError(join) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } md := b.mem.Metadata() @@ -2708,14 +2750,14 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro prefixEqCols := make([]exec.NodeColumnOrdinal, len(join.PrefixKeyCols)) for i, c := range join.PrefixKeyCols { - prefixEqCols[i], err = input.getNodeColumnOrdinal(c) + prefixEqCols[i], err = getNodeColumnOrdinal(inputCols, c) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } - inputCols := join.Input.Relational().OutputCols - lookupCols := join.Cols.Difference(inputCols) + joinInputCols := join.Input.Relational().OutputCols + lookupCols := join.Cols.Difference(joinInputCols) if join.IsFirstJoinInPairedJoiner { lookupCols.Remove(join.ContinuationCol) } @@ -2729,7 +2771,7 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro lookupOrdinals, lookupColMap := b.getColumns(lookupCols, join.Table) // allExprCols are the columns used in expressions evaluated by this join. - allExprCols := joinOutputMap(input.outputCols, lookupColMap) + allExprCols := joinOutputMap(inputCols, lookupColMap) allCols := allExprCols if join.IsFirstJoinInPairedJoiner { @@ -2738,15 +2780,15 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro allCols = allExprCols.Copy() maxValue, ok := allCols.MaxValue() if !ok { - return execPlan{}, errors.AssertionFailedf("allCols should not be empty") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("allCols should not be empty") } // Assign the continuation column the next unused value in the map. allCols.Set(int(join.ContinuationCol), maxValue+1) } - res := execPlan{outputCols: allCols} + outputCols = allCols if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { // For semi and anti join, only the left columns are output. - res.outputCols = input.outputCols + outputCols = inputCols } ctx := buildScalarCtx{ @@ -2755,7 +2797,7 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro } onExpr, err := b.buildScalar(&ctx, &join.On) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // The inverted filter refers to the inverted column's source column, but it @@ -2770,24 +2812,25 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro ctx.ivarMap.Set(int(join.Table.ColumnID(invertedColumn.InvertedSourceColumnOrdinal())), ord) invertedExpr, err := b.buildScalar(&ctx, join.InvertedExpr) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } locking, err := b.buildLocking(join.Locking) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } joinType, err := joinOpToJoinType(join.JoinType) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } b.recordJoinType(joinType) b.recordJoinAlgorithm(exec.InvertedJoin) - reqOrdering, err := res.reqOrdering(join) + reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root, err = b.factory.ConstructInvertedJoin( joinType, invertedExpr, @@ -2802,14 +2845,16 @@ func (b *Builder) buildInvertedJoin(join *memo.InvertedJoinExpr) (execPlan, erro locking, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Apply a post-projection to remove the inverted column. - return b.applySimpleProject(res, join, join.Cols, join.ProvidedPhysical().Ordering) + return b.applySimpleProject(res, outputCols, join, join.Cols, join.ProvidedPhysical().Ordering) } -func (b *Builder) buildZigzagJoin(join *memo.ZigzagJoinExpr) (execPlan, error) { +func (b *Builder) buildZigzagJoin( + join *memo.ZigzagJoinExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() leftTable := md.Table(join.LeftTable) @@ -2854,16 +2899,15 @@ func (b *Builder) buildZigzagJoin(join *memo.ZigzagJoinExpr) (execPlan, error) { leftLocking, err := b.buildLocking(join.LeftLocking) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } rightLocking, err := b.buildLocking(join.RightLocking) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } allCols := joinOutputMap(leftColMap, rightColMap) - - res := execPlan{outputCols: allCols} + outputCols = allCols ctx := buildScalarCtx{ ivh: tree.MakeIndexedVarHelper(nil /* container */, leftColMap.Len()+rightColMap.Len()), @@ -2871,7 +2915,7 @@ func (b *Builder) buildZigzagJoin(join *memo.ZigzagJoinExpr) (execPlan, error) { } onExpr, err := b.buildScalar(&ctx, &join.On) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Build the fixed value scalars. @@ -2888,18 +2932,19 @@ func (b *Builder) buildZigzagJoin(join *memo.ZigzagJoinExpr) (execPlan, error) { leftFixedVals, err := tupleToExprs(join.FixedVals[0].(*memo.TupleExpr)) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } rightFixedVals, err := tupleToExprs(join.FixedVals[1].(*memo.TupleExpr)) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } b.recordJoinAlgorithm(exec.ZigZagJoin) - reqOrdering, err := res.reqOrdering(join) + reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } + var res execPlan res.root, err = b.factory.ConstructZigzagJoin( leftTable, leftIndex, @@ -2917,11 +2962,11 @@ func (b *Builder) buildZigzagJoin(join *memo.ZigzagJoinExpr) (execPlan, error) { reqOrdering, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Apply a post-projection to retain only the columns we need. - return b.applySimpleProject(res, join, join.Cols, join.ProvidedPhysical().Ordering) + return b.applySimpleProject(res, outputCols, join, join.Cols, join.ProvidedPhysical().Ordering) } func (b *Builder) buildLocking(locking opt.Locking) (opt.Locking, error) { @@ -2960,23 +3005,26 @@ func (b *Builder) buildLocking(locking opt.Locking) (opt.Locking, error) { return locking, nil } -func (b *Builder) buildMax1Row(max1Row *memo.Max1RowExpr) (execPlan, error) { - input, err := b.buildRelational(max1Row.Input) +func (b *Builder) buildMax1Row( + max1Row *memo.Max1RowExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(max1Row.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructMax1Row(input.root, max1Row.ErrorText) + var ep execPlan + ep.root, err = b.factory.ConstructMax1Row(input.root, max1Row.ErrorText) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return execPlan{root: node, outputCols: input.outputCols}, nil + return ep, inputCols, nil } -func (b *Builder) buildWith(with *memo.WithExpr) (execPlan, error) { - value, err := b.buildRelational(with.Binding) +func (b *Builder) buildWith(with *memo.WithExpr) (_ execPlan, outputCols opt.ColMap, err error) { + value, valuesCols, err := b.buildRelational(with.Binding) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } var label bytes.Buffer @@ -2987,7 +3035,7 @@ func (b *Builder) buildWith(with *memo.WithExpr) (execPlan, error) { buffer, err := b.factory.ConstructBuffer(value.root, label.String()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // TODO(justin): if the binding here has a spoolNode at its root, we can @@ -3008,28 +3056,30 @@ func (b *Builder) buildWith(with *memo.WithExpr) (execPlan, error) { RowCount: int64(with.Relational().Statistics().RowCountIfAvailable()), }) - b.addBuiltWithExpr(with.ID, value.outputCols, buffer) + b.addBuiltWithExpr(with.ID, valuesCols, buffer) return b.buildRelational(with.Main) } -func (b *Builder) buildRecursiveCTE(rec *memo.RecursiveCTEExpr) (execPlan, error) { - initial, err := b.buildRelational(rec.Initial) +func (b *Builder) buildRecursiveCTE( + rec *memo.RecursiveCTEExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + initial, initialCols, err := b.buildRelational(rec.Initial) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Make sure we have the columns in the correct order. - initial, err = b.ensureColumns(initial, rec.Initial, rec.InitialCols, nil /* ordering */) + initial, initialCols, err = b.ensureColumns(initial, initialCols, rec.Initial, rec.InitialCols, nil /* ordering */) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Renumber the columns so they match the columns expected by the recursive // query. - initial.outputCols = util.FastIntMap{} + initialCols = util.FastIntMap{} for i, col := range rec.OutCols { - initial.outputCols.Set(int(col), i) + initialCols.Set(int(col), i) } // To implement exec.RecursiveCTEIterationFn, we create a special Builder. @@ -3051,13 +3101,13 @@ func (b *Builder) buildRecursiveCTE(rec *memo.RecursiveCTEExpr) (execPlan, error // Use a separate builder each time. innerBld := *innerBldTemplate innerBld.factory = ef - innerBld.addBuiltWithExpr(rec.WithID, initial.outputCols, bufferRef) - plan, err := innerBld.build(rec.Recursive) + innerBld.addBuiltWithExpr(rec.WithID, initialCols, bufferRef) + plan, planCols, err := innerBld.build(rec.Recursive) if err != nil { return nil, err } // Ensure columns are output in the same order. - plan, err = innerBld.ensureColumns(plan, rec.Recursive, rec.RecursiveCols, opt.Ordering{}) + plan, planCols, err = innerBld.ensureColumns(plan, planCols, rec.Recursive, rec.RecursiveCols, opt.Ordering{}) if err != nil { return nil, err } @@ -3072,18 +3122,20 @@ func (b *Builder) buildRecursiveCTE(rec *memo.RecursiveCTEExpr) (execPlan, error var ep execPlan ep.root, err = b.factory.ConstructRecursiveCTE(initial.root, fn, label, rec.Deduplicate) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } for i, col := range rec.OutCols { - ep.outputCols.Set(int(col), i) + outputCols.Set(int(col), i) } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildWithScan(withScan *memo.WithScanExpr) (execPlan, error) { +func (b *Builder) buildWithScan( + withScan *memo.WithScanExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { e := b.findBuiltWithExpr(withScan.With) if e == nil { - return execPlan{}, errors.AssertionFailedf( + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf( "couldn't find With expression with ID %d", withScan.With, ) } @@ -3096,76 +3148,75 @@ func (b *Builder) buildWithScan(withScan *memo.WithScanExpr) (execPlan, error) { node, err := b.factory.ConstructScanBuffer(e.bufferNode, label.String()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - res := execPlan{root: node, outputCols: e.outputCols} + res := execPlan{root: node} // Apply any necessary projection to produce the InCols in the given order. - res, err = b.ensureColumns(res, withScan, withScan.InCols, withScan.ProvidedPhysical().Ordering) + res, _, err = b.ensureColumns(res, e.outputCols, withScan, withScan.InCols, withScan.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Renumber the columns. - res.outputCols = opt.ColMap{} for i, col := range withScan.OutCols { - res.outputCols.Set(int(col), i) + outputCols.Set(int(col), i) } - return res, nil + return res, outputCols, nil } -func (b *Builder) buildProjectSet(projectSet *memo.ProjectSetExpr) (execPlan, error) { - input, err := b.buildRelational(projectSet.Input) +func (b *Builder) buildProjectSet( + projectSet *memo.ProjectSetExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(projectSet.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } zip := projectSet.Zip md := b.mem.Metadata() - scalarCtx := input.makeBuildScalarCtx() + scalarCtx := makeBuildScalarCtx(inputCols) exprs := make(tree.TypedExprs, len(zip)) zipCols := make(colinfo.ResultColumns, 0, len(zip)) numColsPerGen := make([]int, len(zip)) - ep := execPlan{outputCols: input.outputCols} - n := ep.numOutputCols() - + n := numOutputColsInMap(inputCols) for i := range zip { item := &zip[i] exprs[i], err = b.buildScalar(&scalarCtx, item.Fn) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } for _, col := range item.Cols { colMeta := md.ColumnMeta(col) zipCols = append(zipCols, colinfo.ResultColumn{Name: colMeta.Alias, Typ: colMeta.Type}) - ep.outputCols.Set(int(col), n) + outputCols.Set(int(col), n) n++ } numColsPerGen[i] = len(item.Cols) } + var ep execPlan ep.root, err = b.factory.ConstructProjectSet(input.root, exprs, zipCols, numColsPerGen) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildCall(c *memo.CallExpr) (execPlan, error) { +func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols opt.ColMap, err error) { udf := c.Proc.(*memo.UDFCallExpr) if udf.Def == nil { - return execPlan{}, errors.AssertionFailedf("expected non-nil UDF definition") + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("expected non-nil UDF definition") } // Build the argument expressions. - var err error var args tree.TypedExprs ctx := buildScalarCtx{} if len(udf.Args) > 0 { @@ -3173,7 +3224,7 @@ func (b *Builder) buildCall(c *memo.CallExpr) (execPlan, error) { for i := range udf.Args { args[i], err = b.buildScalar(&ctx, udf.Args[i]) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } } @@ -3213,9 +3264,9 @@ func (b *Builder) buildCall(c *memo.CallExpr) (execPlan, error) { var ep execPlan ep.root, err = b.factory.ConstructCall(r) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return ep, nil + return ep, opt.ColMap{}, nil } func (b *Builder) resultColumn(id opt.ColumnID) colinfo.ResultColumn { @@ -3276,8 +3327,8 @@ func (b *Builder) isOffsetMode(boundType treewindow.WindowFrameBoundType) bool { return boundType == treewindow.OffsetPreceding || boundType == treewindow.OffsetFollowing } -func (b *Builder) buildFrame(input execPlan, w *memo.WindowsItem) (*tree.WindowFrame, error) { - scalarCtx := input.makeBuildScalarCtx() +func (b *Builder) buildFrame(inputCols opt.ColMap, w *memo.WindowsItem) (*tree.WindowFrame, error) { + scalarCtx := makeBuildScalarCtx(inputCols) newDef := &tree.WindowFrame{ Mode: w.Frame.Mode, Bounds: tree.WindowFrameBounds{ @@ -3320,10 +3371,10 @@ func (b *Builder) buildFrame(input execPlan, w *memo.WindowsItem) (*tree.WindowF return newDef, nil } -func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { - input, err := b.buildRelational(w.Input) +func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(w.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Rearrange the input so that the input has all the passthrough columns @@ -3339,12 +3390,12 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { // TODO(justin): this call to ensureColumns is kind of unfortunate because it // can result in an extra render beneath each window function. Figure out a // way to alleviate this. - input, err = b.ensureColumns(input, w, desiredCols, opt.Ordering{}) + input, inputCols, err = b.ensureColumns(input, inputCols, w, desiredCols, opt.Ordering{}) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - ctx := input.makeBuildScalarCtx() + ctx := makeBuildScalarCtx(inputCols) ord := w.Ordering.ToOrdering() @@ -3356,7 +3407,7 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { } indexedVar, err := b.indexedVar(&ctx, b.mem.Metadata(), c.ID()) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } orderingExprs[i] = &tree.Order{ Expr: indexedVar, @@ -3369,11 +3420,11 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { i := 0 for col, ok := w.Partition.Next(0); ok; col, ok = w.Partition.Next(col + 1) { - ordinal, _ := input.outputCols.Get(int(col)) + ordinal, _ := inputCols.Get(int(col)) partitionIdxs[i] = exec.NodeColumnOrdinal(ordinal) indexedVar, err := b.indexedVar(&ctx, b.mem.Metadata(), col) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } partitionExprs[i] = indexedVar i++ @@ -3399,16 +3450,16 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { col := fn.Child(j).(*memo.VariableExpr).Col indexedVar, err := b.indexedVar(&ctx, b.mem.Metadata(), col) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } args[j] = indexedVar - idx, _ := input.outputCols.Get(int(col)) + idx, _ := inputCols.Get(int(col)) argIdxs[i][j] = exec.NodeColumnOrdinal(idx) } - frame, err := b.buildFrame(input, item) + frame, err := b.buildFrame(inputCols, item) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } var builtFilter tree.TypedExpr @@ -3416,14 +3467,14 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { if ok { f, ok := filter.(*memo.VariableExpr) if !ok { - return execPlan{}, + return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("expected FILTER expression to be a VariableExpr") } - filterIdxs[i], _ = input.outputCols.Get(int(f.Col)) + filterIdxs[i], _ = inputCols.Get(int(f.Col)) builtFilter, err = b.buildScalar(&ctx, filter) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } else { filterIdxs[i] = -1 @@ -3436,7 +3487,7 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { } wrappedFn, err := b.wrapFunction(name) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } exprs[i] = tree.NewTypedFuncExpr( wrappedFn, @@ -3454,12 +3505,11 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { // All the passthrough cols will keep their ordinal index. passthrough.ForEach(func(col opt.ColumnID) { - ordinal, _ := input.outputCols.Get(int(col)) + ordinal, _ := inputCols.Get(int(col)) resultCols[ordinal] = b.resultColumn(col) }) - var outputCols opt.ColMap - input.outputCols.ForEach(func(key, val int) { + inputCols.ForEach(func(key, val int) { if passthrough.Contains(opt.ColumnID(key)) { outputCols.Set(key, val) } @@ -3477,11 +3527,12 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { outputIdxs[i] = windowStart + i } - sqlOrdering, err := input.sqlOrdering(ord) + sqlOrdering, err := sqlOrdering(ord, inputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructWindow(input.root, exec.WindowInfo{ + var ep execPlan + ep.root, err = b.factory.ConstructWindow(input.root, exec.WindowInfo{ Cols: resultCols, Exprs: exprs, OutputIdxs: outputIdxs, @@ -3491,32 +3542,31 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (execPlan, error) { Ordering: sqlOrdering, }) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return execPlan{ - root: node, - outputCols: outputCols, - }, nil + return ep, outputCols, nil } -func (b *Builder) buildSequenceSelect(seqSel *memo.SequenceSelectExpr) (execPlan, error) { +func (b *Builder) buildSequenceSelect( + seqSel *memo.SequenceSelectExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { seq := b.mem.Metadata().Sequence(seqSel.Sequence) - node, err := b.factory.ConstructSequenceSelect(seq) + var ep execPlan + ep.root, err = b.factory.ConstructSequenceSelect(seq) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - ep := execPlan{root: node} for i, c := range seqSel.Cols { - ep.outputCols.Set(int(c), i) + outputCols.Set(int(c), i) } - return ep, nil + return ep, outputCols, nil } func (b *Builder) applySaveTable( - input execPlan, e memo.RelExpr, saveTableName string, + input execPlan, inputCols opt.ColMap, e memo.RelExpr, saveTableName string, ) (execPlan, error) { name := tree.NewTableNameWithSchema(tree.Name(opt.SaveTablesDatabase), catconstants.PublicSchemaName, tree.Name(saveTableName)) @@ -3526,7 +3576,7 @@ func (b *Builder) applySaveTable( colNames := make([]string, outputCols.Len()) colNameGen := memo.NewColumnNameGenerator(e) for col, ok := outputCols.Next(0); ok; col, ok = outputCols.Next(col + 1) { - ord, _ := input.outputCols.Get(int(col)) + ord, _ := inputCols.Get(int(col)) colNames[ord] = colNameGen.GenerateName(col) } @@ -3538,21 +3588,25 @@ func (b *Builder) applySaveTable( return input, err } -func (b *Builder) buildOpaque(opaque *memo.OpaqueRelPrivate) (execPlan, error) { - node, err := b.factory.ConstructOpaque(opaque.Metadata) +func (b *Builder) buildOpaque( + opaque *memo.OpaqueRelPrivate, +) (_ execPlan, outputCols opt.ColMap, err error) { + var ep execPlan + ep.root, err = b.factory.ConstructOpaque(opaque.Metadata) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - ep := execPlan{root: node} for i, c := range opaque.Columns { - ep.outputCols.Set(int(c), i) + outputCols.Set(int(c), i) } - return ep, nil + return ep, outputCols, nil } -func (b *Builder) buildBarrier(barrier *memo.BarrierExpr) (execPlan, error) { +func (b *Builder) buildBarrier( + barrier *memo.BarrierExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { // BarrierExpr is only used as an optimization barrier. In the execution plan, // it is replaced with its input. return b.buildRelational(barrier.Input) @@ -3562,12 +3616,12 @@ func (b *Builder) buildBarrier(barrier *memo.BarrierExpr) (execPlan, error) { // to produce the given list of columns. If the input plan already produces // the columns (in the same order), returns needProj=false. func (b *Builder) needProjection( - input execPlan, colList opt.ColList, + inputCols opt.ColMap, colList opt.ColList, ) (_ []exec.NodeColumnOrdinal, needProj bool, err error) { - if input.numOutputCols() == len(colList) { + if numOutputColsInMap(inputCols) == len(colList) { identity := true for i, col := range colList { - if ord, ok := input.outputCols.Get(int(col)); !ok || ord != i { + if ord, ok := inputCols.Get(int(col)); !ok || ord != i { identity = false break } @@ -3579,7 +3633,7 @@ func (b *Builder) needProjection( cols := make([]exec.NodeColumnOrdinal, 0, len(colList)) for _, col := range colList { if col != 0 { - ord, err := input.getNodeColumnOrdinal(col) + ord, err := getNodeColumnOrdinal(inputCols, col) if err != nil { return nil, false, err } @@ -3592,50 +3646,55 @@ func (b *Builder) needProjection( // ensureColumns applies a projection as necessary to make the output match the // given list of columns; colNames is optional. func (b *Builder) ensureColumns( - input execPlan, inputExpr memo.RelExpr, colList opt.ColList, provided opt.Ordering, -) (execPlan, error) { - cols, needProj, err := b.needProjection(input, colList) + input execPlan, + inputCols opt.ColMap, + inputExpr memo.RelExpr, + colList opt.ColList, + provided opt.Ordering, +) (_ execPlan, outputCols opt.ColMap, err error) { + cols, needProj, err := b.needProjection(inputCols, colList) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if !needProj { - return input, nil + return input, inputCols, nil } // Since we are constructing a simple project on top of the main operator, // we need to explicitly annotate the latter with estimates since the code // in buildRelational() will attach them to the project. b.maybeAnnotateWithEstimates(input.root, inputExpr) - var res execPlan for i, col := range colList { - res.outputCols.Set(int(col), i) + outputCols.Set(int(col), i) } - sqlOrdering, err := res.sqlOrdering(provided) + sqlOrdering, err := sqlOrdering(provided, outputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } reqOrdering := exec.OutputOrdering(sqlOrdering) + var res execPlan res.root, err = b.factory.ConstructSimpleProject(input.root, cols, reqOrdering) - return res, err + return res, outputCols, err } // applyPresentation adds a projection to a plan to satisfy a required // Presentation property. -func (b *Builder) applyPresentation(input execPlan, pres physical.Presentation) (execPlan, error) { +func (b *Builder) applyPresentation( + input execPlan, inputCols opt.ColMap, pres physical.Presentation, +) (_ execPlan, outputCols opt.ColMap, err error) { cols := make([]exec.NodeColumnOrdinal, len(pres)) colNames := make([]string, len(pres)) var res execPlan for i := range pres { - ord, err := input.getNodeColumnOrdinal(pres[i].ID) + ord, err := getNodeColumnOrdinal(inputCols, pres[i].ID) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } cols[i] = ord - res.outputCols.Set(int(pres[i].ID), i) + outputCols.Set(int(pres[i].ID), i) colNames[i] = pres[i].Alias } - var err error res.root, err = b.factory.ConstructSerializingProject(input.root, cols, colNames) - return res, err + return res, outputCols, err } // getEnvData consolidates the information that must be presented in diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index 92bedb7a0254..d5dfbe812884 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -517,7 +517,7 @@ func (b *Builder) buildArrayFlatten( return nil, b.decorrelationError() } - root, err := b.buildRelational(af.Input) + root, _, err := b.buildRelational(af.Input) if err != nil { return nil, err } @@ -577,14 +577,14 @@ func (b *Builder) buildAny(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ } // Build the execution plan for the input subquery. - plan, err := b.buildRelational(any.Input) + plan, planCols, err := b.buildRelational(any.Input) if err != nil { return nil, err } // Construct tuple type of columns in the row. - contents := make([]*types.T, plan.numOutputCols()) - plan.outputCols.ForEach(func(key, val int) { + contents := make([]*types.T, numOutputColsInMap(planCols)) + planCols.ForEach(func(key, val int) { contents[val] = b.mem.Metadata().ColumnMeta(opt.ColumnID(key)).Type }) typs := types.MakeTuple(contents) @@ -723,7 +723,7 @@ func (b *Builder) buildExistsSubquery( // ConvertUncorrelatedExistsToCoalesceSubquery converts all uncorrelated // Exists with Coalesce+Subquery expressions. Remove this and the execution // support for the Exists mode. - plan, err := b.buildRelational(exists.Input) + plan, _, err := b.buildRelational(exists.Input) if err != nil { return nil, err } @@ -843,7 +843,7 @@ func (b *Builder) buildSubquery( eb.withExprs = withExprs eb.disableTelemetry = true eb.planLazySubqueries = true - ePlan, err := eb.buildRelational(input) + ePlan, _, err := eb.buildRelational(input) if err != nil { return err } @@ -887,7 +887,7 @@ func (b *Builder) buildSubquery( // Build the execution plan for the subquery. Note that the subquery could // have subqueries of its own which are added to b.subqueries. - plan, err := b.buildRelational(input) + plan, _, err := b.buildRelational(input) if err != nil { return nil, err } diff --git a/pkg/sql/opt/exec/execbuilder/statement.go b/pkg/sql/opt/exec/execbuilder/statement.go index a3eb751d58e7..25ce4314e0d3 100644 --- a/pkg/sql/opt/exec/execbuilder/statement.go +++ b/pkg/sql/opt/exec/execbuilder/statement.go @@ -27,31 +27,35 @@ import ( "github.com/cockroachdb/redact" ) -func (b *Builder) buildCreateTable(ct *memo.CreateTableExpr) (execPlan, error) { +func (b *Builder) buildCreateTable( + ct *memo.CreateTableExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { schema := b.mem.Metadata().Schema(ct.Schema) if !ct.Syntax.As() { root, err := b.factory.ConstructCreateTable(schema, ct.Syntax) - return execPlan{root: root}, err + return execPlan{root: root}, opt.ColMap{}, err } // Construct AS input to CREATE TABLE. - input, err := b.buildRelational(ct.Input) + input, inputCols, err := b.buildRelational(ct.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // Impose ordering and naming on input columns, so that they match the // order and names of the table columns into which values will be // inserted. - input, err = b.applyPresentation(input, ct.InputCols) + input, _, err = b.applyPresentation(input, inputCols, ct.InputCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } root, err := b.factory.ConstructCreateTableAs(input.root, schema, ct.Syntax) - return execPlan{root: root}, err + return execPlan{root: root}, opt.ColMap{}, err } -func (b *Builder) buildCreateView(cv *memo.CreateViewExpr) (execPlan, error) { +func (b *Builder) buildCreateView( + cv *memo.CreateViewExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() schema := md.Schema(cv.Schema) cols := make(colinfo.ResultColumns, len(cv.Columns)) @@ -67,10 +71,12 @@ func (b *Builder) buildCreateView(cv *memo.CreateViewExpr) (execPlan, error) { cv.Deps, cv.TypeDeps, ) - return execPlan{root: root}, err + return execPlan{root: root}, opt.ColMap{}, err } -func (b *Builder) buildCreateFunction(cf *memo.CreateFunctionExpr) (execPlan, error) { +func (b *Builder) buildCreateFunction( + cf *memo.CreateFunctionExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { md := b.mem.Metadata() schema := md.Schema(cf.Schema) root, err := b.factory.ConstructCreateFunction( @@ -79,10 +85,12 @@ func (b *Builder) buildCreateFunction(cf *memo.CreateFunctionExpr) (execPlan, er cf.Deps, cf.TypeDeps, ) - return execPlan{root: root}, err + return execPlan{root: root}, opt.ColMap{}, err } -func (b *Builder) buildExplainOpt(explain *memo.ExplainExpr) (execPlan, error) { +func (b *Builder) buildExplainOpt( + explain *memo.ExplainExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { fmtFlags := memo.ExprFmtHideAll switch { case explain.Options.Flags[tree.ExplainFlagVerbose]: @@ -137,23 +145,27 @@ func (b *Builder) buildExplainOpt(explain *memo.ExplainExpr) (execPlan, error) { var err error envOpts, err = b.getEnvData() if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } - node, err := b.factory.ConstructExplainOpt(planText.String(), envOpts) + var ep execPlan + ep.root, err = b.factory.ConstructExplainOpt(planText.String(), envOpts) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, explain.ColList), nil + return ep, outputColsFromList(explain.ColList), nil } -func (b *Builder) buildExplain(explainExpr *memo.ExplainExpr) (execPlan, error) { +func (b *Builder) buildExplain( + explainExpr *memo.ExplainExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { if explainExpr.Options.Mode == tree.ExplainOpt { return b.buildExplainOpt(explainExpr) } - node, err := b.factory.ConstructExplain( + var ep execPlan + ep.root, err = b.factory.ConstructExplain( &explainExpr.Options, explainExpr.StmtType, func(f exec.Factory) (exec.Plan, error) { @@ -179,213 +191,247 @@ func (b *Builder) buildExplain(explainExpr *memo.ExplainExpr) (execPlan, error) }, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, explainExpr.ColList), nil + return ep, outputColsFromList(explainExpr.ColList), nil } -func (b *Builder) buildShowTrace(show *memo.ShowTraceForSessionExpr) (execPlan, error) { - node, err := b.factory.ConstructShowTrace(show.TraceType, show.Compact) +func (b *Builder) buildShowTrace( + show *memo.ShowTraceForSessionExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + var ep execPlan + ep.root, err = b.factory.ConstructShowTrace(show.TraceType, show.Compact) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, show.ColList), nil + return ep, outputColsFromList(show.ColList), nil } -func (b *Builder) buildAlterTableSplit(split *memo.AlterTableSplitExpr) (execPlan, error) { - input, err := b.buildRelational(split.Input) +func (b *Builder) buildAlterTableSplit( + split *memo.AlterTableSplitExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(split.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } scalarCtx := buildScalarCtx{} expiration, err := b.buildScalar(&scalarCtx, split.Expiration) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } table := b.mem.Metadata().Table(split.Table) - node, err := b.factory.ConstructAlterTableSplit( + var ep execPlan + ep.root, err = b.factory.ConstructAlterTableSplit( table.Index(split.Index), input.root, expiration, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, split.Columns), nil + return ep, outputColsFromList(split.Columns), nil } -func (b *Builder) buildAlterTableUnsplit(unsplit *memo.AlterTableUnsplitExpr) (execPlan, error) { - input, err := b.buildRelational(unsplit.Input) +func (b *Builder) buildAlterTableUnsplit( + unsplit *memo.AlterTableUnsplitExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(unsplit.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } table := b.mem.Metadata().Table(unsplit.Table) - node, err := b.factory.ConstructAlterTableUnsplit( + var ep execPlan + ep.root, err = b.factory.ConstructAlterTableUnsplit( table.Index(unsplit.Index), input.root, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, unsplit.Columns), nil + return ep, outputColsFromList(unsplit.Columns), nil } func (b *Builder) buildAlterTableUnsplitAll( unsplitAll *memo.AlterTableUnsplitAllExpr, -) (execPlan, error) { +) (_ execPlan, outputCols opt.ColMap, err error) { table := b.mem.Metadata().Table(unsplitAll.Table) - node, err := b.factory.ConstructAlterTableUnsplitAll(table.Index(unsplitAll.Index)) + var ep execPlan + ep.root, err = b.factory.ConstructAlterTableUnsplitAll(table.Index(unsplitAll.Index)) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, unsplitAll.Columns), nil + return ep, outputColsFromList(unsplitAll.Columns), nil } -func (b *Builder) buildAlterTableRelocate(relocate *memo.AlterTableRelocateExpr) (execPlan, error) { - input, err := b.buildRelational(relocate.Input) +func (b *Builder) buildAlterTableRelocate( + relocate *memo.AlterTableRelocateExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(relocate.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } table := b.mem.Metadata().Table(relocate.Table) - node, err := b.factory.ConstructAlterTableRelocate( + var ep execPlan + ep.root, err = b.factory.ConstructAlterTableRelocate( table.Index(relocate.Index), input.root, relocate.SubjectReplicas, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, relocate.Columns), nil + return ep, outputColsFromList(relocate.Columns), nil } -func (b *Builder) buildAlterRangeRelocate(relocate *memo.AlterRangeRelocateExpr) (execPlan, error) { - input, err := b.buildRelational(relocate.Input) +func (b *Builder) buildAlterRangeRelocate( + relocate *memo.AlterRangeRelocateExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(relocate.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } scalarCtx := buildScalarCtx{} toStoreID, err := b.buildScalar(&scalarCtx, relocate.ToStoreID) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } fromStoreID, err := b.buildScalar(&scalarCtx, relocate.FromStoreID) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructAlterRangeRelocate( + var ep execPlan + ep.root, err = b.factory.ConstructAlterRangeRelocate( input.root, relocate.SubjectReplicas, toStoreID, fromStoreID, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, relocate.Columns), nil + return ep, outputColsFromList(relocate.Columns), nil } -func (b *Builder) buildControlJobs(ctl *memo.ControlJobsExpr) (execPlan, error) { - input, err := b.buildRelational(ctl.Input) +func (b *Builder) buildControlJobs( + ctl *memo.ControlJobsExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(ctl.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } scalarCtx := buildScalarCtx{} reason, err := b.buildScalar(&scalarCtx, ctl.Reason) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructControlJobs( + var ep execPlan + ep.root, err = b.factory.ConstructControlJobs( ctl.Command, input.root, reason, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // ControlJobs returns no columns. - return execPlan{root: node}, nil + return ep, opt.ColMap{}, nil } -func (b *Builder) buildControlSchedules(ctl *memo.ControlSchedulesExpr) (execPlan, error) { - input, err := b.buildRelational(ctl.Input) +func (b *Builder) buildControlSchedules( + ctl *memo.ControlSchedulesExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(ctl.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructControlSchedules( + var ep execPlan + ep.root, err = b.factory.ConstructControlSchedules( ctl.Command, input.root, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // ControlSchedules returns no columns. - return execPlan{root: node}, nil + return ep, opt.ColMap{}, nil } -func (b *Builder) buildShowCompletions(ctl *memo.ShowCompletionsExpr) (execPlan, error) { - node, err := b.factory.ConstructShowCompletions( +func (b *Builder) buildShowCompletions( + ctl *memo.ShowCompletionsExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + var ep execPlan + ep.root, err = b.factory.ConstructShowCompletions( ctl.Command, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, ctl.Columns), nil + return ep, outputColsFromList(ctl.Columns), nil } -func (b *Builder) buildCancelQueries(cancel *memo.CancelQueriesExpr) (execPlan, error) { - input, err := b.buildRelational(cancel.Input) +func (b *Builder) buildCancelQueries( + cancel *memo.CancelQueriesExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(cancel.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructCancelQueries(input.root, cancel.IfExists) + var ep execPlan + ep.root, err = b.factory.ConstructCancelQueries(input.root, cancel.IfExists) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if !b.disableTelemetry { telemetry.Inc(sqltelemetry.CancelQueriesUseCounter) } // CancelQueries returns no columns. - return execPlan{root: node}, nil + return ep, opt.ColMap{}, nil } -func (b *Builder) buildCancelSessions(cancel *memo.CancelSessionsExpr) (execPlan, error) { - input, err := b.buildRelational(cancel.Input) +func (b *Builder) buildCancelSessions( + cancel *memo.CancelSessionsExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, _, err := b.buildRelational(cancel.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } node, err := b.factory.ConstructCancelSessions(input.root, cancel.IfExists) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } if !b.disableTelemetry { telemetry.Inc(sqltelemetry.CancelSessionsUseCounter) } // CancelSessions returns no columns. - return execPlan{root: node}, nil + return execPlan{root: node}, opt.ColMap{}, nil } -func (b *Builder) buildCreateStatistics(c *memo.CreateStatisticsExpr) (execPlan, error) { +func (b *Builder) buildCreateStatistics( + c *memo.CreateStatisticsExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { node, err := b.factory.ConstructCreateStatistics(c.Syntax) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } // CreateStatistics returns no columns. - return execPlan{root: node}, nil + return execPlan{root: node}, opt.ColMap{}, nil } -func (b *Builder) buildExport(export *memo.ExportExpr) (execPlan, error) { - input, err := b.buildRelational(export.Input) +func (b *Builder) buildExport( + export *memo.ExportExpr, +) (_ execPlan, outputCols opt.ColMap, err error) { + input, inputCols, err := b.buildRelational(export.Input) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } scalarCtx := buildScalarCtx{} fileName, err := b.buildScalar(&scalarCtx, export.FileName) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } opts := make([]exec.KVOption, len(export.Options)) @@ -394,15 +440,16 @@ func (b *Builder) buildExport(export *memo.ExportExpr) (execPlan, error) { var err error opts[i].Value, err = b.buildScalar(&scalarCtx, o.Value) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } } - notNullColsSet, err := input.getNodeColumnOrdinalSet(export.Input.Relational().NotNullCols) + notNullColsSet, err := getNodeColumnOrdinalSet(inputCols, export.Input.Relational().NotNullCols) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - node, err := b.factory.ConstructExport( + var ep execPlan + ep.root, err = b.factory.ConstructExport( input.root, fileName, export.FileFormat, @@ -410,17 +457,17 @@ func (b *Builder) buildExport(export *memo.ExportExpr) (execPlan, error) { notNullColsSet, ) if err != nil { - return execPlan{}, err + return execPlan{}, opt.ColMap{}, err } - return planWithColumns(node, export.Columns), nil + return ep, outputColsFromList(export.Columns), nil } // planWithColumns creates an execPlan for a node which has a fixed output // schema. -func planWithColumns(node exec.Node, cols opt.ColList) execPlan { - ep := execPlan{root: node} +func outputColsFromList(cols opt.ColList) opt.ColMap { + var outputCols opt.ColMap for i, c := range cols { - ep.outputCols.Set(int(c), i) + outputCols.Set(int(c), i) } - return ep + return outputCols } From b5cbf605513e362c3fc4ded55be8f50ccbb7ac2b Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 8 Feb 2024 17:52:04 -0500 Subject: [PATCH 3/6] opt/exebuilder: introduce colOrdMap This commit introduces a new struct, `colOrdMap`, which maps column IDs to ordinals. See the comment for `colOrdMap` for more details. This type will be used in execbuilder in future commits to store output column mappings. Release note: None --- pkg/sql/opt/exec/execbuilder/BUILD.bazel | 2 + pkg/sql/opt/exec/execbuilder/col_ord_map.go | 125 ++++++++++++++++++ .../opt/exec/execbuilder/col_ord_map_test.go | 79 +++++++++++ 3 files changed, 206 insertions(+) create mode 100644 pkg/sql/opt/exec/execbuilder/col_ord_map.go create mode 100644 pkg/sql/opt/exec/execbuilder/col_ord_map_test.go diff --git a/pkg/sql/opt/exec/execbuilder/BUILD.bazel b/pkg/sql/opt/exec/execbuilder/BUILD.bazel index b80e06397c80..33a094573edd 100644 --- a/pkg/sql/opt/exec/execbuilder/BUILD.bazel +++ b/pkg/sql/opt/exec/execbuilder/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "builder.go", "cascades.go", + "col_ord_map.go", "format.go", "mutation.go", "relational.go", @@ -65,6 +66,7 @@ go_test( name = "execbuilder_test", size = "small", srcs = [ + "col_ord_map_test.go", "main_test.go", "mutation_test.go", ], diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map.go b/pkg/sql/opt/exec/execbuilder/col_ord_map.go new file mode 100644 index 000000000000..4b56fd004e72 --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map.go @@ -0,0 +1,125 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execbuilder + +import ( + "math" + + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/errors" +) + +// colOrdMap is a map from column IDs to ordinals. +// +// The map is implemented as a slice of integers, with the slice's indexes +// representing column IDs and the slice's elements representing ordinals. This +// makes Set and Get operations on the map extremely fast. +// +// This implementation does have some drawbacks. First, the map can only store +// column IDs less than or equal to the length of the slice. Column IDs are +// assigned as sequential integers starting at 1, so in most cases the length of +// the slice should be in the hundreds or thousands. +// +// Second, the map currently does not permit resizing, so the maximum column ID +// must be known when the map is initialized. This makes the map suitable for +// use within execbuilder after all column IDs have been assigned, but not +// elsewhere. +// +// Finally, the memory footprint of the map is dependent on the maximum column +// ID it can store, rather than on the number of entries in the map. A map with +// only a few entries has the same memory footprint as a map with every column +// ID set. This can be mitigated by reusing already-allocated maps when +// possible. +type colOrdMap struct { + // ords is a mapping from column ID to an ordinal. The values are biased by + // 1, which allows the map to store the zero ordinal and have the zero value + // in the map represent an unset column ID. + ords []int32 +} + +// newColOrdMap returns a new column mapping that can store column IDs less than +// or equal to maxCol. +func newColOrdMap(maxCol opt.ColumnID) colOrdMap { + return colOrdMap{ + ords: make([]int32, maxCol+1), + } +} + +// Set maps a column to the given ordinal. +func (m *colOrdMap) Set(col opt.ColumnID, ord int) { + if int(col) >= len(m.ords) { + panic(errors.AssertionFailedf("column %d exceeds max column of map %d", col, len(m.ords)-1)) + } + // Bias the ordinal by 1 when adding it to the map. + ord++ + if ord > math.MaxInt32 { + panic(errors.AssertionFailedf("ordinal %d exceeds max ordinal %d", ord-1, math.MaxInt32-1)) + } + m.ords[col] = int32(ord) +} + +// Get returns the current value mapped to key, or (-1, false) if the +// key is unmapped. +func (m colOrdMap) Get(col opt.ColumnID) (ord int, ok bool) { + if int(col) >= len(m.ords) { + return -1, false + } + ord = int(m.ords[col]) + if ord == 0 { + return -1, false + } + // Reverse the bias when fetching from the map. + return ord - 1, true +} + +// MaxOrd returns the maximum ordinal stored in the map, or -1 if the map is +// empty. +func (m colOrdMap) MaxOrd() int { + maxOrd := -1 + for _, ord := range m.ords { + if ord == 0 { + continue + } + // Reverse the bias when fetching the max ordinal from the map. + ord-- + maxOrd = max(maxOrd, int(ord)) + } + return maxOrd +} + +// ForEach calls the given function for each column ID and ordinal pair in the +// map. +func (m colOrdMap) ForEach(fn func(col opt.ColumnID, ord int)) { + for col, ord := range m.ords { + if ord == 0 { + continue + } + // Reverse the bias when fetching from the map. + fn(opt.ColumnID(col), int(ord-1)) + } +} + +// CopyFrom copies all entries from the given map, and unsets any column IDs not +// in the given map. +func (m *colOrdMap) CopyFrom(other colOrdMap) { + if len(m.ords) < len(other.ords) { + panic(errors.AssertionFailedf("map of size %d is too small to copy from map of size %d", + len(m.ords), len(other.ords))) + } + copy(m.ords, other.ords) +} + +// clear clears the map. The allocated memory is retained for future reuse. +func (m *colOrdMap) clear() { + for i := range m.ords { + m.ords[i] = 0 + } +} diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go b/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go new file mode 100644 index 000000000000..63ba8891de08 --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go @@ -0,0 +1,79 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execbuilder + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestColOrdMap(t *testing.T) { + defer leaktest.AfterTest(t)() + + const maxCol = 100 + m := newColOrdMap(maxCol) + oracle := make(map[opt.ColumnID]int) + + if m.MaxOrd() != -1 { + t.Errorf("expected empty map to have MaxOrd of -1, got %d", m.MaxOrd()) + } + + rng, _ := randutil.NewTestRand() + + const numOps = 1000 + for i := 0; i < numOps; i++ { + col := opt.ColumnID(rng.Intn(maxCol + 1)) + ord := int(rng.Int31()) + + oracle[col] = ord + m.Set(col, ord) + + validate(t, m, oracle) + + // Periodically clear or copy the map. + n := rng.Intn(100) + switch { + case n < 5: + oracle = make(map[opt.ColumnID]int) + m.clear() + validate(t, m, oracle) + case n < 15: + cpy := newColOrdMap(maxCol) + cpy.CopyFrom(m) + m = cpy + validate(t, m, oracle) + } + } +} + +func validate(t *testing.T, m colOrdMap, oracle map[opt.ColumnID]int) { + maxOracleOrd := -1 + for col, oracleOrd := range oracle { + if ord, ok := m.Get(col); !ok || ord != oracleOrd { + t.Errorf("expected map to contain %d:%d", col, oracleOrd) + } + maxOracleOrd = max(maxOracleOrd, oracleOrd) + } + + if m.MaxOrd() != maxOracleOrd { + t.Errorf("expected max ordinal of %d, found %d", maxOracleOrd, m.MaxOrd()) + } + + m.ForEach(func(col opt.ColumnID, ord int) { + oracleOrd, ok := oracle[col] + if !ok || ord != oracleOrd { + t.Errorf("unexpected col:ord in map %d:%d, oracle: %v", col, ord, oracle) + } + }) +} From 02b77f17ffa0ad25ab53a415f1cd408810ff0c8d Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 9 Feb 2024 15:02:22 -0500 Subject: [PATCH 4/6] opt/execbuilder: use colOrdMap to store output columns Output columns of execution nodes are now stored in `colOrdMap`s instead of `opt.ColMap`s. The `colOrdMapAllocator` struct, which is used to allocate new `colOrdMaps` has been added as a field of `Builder`. It currently is a simple implementation. Future commits will extend it to reuse allocated `colOrdMap`s when possible. Release note: None --- pkg/sql/opt/exec/execbuilder/builder.go | 16 +- pkg/sql/opt/exec/execbuilder/cascades.go | 19 +- pkg/sql/opt/exec/execbuilder/col_ord_map.go | 25 + pkg/sql/opt/exec/execbuilder/mutation.go | 114 ++-- pkg/sql/opt/exec/execbuilder/relational.go | 631 ++++++++++---------- pkg/sql/opt/exec/execbuilder/scalar.go | 14 +- pkg/sql/opt/exec/execbuilder/statement.go | 148 ++--- pkg/sql/opt/metadata.go | 5 + 8 files changed, 505 insertions(+), 467 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 284767f0e17d..7f5c6c7c5fea 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -61,6 +61,7 @@ type Builder struct { disableTelemetry bool semaCtx *tree.SemaContext evalCtx *eval.Context + colOrdsAlloc colOrdMapAllocator // subqueries accumulates information about subqueries that are part of scalar // expressions we built. Each entry is associated with a tree.Subquery @@ -205,6 +206,7 @@ func New( initialAllowAutoCommit: allowAutoCommit, IsANSIDML: isANSIDML, } + b.colOrdsAlloc.Init(mem.Metadata().MaxColumn()) if evalCtx != nil { sd := evalCtx.SessionData() if sd.SaveTablesPrefix != "" { @@ -257,7 +259,7 @@ func (b *Builder) wrapFunction(fnName string) (tree.ResolvableFunctionReference, return tree.WrapFunction(fnName), nil } -func (b *Builder) build(e opt.Expr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) build(e opt.Expr) (_ execPlan, outputCols colOrdMap, err error) { defer func() { if r := recover(); r != nil { // This code allows us to propagate errors without adding lots of checks @@ -274,7 +276,7 @@ func (b *Builder) build(e opt.Expr) (_ execPlan, outputCols opt.ColMap, err erro rel, ok := e.(memo.RelExpr) if !ok { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf( + return execPlan{}, colOrdMap{}, errors.AssertionFailedf( "building execution for non-relational operator %s", redact.Safe(e.Op()), ) } @@ -297,12 +299,12 @@ func (b *Builder) BuildScalar() (tree.TypedExpr, error) { if !ok { return nil, errors.AssertionFailedf("BuildScalar cannot be called for non-scalar operator %s", redact.Safe(b.e.Op())) } - var ctx buildScalarCtx md := b.mem.Metadata() - ctx.ivh = tree.MakeIndexedVarHelper(&mdVarContainer{md: md}, md.NumColumns()) + cols := b.colOrdsAlloc.Alloc() for i := 0; i < md.NumColumns(); i++ { - ctx.ivarMap.Set(i+1, i) + cols.Set(opt.ColumnID(i+1), i) } + ctx := makeBuildScalarCtx(cols) return b.buildScalar(&ctx, scalar) } @@ -320,11 +322,11 @@ type builtWithExpr struct { id opt.WithID // outputCols maps the output ColumnIDs of the With expression to the ordinal // positions they are output to. See execPlan.outputCols for more details. - outputCols opt.ColMap + outputCols colOrdMap bufferNode exec.Node } -func (b *Builder) addBuiltWithExpr(id opt.WithID, outputCols opt.ColMap, bufferNode exec.Node) { +func (b *Builder) addBuiltWithExpr(id opt.WithID, outputCols colOrdMap, bufferNode exec.Node) { b.withExprs = append(b.withExprs, builtWithExpr{ id: id, outputCols: outputCols, diff --git a/pkg/sql/opt/exec/execbuilder/cascades.go b/pkg/sql/opt/exec/execbuilder/cascades.go index b9b74a03e572..467e5d0cb167 100644 --- a/pkg/sql/opt/exec/execbuilder/cascades.go +++ b/pkg/sql/opt/exec/execbuilder/cascades.go @@ -113,7 +113,7 @@ type cascadeBuilder struct { mutationBuffer exec.Node // mutationBufferCols maps With column IDs from the original memo to buffer // node column ordinals; see builtWithExpr.outputCols. - mutationBufferCols opt.ColMap + mutationBufferCols colOrdMap // colMeta remembers the metadata of the With columns from the original memo. colMeta []opt.ColumnMeta @@ -142,10 +142,9 @@ func makeCascadeBuilder(b *Builder, mutationWithID opt.WithID) (*cascadeBuilder, // Remember the column metadata, as we will need to recreate it in the new // memo. md := b.mem.Metadata() - cb.colMeta = make([]opt.ColumnMeta, 0, cb.mutationBufferCols.Len()) - cb.mutationBufferCols.ForEach(func(key, val int) { - id := opt.ColumnID(key) - cb.colMeta = append(cb.colMeta, *md.ColumnMeta(id)) + cb.colMeta = make([]opt.ColumnMeta, 0, cb.mutationBufferCols.MaxOrd()) + cb.mutationBufferCols.ForEach(func(col opt.ColumnID, ord int) { + cb.colMeta = append(cb.colMeta, *md.ColumnMeta(col)) }) return cb, nil @@ -198,7 +197,7 @@ func (cb *cascadeBuilder) planCascade( var relExpr memo.RelExpr // bufferColMap is the mapping between the column IDs in the new memo and // the column ordinal in the buffer node. - var bufferColMap opt.ColMap + var bufferColMap colOrdMap if bufferRef == nil { // No input buffering. var err error @@ -219,6 +218,10 @@ func (cb *cascadeBuilder) planCascade( } else { // Set up metadata for the buffer columns. + // Allocate a map with enough capacity to store the new columns being + // added below. + bufferColMap = newColOrdMap(md.MaxColumn() + opt.ColumnID(len(cb.colMeta))) + // withColRemap is the mapping between the With column IDs in the original // memo and the corresponding column IDs in the new memo. var withColRemap opt.ColMap @@ -226,8 +229,8 @@ func (cb *cascadeBuilder) planCascade( for i := range cb.colMeta { id := md.AddColumn(cb.colMeta[i].Alias, cb.colMeta[i].Type) withCols.Add(id) - ordinal, _ := cb.mutationBufferCols.Get(int(cb.colMeta[i].MetaID)) - bufferColMap.Set(int(id), ordinal) + ordinal, _ := cb.mutationBufferCols.Get(cb.colMeta[i].MetaID) + bufferColMap.Set(id, ordinal) withColRemap.Set(int(cb.colMeta[i].MetaID), int(id)) } diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map.go b/pkg/sql/opt/exec/execbuilder/col_ord_map.go index 4b56fd004e72..d636ef897591 100644 --- a/pkg/sql/opt/exec/execbuilder/col_ord_map.go +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map.go @@ -17,6 +17,31 @@ import ( "github.com/cockroachdb/errors" ) +// colOrdMapAllocator is used to allocate colOrdMaps. +// +type colOrdMapAllocator struct { + maxCol opt.ColumnID +} + +// Init initialized the allocator that can allocate maps that support column IDs +// up to maxCol. +func (a *colOrdMapAllocator) Init(maxCol opt.ColumnID) { + a.maxCol = maxCol +} + +// Alloc returns an empty colOrdMap. It will return a previously Free'd +// colOrdMap, if one is available. +func (a *colOrdMapAllocator) Alloc() colOrdMap { + return newColOrdMap(a.maxCol) +} + +// Copy returns a copy of the given colOrdMap. +func (a *colOrdMapAllocator) Copy(from colOrdMap) colOrdMap { + m := a.Alloc() + m.CopyFrom(from) + return m +} + // colOrdMap is a map from column IDs to ordinals. // // The map is implemented as a slice of integers, with the slice's indexes diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index a4620038ec0f..27e105131947 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -30,10 +30,10 @@ import ( func (b *Builder) buildMutationInput( mutExpr, inputExpr memo.RelExpr, colList opt.ColList, p *memo.MutationPrivate, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { shouldApplyImplicitLocking, err := b.shouldApplyImplicitLockingToMutationInput(mutExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if shouldApplyImplicitLocking { // Re-entrance is not possible because mutations are never nested. @@ -43,7 +43,7 @@ func (b *Builder) buildMutationInput( input, inputCols, err := b.buildRelational(inputExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // TODO(mgartner/radu): This can incorrectly append columns in a FK cascade @@ -62,14 +62,14 @@ func (b *Builder) buildMutationInput( input, inputCols, err = b.ensureColumns(input, inputCols, inputExpr, colList, inputExpr.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if p.WithID != 0 { label := fmt.Sprintf("buffer %d", p.WithID) bufferNode, err := b.factory.ConstructBuffer(input.root, label) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } b.addBuiltWithExpr(p.WithID, inputCols, bufferNode) @@ -78,7 +78,7 @@ func (b *Builder) buildMutationInput( return input, inputCols, nil } -func (b *Builder) buildInsert(ins *memo.InsertExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildInsert(ins *memo.InsertExpr) (_ execPlan, outputCols colOrdMap, err error) { if ep, cols, ok, err := b.tryBuildFastPathInsert(ins); err != nil || ok { return ep, cols, err } @@ -90,7 +90,7 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (_ execPlan, outputCols opt. colList = appendColsWhenPresent(colList, ins.PartialIndexPutCols) input, _, err := b.buildMutationInput(ins, ins.Input, colList, &ins.MutationPrivate) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the Insert node. @@ -110,20 +110,20 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (_ execPlan, outputCols opt. len(ins.FKChecks) == 0 && len(ins.FKCascades) == 0, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the output column map. ep := execPlan{root: node} if ins.NeedResults() { - outputCols = mutationOutputColMap(ins) + outputCols = b.mutationOutputColMap(ins) } if err := b.buildUniqueChecks(ins.UniqueChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKChecks(ins.FKChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil @@ -133,7 +133,7 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (_ execPlan, outputCols opt. // checking all required conditions. See exec.Factory.ConstructInsertFastPath. func (b *Builder) tryBuildFastPathInsert( ins *memo.InsertExpr, -) (_ execPlan, outputCols opt.ColMap, ok bool, _ error) { +) (_ execPlan, outputCols colOrdMap, ok bool, _ error) { // Conditions from ConstructFastPathInsert: // // - there are no other mutations in the statement, and the output of the @@ -142,12 +142,12 @@ func (b *Builder) tryBuildFastPathInsert( // // This condition was taken into account in build(). if !b.allowInsertFastPath { - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } // If there are unique checks required, there must be the same number of fast // path unique checks. if len(ins.UniqueChecks) != len(ins.FastPathUniqueChecks) { - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } insInput := ins.Input @@ -155,7 +155,7 @@ func (b *Builder) tryBuildFastPathInsert( // Values expressions containing subqueries or UDFs, or having a size larger // than the max mutation batch size are disallowed. if !ok || !memo.ValuesLegalForInsertFastPath(values) { - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } md := b.mem.Metadata() @@ -169,7 +169,7 @@ func (b *Builder) tryBuildFastPathInsert( // uniqueness checks during fast-path insert. Even if DatumsFromConstraint // contains no Datums, that case indicates that all values to check come // from the input row. - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } execFastPathCheck := &uniqChecks[i] // Set up the execbuilder structure from the elements built during @@ -179,7 +179,7 @@ func (b *Builder) tryBuildFastPathInsert( execFastPathCheck.CheckOrdinal = c.CheckOrdinal locking, err := b.buildLocking(c.Locking) if err != nil { - return execPlan{}, opt.ColMap{}, false, err + return execPlan{}, colOrdMap{}, false, err } execFastPathCheck.Locking = locking execFastPathCheck.InsertCols = make([]exec.TableColumnOrdinal, len(c.InsertCols)) @@ -212,19 +212,19 @@ func (b *Builder) tryBuildFastPathInsert( c := &ins.FKChecks[i] if md.Table(c.ReferencedTable).ID() == md.Table(ins.Table).ID() { // Self-referencing FK. - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } fk := tab.OutboundForeignKey(c.FKOrdinal) lookupJoin, isLookupJoin := c.Check.(*memo.LookupJoinExpr) if !isLookupJoin || lookupJoin.JoinType != opt.AntiJoinOp { // Not a lookup anti-join. - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } // TODO(rytaft): see if we can remove the requirement that LookupExpr is // empty. if len(lookupJoin.On) > 0 || len(lookupJoin.LookupExpr) > 0 || len(lookupJoin.KeyCols) != fk.ColumnCount() { - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } inputExpr := lookupJoin.Input // Ignore any select (used to deal with NULLs). @@ -233,15 +233,15 @@ func (b *Builder) tryBuildFastPathInsert( } withScan, isWithScan := inputExpr.(*memo.WithScanExpr) if !isWithScan { - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } if withScan.With != ins.WithID { - return execPlan{}, opt.ColMap{}, false, nil + return execPlan{}, colOrdMap{}, false, nil } locking, err := b.buildLocking(lookupJoin.Locking) if err != nil { - return execPlan{}, opt.ColMap{}, false, err + return execPlan{}, colOrdMap{}, false, err } out := &fkChecks[i] @@ -252,12 +252,12 @@ func (b *Builder) tryBuildFastPathInsert( var withColOrd, inputColOrd int withColOrd, ok = withScan.OutCols.Find(keyCol) if !ok { - return execPlan{}, opt.ColMap{}, false, errors.AssertionFailedf("cannot find column %d", keyCol) + return execPlan{}, colOrdMap{}, false, errors.AssertionFailedf("cannot find column %d", keyCol) } inputCol := withScan.InCols[withColOrd] inputColOrd, ok = ins.InsertCols.Find(inputCol) if !ok { - return execPlan{}, opt.ColMap{}, false, errors.AssertionFailedf("cannot find column %d", inputCol) + return execPlan{}, colOrdMap{}, false, errors.AssertionFailedf("cannot find column %d", inputCol) } out.InsertCols[j] = exec.TableColumnOrdinal(inputColOrd) } @@ -296,12 +296,12 @@ func (b *Builder) tryBuildFastPathInsert( colList = appendColsWhenPresent(colList, ins.PartialIndexPutCols) rows, err := b.buildValuesRows(values) if err != nil { - return execPlan{}, opt.ColMap{}, false, err + return execPlan{}, colOrdMap{}, false, err } // We may need to rearrange the columns. rows, err = rearrangeColumns(values.Cols, rows, colList) if err != nil { - return execPlan{}, opt.ColMap{}, false, err + return execPlan{}, colOrdMap{}, false, err } // Construct the InsertFastPath node. @@ -319,12 +319,12 @@ func (b *Builder) tryBuildFastPathInsert( b.allowAutoCommit, ) if err != nil { - return execPlan{}, opt.ColMap{}, false, err + return execPlan{}, colOrdMap{}, false, err } // Construct the output column map. ep := execPlan{root: node} if ins.NeedResults() { - outputCols = mutationOutputColMap(ins) + outputCols = b.mutationOutputColMap(ins) } return ep, outputCols, true, nil } @@ -359,7 +359,7 @@ func rearrangeColumns( return outRows, nil } -func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (_ execPlan, outputCols colOrdMap, err error) { // Currently, the execution engine requires one input column for each fetch // and update expression, so use ensureColumns to map and reorder columns so // that they correspond to target table columns. For example: @@ -389,7 +389,7 @@ func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (_ execPlan, outputCols opt. input, _, err := b.buildMutationInput(upd, upd.Input, colList, &upd.MutationPrivate) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the Update node. @@ -421,30 +421,30 @@ func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (_ execPlan, outputCols opt. len(upd.FKChecks) == 0 && len(upd.FKCascades) == 0, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildUniqueChecks(upd.UniqueChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKChecks(upd.FKChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKCascades(upd.WithID, upd.FKCascades); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the output column map. ep := execPlan{root: node} if upd.NeedResults() { - outputCols = mutationOutputColMap(upd) + outputCols = b.mutationOutputColMap(upd) } return ep, outputCols, nil } -func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols colOrdMap, err error) { // Currently, the execution engine requires one input column for each insert, // fetch, and update expression, so use ensureColumns to map and reorder // columns so that they correspond to target table columns. For example: @@ -478,7 +478,7 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols opt. input, inputCols, err := b.buildMutationInput(ups, ups.Input, colList, &ups.MutationPrivate) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the Upsert node. @@ -488,7 +488,7 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols opt. if ups.CanaryCol != 0 { canaryCol, err = getNodeColumnOrdinal(inputCols, ups.CanaryCol) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } insertColOrds := ordinalSetFromColList(ups.InsertCols) @@ -511,19 +511,19 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols opt. len(ups.FKChecks) == 0 && len(ups.FKCascades) == 0, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildUniqueChecks(ups.UniqueChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKChecks(ups.FKChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKCascades(ups.WithID, ups.FKCascades); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // If UPSERT returns rows, they contain all non-mutation columns from the @@ -532,15 +532,15 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (_ execPlan, outputCols opt. // result of the UPSERT operation for that row. ep := execPlan{root: node} if ups.NeedResults() { - outputCols = mutationOutputColMap(ups) + outputCols = b.mutationOutputColMap(ups) } return ep, outputCols, nil } -func (b *Builder) buildDelete(del *memo.DeleteExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildDelete(del *memo.DeleteExpr) (_ execPlan, outputCols colOrdMap, err error) { // Check for the fast-path delete case that can use a range delete. if ep, ok, err := b.tryBuildDeleteRange(del); err != nil || ok { - return ep, opt.ColMap{}, err + return ep, colOrdMap{}, err } // Ensure that order of input columns matches order of target table columns. @@ -559,7 +559,7 @@ func (b *Builder) buildDelete(del *memo.DeleteExpr) (_ execPlan, outputCols opt. input, _, err := b.buildMutationInput(del, del.Input, colList, &del.MutationPrivate) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the Delete node. md := b.mem.Metadata() @@ -586,21 +586,21 @@ func (b *Builder) buildDelete(del *memo.DeleteExpr) (_ execPlan, outputCols opt. b.allowAutoCommit && len(del.FKChecks) == 0 && len(del.FKCascades) == 0, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKChecks(del.FKChecks); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if err := b.buildFKCascades(del.WithID, del.FKCascades); err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Construct the output column map. ep := execPlan{root: node} if del.NeedResults() { - outputCols = mutationOutputColMap(del) + outputCols = b.mutationOutputColMap(del) } return ep, outputCols, nil @@ -722,18 +722,18 @@ func ordinalSetFromColList(colList opt.OptionalColList) intsets.Fast { // mutationOutputColMap constructs a ColMap for the execPlan that maps from the // opt.ColumnID of each output column to the ordinal position of that column in // the result. -func mutationOutputColMap(mutation memo.RelExpr) opt.ColMap { +func (b *Builder) mutationOutputColMap(mutation memo.RelExpr) colOrdMap { private := mutation.Private().(*memo.MutationPrivate) tab := mutation.Memo().Metadata().Table(private.Table) outCols := mutation.Relational().OutputCols - var colMap opt.ColMap + colMap := b.colOrdsAlloc.Alloc() ord := 0 for i, n := 0, tab.ColumnCount(); i < n; i++ { colID := private.Table.ColumnID(i) // System columns should not be included in mutations. if outCols.Contains(colID) && tab.Column(i).Kind() != cat.System { - colMap.Set(int(colID), ord) + colMap.Set(colID, ord) ord++ } } @@ -742,7 +742,7 @@ func mutationOutputColMap(mutation memo.RelExpr) opt.ColMap { // columns it allowed to pass through. for _, colID := range private.PassthroughCols { if colID != 0 { - colMap.Set(int(colID), ord) + colMap.Set(colID, ord) ord++ } } @@ -1142,11 +1142,11 @@ func unwrapProjectExprs(input memo.RelExpr) memo.RelExpr { return input } -func (b *Builder) buildLock(lock *memo.LockExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildLock(lock *memo.LockExpr) (_ execPlan, outputCols colOrdMap, err error) { // Don't bother creating the lookup join if we don't need it. locking, err := b.buildLocking(lock.Locking) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if !locking.IsLocking() { return b.buildRelational(lock.Input) diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 6e69d6d76bc1..7f5fde60c93c 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -59,21 +59,11 @@ type execPlan struct { root exec.Node } -// numOutputColsInMap returns the number of slots required to fill in all of -// the columns referred to by this ColMap. -func numOutputColsInMap(m opt.ColMap) int { - max, ok := m.MaxValue() - if !ok { - return 0 - } - return max + 1 -} - // makeBuildScalarCtx returns a buildScalarCtx that can be used with expressions // that refer the output columns of this plan. -func makeBuildScalarCtx(cols opt.ColMap) buildScalarCtx { +func makeBuildScalarCtx(cols colOrdMap) buildScalarCtx { return buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, numOutputColsInMap(cols)), + ivh: tree.MakeIndexedVarHelper(nil /* container */, cols.MaxOrd()+1), ivarMap: cols, } } @@ -81,17 +71,15 @@ func makeBuildScalarCtx(cols opt.ColMap) buildScalarCtx { // getNodeColumnOrdinal takes a column that is known to be produced by the execPlan // and returns the ordinal index of that column in the result columns of the // node. -func getNodeColumnOrdinal(colMap opt.ColMap, col opt.ColumnID) (exec.NodeColumnOrdinal, error) { - ord, ok := colMap.Get(int(col)) +func getNodeColumnOrdinal(colMap colOrdMap, col opt.ColumnID) (exec.NodeColumnOrdinal, error) { + ord, ok := colMap.Get(col) if !ok { return 0, errors.AssertionFailedf("column %d not in input", redact.Safe(col)) } return exec.NodeColumnOrdinal(ord), nil } -func getNodeColumnOrdinalSet( - colMap opt.ColMap, cols opt.ColSet, -) (exec.NodeColumnOrdinalSet, error) { +func getNodeColumnOrdinalSet(colMap colOrdMap, cols opt.ColSet) (exec.NodeColumnOrdinalSet, error) { var res exec.NodeColumnOrdinalSet for colID, ok := cols.Next(0); ok; colID, ok = cols.Next(colID + 1) { colOrd, err := getNodeColumnOrdinal(colMap, colID) @@ -105,14 +93,14 @@ func getNodeColumnOrdinalSet( // reqOrdering converts the provided ordering of a relational expression to an // OutputOrdering (according to the outputCols map). -func reqOrdering(expr memo.RelExpr, cols opt.ColMap) (exec.OutputOrdering, error) { +func reqOrdering(expr memo.RelExpr, cols colOrdMap) (exec.OutputOrdering, error) { ordering, err := sqlOrdering(expr.ProvidedPhysical().Ordering, cols) return exec.OutputOrdering(ordering), err } // sqlOrdering converts an Ordering to a ColumnOrdering (according to the // outputCols map). -func sqlOrdering(ordering opt.Ordering, cols opt.ColMap) (colinfo.ColumnOrdering, error) { +func sqlOrdering(ordering opt.Ordering, cols colOrdMap) (colinfo.ColumnOrdering, error) { if ordering.Empty() { return nil, nil } @@ -161,8 +149,8 @@ func sqlOrdering(ordering opt.Ordering, cols opt.ColMap) (colinfo.ColumnOrdering // // Note: conceptually, this could be a ColList; however, the map is more // convenient when converting VariableOps to IndexedVars. -// outputCols opt.ColMap -func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { +// outputCols colOrdMap +func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { var ep execPlan if opt.IsDDLOp(e) { @@ -175,7 +163,7 @@ func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.Co b.flags.Set(exec.PlanFlagContainsMutation) // Raise error if mutation op is part of a read-only transaction. if b.evalCtx.TxnReadOnly { - return execPlan{}, opt.ColMap{}, pgerror.Newf(pgcode.ReadOnlySQLTransaction, + return execPlan{}, colOrdMap{}, pgerror.Newf(pgcode.ReadOnlySQLTransaction, "cannot execute %s in a read-only transaction", b.statementTag(e)) } } @@ -183,7 +171,7 @@ func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.Co // Raise error if bounded staleness is used incorrectly. if b.boundedStaleness() { if _, ok := boundedStalenessAllowList[e.Op()]; !ok { - return execPlan{}, opt.ColMap{}, unimplemented.NewWithIssuef(67562, + return execPlan{}, colOrdMap{}, unimplemented.NewWithIssuef(67562, "cannot use bounded staleness for %s", b.statementTag(e), ) } @@ -377,7 +365,7 @@ func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.Co } } if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // In test builds, assert that the exec plan output columns match the opt @@ -385,11 +373,11 @@ func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.Co if buildutil.CrdbTestBuild { optCols := e.Relational().OutputCols var execCols opt.ColSet - outputCols.ForEach(func(key, val int) { - execCols.Add(opt.ColumnID(key)) + outputCols.ForEach(func(col opt.ColumnID, ord int) { + execCols.Add(col) }) if !execCols.Equals(optCols) { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf( + return execPlan{}, colOrdMap{}, errors.AssertionFailedf( "exec columns do not match opt columns: expected %v, got %v. op: %T", optCols, execCols, e) } } @@ -400,7 +388,7 @@ func (b *Builder) buildRelational(e memo.RelExpr) (_ execPlan, outputCols opt.Co // The output columns do not change in applySaveTable. ep, err = b.applySaveTable(ep, outputCols, e, saveTableName) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -461,10 +449,10 @@ func (b *Builder) maybeAnnotateWithEstimates(node exec.Node, e memo.RelExpr) { func (b *Builder) buildValues( values *memo.ValuesExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { rows, err := b.buildValuesRows(values) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return b.constructValues(rows, values.Cols) } @@ -504,7 +492,7 @@ func makeTypedExprMatrix(numRows, numCols int) [][]tree.TypedExpr { func (b *Builder) constructValues( rows [][]tree.TypedExpr, cols opt.ColList, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() resultCols := make(colinfo.ResultColumns, len(cols)) for i, col := range cols { @@ -514,11 +502,12 @@ func (b *Builder) constructValues( } node, err := b.factory.ConstructValues(rows, resultCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } ep := execPlan{root: node} + outputCols = b.colOrdsAlloc.Alloc() for i, col := range cols { - outputCols.Set(int(col), i) + outputCols.Set(col, i) } return ep, outputCols, nil @@ -526,7 +515,7 @@ func (b *Builder) constructValues( func (b *Builder) buildLiteralValues( values *memo.LiteralValuesExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() resultCols := make(colinfo.ResultColumns, len(values.ColList())) for i, col := range values.ColList() { @@ -536,11 +525,12 @@ func (b *Builder) buildLiteralValues( } node, err := b.factory.ConstructLiteralValues(values.Rows.Rows, resultCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } ep := execPlan{root: node} + outputCols = b.colOrdsAlloc.Alloc() for i, col := range values.ColList() { - outputCols.Set(int(col), i) + outputCols.Set(col, i) } return ep, outputCols, nil @@ -551,9 +541,9 @@ func (b *Builder) buildLiteralValues( // (starting with 0). func (b *Builder) getColumns( cols opt.ColSet, tableID opt.TableID, -) (exec.TableColumnOrdinalSet, opt.ColMap) { +) (exec.TableColumnOrdinalSet, colOrdMap) { var needed exec.TableColumnOrdinalSet - var output opt.ColMap + output := b.colOrdsAlloc.Alloc() columnCount := b.mem.Metadata().Table(tableID).ColumnCount() n := 0 @@ -561,7 +551,7 @@ func (b *Builder) getColumns( colID := tableID.ColumnID(i) if cols.Contains(colID) { needed.Add(i) - output.Set(int(colID), n) + output.Set(colID, n) n++ } } @@ -595,7 +585,7 @@ func (b *Builder) indexConstraintMaxResults( // scanParams populates ScanParams and the output column mapping. func (b *Builder) scanParams( tab cat.Table, scan *memo.ScanPrivate, relProps *props.Relational, reqProps *physical.Required, -) (exec.ScanParams, opt.ColMap, error) { +) (exec.ScanParams, colOrdMap, error) { // Check if we tried to force a specific index but there was no Scan with that // index in the memo. if scan.Flags.ForceIndex && scan.Flags.Index != scan.Index { @@ -632,13 +622,13 @@ func (b *Builder) scanParams( } } - return exec.ScanParams{}, opt.ColMap{}, err + return exec.ScanParams{}, colOrdMap{}, err } locking, err := b.buildLocking(scan.Locking) if err != nil { - return exec.ScanParams{}, opt.ColMap{}, err + return exec.ScanParams{}, colOrdMap{}, err } needed, outputMap := b.getColumns(scan.Cols, scan.Table) @@ -690,7 +680,7 @@ func (b *Builder) scanParams( valid = maxResultsOk && maxResults == 1 } if !valid { - return exec.ScanParams{}, opt.ColMap{}, unimplemented.NewWithIssuef(67562, + return exec.ScanParams{}, colOrdMap{}, unimplemented.NewWithIssuef(67562, "cannot use bounded staleness for queries that may touch more than one range or require an index join", ) } @@ -724,7 +714,7 @@ func (b *Builder) scanParams( &reqProps.Ordering, ) if !ok { - return exec.ScanParams{}, opt.ColMap{}, errors.AssertionFailedf("scan can't provide required ordering") + return exec.ScanParams{}, colOrdMap{}, errors.AssertionFailedf("scan can't provide required ordering") } return exec.ScanParams{ @@ -741,7 +731,7 @@ func (b *Builder) scanParams( }, outputMap, nil } -func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() tab := md.Table(scan.Table) @@ -750,7 +740,7 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.Col } if scan.Flags.ForceZigzag { - return execPlan{}, opt.ColMap{}, fmt.Errorf("could not produce a query plan conforming to the FORCE_ZIGZAG hint") + return execPlan{}, colOrdMap{}, fmt.Errorf("could not produce a query plan conforming to the FORCE_ZIGZAG hint") } isUnfiltered := scan.IsUnfiltered(md) @@ -760,13 +750,13 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.Col // user has explicitly forced the partial index *and* used NO_FULL_SCAN, we // disallow the full index scan. if isUnfiltered || (scan.Flags.ForceIndex && scan.IsFullIndexScan(md)) { - return execPlan{}, opt.ColMap{}, fmt.Errorf("could not produce a query plan conforming to the NO_FULL_SCAN hint") + return execPlan{}, colOrdMap{}, fmt.Errorf("could not produce a query plan conforming to the NO_FULL_SCAN hint") } } idx := tab.Index(scan.Index) if idx.IsInverted() && len(scan.InvertedConstraint) == 0 { - return execPlan{}, opt.ColMap{}, + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("expected inverted index scan to have an inverted constraint") } b.IndexesUsed = util.CombineUnique(b.IndexesUsed, []string{fmt.Sprintf("%d@%d", tab.ID(), idx.ID())}) @@ -847,11 +837,11 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.Col var params exec.ScanParams params, outputCols, err = b.scanParams(tab, &scan.ScanPrivate, scan.Relational(), scan.RequiredPhysical()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } reqOrdering, err := reqOrdering(scan, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } root, err := b.factory.ConstructScan( tab, @@ -860,7 +850,7 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.Col reqOrdering, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan @@ -878,9 +868,9 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols opt.Col func (b *Builder) buildPlaceholderScan( scan *memo.PlaceholderScanExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { if scan.Constraint != nil || scan.InvertedConstraint != nil { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("PlaceholderScan cannot have constraints") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("PlaceholderScan cannot have constraints") } md := b.mem.Metadata() @@ -905,7 +895,7 @@ func (b *Builder) buildPlaceholderScan( if p, ok := expr.(*memo.PlaceholderExpr); ok { val, err := eval.Expr(b.ctx, b.evalCtx, p.Value) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } values[i] = val } else { @@ -928,11 +918,11 @@ func (b *Builder) buildPlaceholderScan( var params exec.ScanParams params, outputCols, err = b.scanParams(tab, &private, scan.Relational(), scan.RequiredPhysical()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } reqOrdering, err := reqOrdering(scan, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } root, err := b.factory.ConstructScan( tab, @@ -941,7 +931,7 @@ func (b *Builder) buildPlaceholderScan( reqOrdering, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan @@ -949,23 +939,23 @@ func (b *Builder) buildPlaceholderScan( return res, outputCols, nil } -func (b *Builder) buildSelect(sel *memo.SelectExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildSelect(sel *memo.SelectExpr) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(sel.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } filter, err := b.buildScalarWithMap(inputCols, &sel.Filters) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } reqOrder, err := reqOrdering(sel, inputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan res.root, err = b.factory.ConstructFilter(input.root, filter, reqOrder) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // A filtering node does not modify the schema, so we can pass along the // input's output columns. @@ -974,37 +964,37 @@ func (b *Builder) buildSelect(sel *memo.SelectExpr) (_ execPlan, outputCols opt. func (b *Builder) buildInvertedFilter( invFilter *memo.InvertedFilterExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(invFilter.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan invertedCol, err := getNodeColumnOrdinal(inputCols, invFilter.InvertedColumn) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var typedPreFilterExpr tree.TypedExpr var typ *types.T if invFilter.PreFiltererState != nil && invFilter.PreFiltererState.Expr != nil { // The expression has a single variable, corresponding to the indexed // column. We assign it an ordinal of 0. - var colMap opt.ColMap - colMap.Set(int(invFilter.PreFiltererState.Col), 0) + colMap := b.colOrdsAlloc.Alloc() + colMap.Set(invFilter.PreFiltererState.Col, 0) ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, colMap.Len()), + ivh: tree.MakeIndexedVarHelper(nil /* container */, 1), ivarMap: colMap, } typedPreFilterExpr, err = b.buildScalar(&ctx, invFilter.PreFiltererState.Expr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } typ = invFilter.PreFiltererState.Typ } res.root, err = b.factory.ConstructInvertedFilter( input.root, invFilter.InvertedExpression, typedPreFilterExpr, typ, invertedCol) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Apply a post-projection to remove the inverted column. // @@ -1017,11 +1007,11 @@ func (b *Builder) buildInvertedFilter( // applySimpleProject adds a simple projection on top of an existing plan. func (b *Builder) applySimpleProject( input execPlan, - inputCols opt.ColMap, + inputCols colOrdMap, inputExpr memo.RelExpr, cols opt.ColSet, providedOrd opt.Ordering, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { // Since we are constructing a simple project on top of the main operator, // we need to explicitly annotate the latter with estimates since the code // in buildRelational() will attach them to the project. @@ -1029,34 +1019,35 @@ func (b *Builder) applySimpleProject( // We have only pass-through columns. colList := make([]exec.NodeColumnOrdinal, 0, cols.Len()) var res execPlan + outputCols = b.colOrdsAlloc.Alloc() for i, ok := cols.Next(0); ok; i, ok = cols.Next(i + 1) { - outputCols.Set(int(i), len(colList)) + outputCols.Set(i, len(colList)) ord, err := getNodeColumnOrdinal(inputCols, i) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } colList = append(colList, ord) } sqlOrdering, err := sqlOrdering(providedOrd, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } res.root, err = b.factory.ConstructSimpleProject( input.root, colList, exec.OutputOrdering(sqlOrdering), ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return res, outputCols, nil } func (b *Builder) buildProject( prj *memo.ProjectExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() input, inputCols, err := b.buildRelational(prj.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } projections := prj.Projections @@ -1069,13 +1060,14 @@ func (b *Builder) buildProject( exprs := make(tree.TypedExprs, 0, numExprs) cols := make(colinfo.ResultColumns, 0, numExprs) ctx := makeBuildScalarCtx(inputCols) + outputCols = b.colOrdsAlloc.Alloc() for i := range projections { item := &projections[i] expr, err := b.buildScalar(&ctx, item.Element) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - outputCols.Set(int(item.Col), i) + outputCols.Set(item.Col, i) exprs = append(exprs, expr) cols = append(cols, colinfo.ResultColumn{ Name: md.ColumnMeta(item.Col).Alias, @@ -1083,10 +1075,10 @@ func (b *Builder) buildProject( }) } for colID, ok := prj.Passthrough.Next(0); ok; colID, ok = prj.Passthrough.Next(colID + 1) { - outputCols.Set(int(colID), len(exprs)) + outputCols.Set(colID, len(exprs)) indexedVar, err := b.indexedVar(&ctx, md, colID) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } exprs = append(exprs, indexedVar) meta := md.ColumnMeta(colID) @@ -1097,25 +1089,25 @@ func (b *Builder) buildProject( } reqOrdering, err := reqOrdering(prj, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan res.root, err = b.factory.ConstructRender(input.root, cols, exprs, reqOrdering) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return res, outputCols, nil } -func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { switch join.Op() { case opt.InnerJoinApplyOp, opt.LeftJoinApplyOp, opt.SemiJoinApplyOp, opt.AntiJoinApplyOp: default: - return execPlan{}, opt.ColMap{}, fmt.Errorf("couldn't execute correlated subquery with op %s", join.Op()) + return execPlan{}, colOrdMap{}, fmt.Errorf("couldn't execute correlated subquery with op %s", join.Op()) } joinType, err := joinOpToJoinType(join.Op()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } leftExpr := join.Child(0).(memo.RelExpr) leftProps := leftExpr.Relational() @@ -1129,7 +1121,7 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt. leftPlan, leftCols, err := b.buildRelational(leftExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Make a copy of the required props for the right side. @@ -1142,13 +1134,13 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt. // leftBoundColMap is a map from opt.ColumnID to opt.ColumnOrdinal that maps // a column bound by the left side of this apply join to the column ordinal // in the left side that contains the binding. - var leftBoundColMap opt.ColMap + leftBoundColMap := b.colOrdsAlloc.Alloc() for col, ok := leftBoundCols.Next(0); ok; col, ok = leftBoundCols.Next(col + 1) { - v, ok := leftCols.Get(int(col)) + v, ok := leftCols.Get(col) if !ok { - return execPlan{}, opt.ColMap{}, fmt.Errorf("couldn't find binding column %d in left output columns", col) + return execPlan{}, colOrdMap{}, fmt.Errorf("couldn't find binding column %d in left output columns", col) } - leftBoundColMap.Set(int(col), v) + leftBoundColMap.Set(col, v) } // Now, the cool part! We set up an ApplyJoinPlanRightSideFn which plans the @@ -1192,7 +1184,7 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt. replaceFn = func(e opt.Expr) opt.Expr { switch t := e.(type) { case *memo.VariableExpr: - if leftOrd, ok := leftBoundColMap.Get(int(t.Col)); ok { + if leftOrd, ok := leftBoundColMap.Get(t.Col); ok { return f.ConstructConstVal(leftRow[leftOrd], t.Typ) } @@ -1248,21 +1240,21 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt. // The right plan will always produce the columns in the presentation, in // the same order. - var rightOutputCols opt.ColMap + rightOutputCols := b.colOrdsAlloc.Alloc() for i := range rightRequiredProps.Presentation { - rightOutputCols.Set(int(rightRequiredProps.Presentation[i].ID), i) + rightOutputCols.Set(rightRequiredProps.Presentation[i].ID, i) } - allCols := joinOutputMap(leftCols, rightOutputCols) + allCols := b.joinOutputMap(leftCols, rightOutputCols) var onExpr tree.TypedExpr if len(*filters) != 0 { scalarCtx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, numOutputColsInMap(allCols)), + ivh: tree.MakeIndexedVarHelper(nil /* container */, allCols.MaxOrd()+1), ivarMap: allCols, } onExpr, err = b.buildScalar(&scalarCtx, filters) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -1284,7 +1276,7 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols opt. planRightSideFn, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil } @@ -1317,7 +1309,7 @@ func (b *Builder) presentationToResultColumns(pres physical.Presentation) colinf return result } -func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { if f := join.Private().(*memo.JoinPrivate).Flags; f.Has(memo.DisallowHashJoinStoreRight) { // We need to do a bit of reverse engineering here to determine what the // hint was. @@ -1328,14 +1320,14 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols opt.C hint = tree.AstInverted } - return execPlan{}, opt.ColMap{}, errors.Errorf( + return execPlan{}, colOrdMap{}, errors.Errorf( "could not produce a query plan conforming to the %s JOIN hint", hint, ) } joinType, err := joinOpToJoinType(join.Op()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } leftExpr := join.Child(0).(memo.RelExpr) rightExpr := join.Child(1).(memo.RelExpr) @@ -1384,7 +1376,7 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols opt.C joinType, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Convert leftEq/rightEq to ordinals. @@ -1394,11 +1386,11 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols opt.C for i := range leftEq { leftEqOrdinals[i], err = getNodeColumnOrdinal(leftCols, leftEq[i]) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } rightEqOrdinals[i], err = getNodeColumnOrdinal(rightCols, rightEq[i]) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -1420,14 +1412,14 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols opt.C onExpr, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil } func (b *Builder) buildMergeJoin( join *memo.MergeJoinExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { if !b.disableTelemetry { telemetry.Inc(sqltelemetry.JoinAlgoMergeUseCounter) telemetry.Inc(opt.JoinTypeToUseCounter(join.JoinType)) @@ -1435,7 +1427,7 @@ func (b *Builder) buildMergeJoin( joinType, err := joinOpToJoinType(join.JoinType) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } leftExpr, rightExpr := join.Left, join.Right leftEq, rightEq := join.LeftEq, join.RightEq @@ -1463,19 +1455,19 @@ func (b *Builder) buildMergeJoin( leftExpr, rightExpr, join.On, joinType, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } leftOrd, err := sqlOrdering(leftEq, leftCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } rightOrd, err := sqlOrdering(rightEq, rightCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } reqOrd, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } leftEqColsAreKey := leftExpr.Relational().FuncDeps.ColsAreStrictKey(leftEq.ColSet()) rightEqColsAreKey := rightExpr.Relational().FuncDeps.ColsAreStrictKey(rightEq.ColSet()) @@ -1490,7 +1482,7 @@ func (b *Builder) buildMergeJoin( leftEqColsAreKey, rightEqColsAreKey, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil } @@ -1504,24 +1496,24 @@ func (b *Builder) initJoinBuild( ) ( leftPlan, rightPlan execPlan, onExpr tree.TypedExpr, - leftCols, rightCols, outputCols opt.ColMap, + leftCols, rightCols, outputCols colOrdMap, _ error, ) { leftPlan, leftCols, err := b.buildRelational(leftChild) if err != nil { - return execPlan{}, execPlan{}, nil, opt.ColMap{}, opt.ColMap{}, opt.ColMap{}, err + return execPlan{}, execPlan{}, nil, colOrdMap{}, colOrdMap{}, colOrdMap{}, err } rightPlan, rightCols, err = b.buildRelational(rightChild) if err != nil { - return execPlan{}, execPlan{}, nil, opt.ColMap{}, opt.ColMap{}, opt.ColMap{}, err + return execPlan{}, execPlan{}, nil, colOrdMap{}, colOrdMap{}, colOrdMap{}, err } - allCols := joinOutputMap(leftCols, rightCols) + allCols := b.joinOutputMap(leftCols, rightCols) if len(filters) != 0 { onExpr, err = b.buildScalarWithMap(allCols, &filters) if err != nil { - return execPlan{}, execPlan{}, nil, opt.ColMap{}, opt.ColMap{}, opt.ColMap{}, err + return execPlan{}, execPlan{}, nil, colOrdMap{}, colOrdMap{}, colOrdMap{}, err } } @@ -1536,12 +1528,12 @@ func (b *Builder) initJoinBuild( // joinOutputMap determines the outputCols map for a (non-semi/anti) join, given // the outputCols maps for its inputs. -func joinOutputMap(left, right opt.ColMap) opt.ColMap { - numLeftCols := numOutputColsInMap(left) +func (b *Builder) joinOutputMap(left, right colOrdMap) colOrdMap { + numLeftCols := left.MaxOrd() + 1 - res := left.Copy() - right.ForEach(func(colIdx, rightIdx int) { - res.Set(colIdx, rightIdx+numLeftCols) + res := b.colOrdsAlloc.Copy(left) + right.ForEach(func(col opt.ColumnID, rightIdx int) { + res.Set(col, rightIdx+numLeftCols) }) return res } @@ -1571,21 +1563,20 @@ func joinOpToJoinType(op opt.Operator) (descpb.JoinType, error) { } } -func (b *Builder) buildGroupBy( - groupBy memo.RelExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildGroupByInput(groupBy) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } groupingCols := groupBy.Private().(*memo.GroupingPrivate).GroupingCols groupingColIdx := make([]exec.NodeColumnOrdinal, 0, groupingCols.Len()) + outputCols = b.colOrdsAlloc.Alloc() for i, ok := groupingCols.Next(0); ok; i, ok = groupingCols.Next(i + 1) { - outputCols.Set(int(i), len(groupingColIdx)) + outputCols.Set(i, len(groupingColIdx)) ord, err := getNodeColumnOrdinal(inputCols, i) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } groupingColIdx = append(groupingColIdx, ord) } @@ -1600,11 +1591,11 @@ func (b *Builder) buildGroupBy( if aggFilter, ok := agg.(*memo.AggFilterExpr); ok { filter, ok := aggFilter.Filter.(*memo.VariableExpr) if !ok { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("only VariableOp args supported") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("only VariableOp args supported") } filterOrd, err = getNodeColumnOrdinal(inputCols, filter.Col) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } agg = aggFilter.Input } @@ -1625,16 +1616,16 @@ func (b *Builder) buildGroupBy( child := agg.Child(j) if variable, ok := child.(*memo.VariableExpr); ok { if len(constArgs) != 0 { - return execPlan{}, opt.ColMap{}, errors.Errorf("constant args must come after variable args") + return execPlan{}, colOrdMap{}, errors.Errorf("constant args must come after variable args") } ord, err := getNodeColumnOrdinal(inputCols, variable.Col) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } argCols = append(argCols, ord) } else { if len(argCols) == 0 { - return execPlan{}, opt.ColMap{}, errors.Errorf("a constant arg requires at least one variable arg") + return execPlan{}, colOrdMap{}, errors.Errorf("a constant arg requires at least one variable arg") } constArgs = append(constArgs, memo.ExtractConstDatum(child)) } @@ -1649,7 +1640,7 @@ func (b *Builder) buildGroupBy( Filter: filterOrd, DistsqlBlocklist: overload.DistsqlBlocklist, } - outputCols.Set(int(item.Col), len(groupingColIdx)+i) + outputCols.Set(item.Col, len(groupingColIdx)+i) } var ep execPlan @@ -1662,12 +1653,12 @@ func (b *Builder) buildGroupBy( &groupBy.GroupingPrivate, &groupBy.RequiredPhysical().Ordering, ), inputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var reqOrd exec.OutputOrdering reqOrd, err = reqOrdering(groupBy, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } orderType := exec.GroupingOrderType(groupBy.GroupingOrderType(&groupBy.RequiredPhysical().Ordering)) var rowCount uint64 @@ -1679,29 +1670,29 @@ func (b *Builder) buildGroupBy( ) } if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil } func (b *Builder) buildDistinct( distinct memo.RelExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { private := distinct.Private().(*memo.GroupingPrivate) if private.GroupingCols.Empty() { // A DistinctOn with no grouping columns should have been converted to a // LIMIT 1 or Max1Row by normalization rules. - return execPlan{}, opt.ColMap{}, fmt.Errorf("cannot execute distinct on no columns") + return execPlan{}, colOrdMap{}, fmt.Errorf("cannot execute distinct on no columns") } input, inputCols, err := b.buildGroupByInput(distinct) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } distinctCols, err := getNodeColumnOrdinalSet(inputCols, private.GroupingCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var orderedCols exec.NodeColumnOrdinalSet ordering := ordering.StreamingGroupingColOrdering( @@ -1710,27 +1701,27 @@ func (b *Builder) buildDistinct( for i := range ordering { ord, err := getNodeColumnOrdinal(inputCols, ordering[i].ID()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } orderedCols.Add(int(ord)) } reqOrdering, err := reqOrdering(distinct, inputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructDistinct( input.root, distinctCols, orderedCols, reqOrdering, private.NullsAreDistinct, private.ErrorOnDup) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // buildGroupByInput can add extra sort column(s), so discard those if they // are present by using an additional projection. outCols := distinct.Relational().OutputCols - if inputCols.Len() == outCols.Len() { + if inputCols.MaxOrd()+1 == outCols.Len() { return ep, inputCols, nil } return b.ensureColumns(ep, inputCols, distinct, outCols.ToList(), distinct.ProvidedPhysical().Ordering) @@ -1738,11 +1729,11 @@ func (b *Builder) buildDistinct( func (b *Builder) buildGroupByInput( groupBy memo.RelExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { groupByInput := groupBy.Child(0).(memo.RelExpr) input, inputCols, err := b.buildRelational(groupByInput) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // TODO(radu): this is a one-off fix for an otherwise bigger gap: we should @@ -1775,37 +1766,38 @@ func (b *Builder) buildGroupByInput( // The input is producing columns that are not useful; set up a projection. cols := make([]exec.NodeColumnOrdinal, 0, neededCols.Len()) + outputCols = b.colOrdsAlloc.Alloc() for colID, ok := neededCols.Next(0); ok; colID, ok = neededCols.Next(colID + 1) { - ordinal, ordOk := inputCols.Get(int(colID)) + ordinal, ordOk := inputCols.Get(colID) if !ordOk { - return execPlan{}, opt.ColMap{}, + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("needed column not produced by group-by input") } - outputCols.Set(int(colID), len(cols)) + outputCols.Set(colID, len(cols)) cols = append(cols, exec.NodeColumnOrdinal(ordinal)) } reqOrdering, err := reqOrdering(groupByInput, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } input.root, err = b.factory.ConstructSimpleProject(input.root, cols, reqOrdering) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return input, outputCols, nil } -func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { leftExpr := set.Child(0).(memo.RelExpr) left, leftCols, err := b.buildRelational(leftExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } rightExpr := set.Child(1).(memo.RelExpr) right, rightCols, err := b.buildRelational(rightExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } private := set.Private().(*memo.SetPrivate) @@ -1831,11 +1823,11 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMa // the order `b, c, a`). left, leftCols, err = b.ensureColumns(left, leftCols, leftExpr, private.LeftCols, leftExpr.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } right, rightCols, err = b.ensureColumns(right, rightCols, rightExpr, private.RightCols, rightExpr.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var typ tree.UnionType @@ -1854,7 +1846,7 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMa case opt.ExceptAllOp: typ, all = tree.ExceptOp, true default: - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("invalid operator %s", redact.Safe(set.Op())) + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("invalid operator %s", redact.Safe(set.Op())) } switch typ { @@ -1883,19 +1875,20 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMa enforceHomeRegion = b.IsANSIDML && b.evalCtx.SessionData().EnforceHomeRegion } + outputCols = b.colOrdsAlloc.Alloc() for i, col := range private.OutCols { - outputCols.Set(int(col), i) + outputCols.Set(col, i) } streamingOrdering, err := sqlOrdering( ordering.StreamingSetOpOrdering(set, &set.RequiredPhysical().Ordering), outputCols, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } reqOrdering, err := reqOrdering(set, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan @@ -1908,7 +1901,7 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMa ep.root, err = b.factory.ConstructStreamingSetOp(typ, all, left.root, right.root, streamingOrdering, reqOrdering) } else { if len(reqOrdering) > 0 { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("hash set op is not supported with a required ordering") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("hash set op is not supported with a required ordering") } if typ != tree.UnionOp { b.recordJoinAlgorithm(exec.HashJoin) @@ -1916,24 +1909,24 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols opt.ColMa ep.root, err = b.factory.ConstructHashSetOp(typ, all, left.root, right.root) } if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil } // buildTopK builds a plan for a TopKOp, which is like a combined SortOp and LimitOp. -func (b *Builder) buildTopK(e *memo.TopKExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildTopK(e *memo.TopKExpr) (_ execPlan, outputCols colOrdMap, err error) { inputExpr := e.Input input, inputCols, err := b.buildRelational(inputExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } ordering := e.Ordering.ToOrdering() inputOrdering := e.Input.ProvidedPhysical().Ordering alreadyOrderedPrefix := 0 for i := range inputOrdering { if i == len(ordering) { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("sort ordering already provided by input") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("sort ordering already provided by input") } if inputOrdering[i] != ordering[i] { break @@ -1942,7 +1935,7 @@ func (b *Builder) buildTopK(e *memo.TopKExpr) (_ execPlan, outputCols opt.ColMap } sqlOrdering, err := sqlOrdering(ordering, inputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructTopK( @@ -1951,22 +1944,22 @@ func (b *Builder) buildTopK(e *memo.TopKExpr) (_ execPlan, outputCols opt.ColMap exec.OutputOrdering(sqlOrdering), alreadyOrderedPrefix) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, inputCols, nil } // buildLimitOffset builds a plan for a LimitOp or OffsetOp -func (b *Builder) buildLimitOffset(e memo.RelExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildLimitOffset(e memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(e.Child(0).(memo.RelExpr)) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // LIMIT/OFFSET expression should never need buildScalarContext, because it // can't refer to the input expression. expr, err := b.buildScalar(nil, e.Child(1).(opt.ScalarExpr)) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan if e.Op() == opt.LimitOp { @@ -1975,15 +1968,15 @@ func (b *Builder) buildLimitOffset(e memo.RelExpr) (_ execPlan, outputCols opt.C ep.root, err = b.factory.ConstructLimit(input.root, nil, expr) } if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, inputCols, nil } -func (b *Builder) buildSort(sort *memo.SortExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildSort(sort *memo.SortExpr) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(sort.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } ordering := sort.ProvidedPhysical().Ordering @@ -1991,7 +1984,7 @@ func (b *Builder) buildSort(sort *memo.SortExpr) (_ execPlan, outputCols opt.Col alreadyOrderedPrefix := 0 for i := range inputOrdering { if i == len(ordering) { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("sort ordering already provided by input") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("sort ordering already provided by input") } if inputOrdering[i] != ordering[i] { break @@ -2001,7 +1994,7 @@ func (b *Builder) buildSort(sort *memo.SortExpr) (_ execPlan, outputCols opt.Col sqlOrdering, err := sqlOrdering(ordering, inputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructSort( @@ -2010,7 +2003,7 @@ func (b *Builder) buildSort(sort *memo.SortExpr) (_ execPlan, outputCols opt.Col alreadyOrderedPrefix, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, inputCols, nil } @@ -2112,10 +2105,10 @@ func (b *Builder) enforceScanWithHomeRegion(skipID cat.StableID) error { func (b *Builder) buildDistribute( distribute *memo.DistributeExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(distribute.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if distribute.NoOpDistribution() { @@ -2147,11 +2140,11 @@ func (b *Builder) buildDistribute( _, _, err = b.buildRelational(distribute.Input) b.doScanExprCollection = saveDoScanExprCollection if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } err = b.enforceScanWithHomeRegion(mutationStableID) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } homeRegion, ok := distribute.GetInputHomeRegion() @@ -2177,7 +2170,7 @@ func (b *Builder) buildDistribute( msgString := errorStringBuilder.String() err = pgerror.Newf(errCode, "%s", msgString) } - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // TODO(rytaft): This is currently a no-op. We should pass this distribution @@ -2187,10 +2180,10 @@ func (b *Builder) buildDistribute( func (b *Builder) buildOrdinality( ord *memo.OrdinalityExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(ord.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } colName := b.mem.Metadata().ColumnMeta(ord.ColID).Alias @@ -2198,24 +2191,24 @@ func (b *Builder) buildOrdinality( var ep execPlan ep.root, err = b.factory.ConstructOrdinality(input.root, colName) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // We have one additional ordinality column, which is ordered at the end of // the list. // TODO(mgartner): Is the copy here necessary? - outputCols = inputCols.Copy() - outputCols.Set(int(ord.ColID), outputCols.Len()) + outputCols = b.colOrdsAlloc.Copy(inputCols) + outputCols.Set(ord.ColID, outputCols.MaxOrd()+1) return ep, outputCols, nil } func (b *Builder) buildIndexJoin( join *memo.IndexJoinExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(join.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } md := b.mem.Metadata() @@ -2229,7 +2222,7 @@ func (b *Builder) buildIndexJoin( for i := range keyCols { keyCols[i], err = getNodeColumnOrdinal(inputCols, join.Table.ColumnID(pri.Column(i).Ordinal())) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -2238,20 +2231,20 @@ func (b *Builder) buildIndexJoin( locking, err := b.buildLocking(join.Locking) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } b.recordJoinAlgorithm(exec.IndexJoin) reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan res.root, err = b.factory.ConstructIndexJoin( input.root, tab, keyCols, needed, reqOrdering, locking, join.RequiredPhysical().LimitHintInt64(), ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return res, outputCols, nil @@ -2467,7 +2460,7 @@ func (b *Builder) handleRemoteLookupJoinError(join *memo.LookupJoinExpr) (err er func (b *Builder) buildLookupJoin( join *memo.LookupJoinExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() if !b.disableTelemetry { telemetry.Inc(sqltelemetry.JoinAlgoLookupUseCounter) @@ -2492,7 +2485,7 @@ func (b *Builder) buildLookupJoin( } input, inputCols, err := b.buildRelational(join.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if enforceHomeRegion { b.doScanExprCollection = saveDoScanExprCollection @@ -2501,14 +2494,14 @@ func (b *Builder) buildLookupJoin( // merged. err = b.handleRemoteLookupJoinError(join) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } keyCols := make([]exec.NodeColumnOrdinal, len(join.KeyCols)) for i, c := range join.KeyCols { keyCols[i], err = getNodeColumnOrdinal(inputCols, c) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -2520,18 +2513,18 @@ func (b *Builder) buildLookupJoin( lookupOrdinals, lookupColMap := b.getColumns(lookupCols, join.Table) // allExprCols are the columns used in expressions evaluated by this join. - allExprCols := joinOutputMap(inputCols, lookupColMap) + allExprCols := b.joinOutputMap(inputCols, lookupColMap) allCols := allExprCols if join.IsFirstJoinInPairedJoiner { // allCols needs to include the continuation column since it will be // in the result output by this join. - allCols = allExprCols.Copy() - maxValue, ok := allCols.MaxValue() - if !ok { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("allCols should not be empty") + allCols = b.colOrdsAlloc.Copy(allExprCols) + maxOrd := allCols.MaxOrd() + if maxOrd == -1 { + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("allCols should not be empty") } // Assign the continuation column the next unused value in the map. - allCols.Set(int(join.ContinuationCol), maxValue+1) + allCols.Set(join.ContinuationCol, maxOrd+1) } outputCols = allCols if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { @@ -2540,7 +2533,7 @@ func (b *Builder) buildLookupJoin( } ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, allExprCols.Len()), + ivh: tree.MakeIndexedVarHelper(nil /* container */, allExprCols.MaxOrd()+1), ivarMap: allExprCols, } var lookupExpr, remoteLookupExpr tree.TypedExpr @@ -2548,14 +2541,14 @@ func (b *Builder) buildLookupJoin( var err error lookupExpr, err = b.buildScalar(&ctx, &join.LookupExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } if len(join.RemoteLookupExpr) > 0 { var err error remoteLookupExpr, err = b.buildScalar(&ctx, &join.RemoteLookupExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } var onExpr tree.TypedExpr @@ -2563,7 +2556,7 @@ func (b *Builder) buildLookupJoin( var err error onExpr, err = b.buildScalar(&ctx, &join.On) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -2573,18 +2566,18 @@ func (b *Builder) buildLookupJoin( locking, err := b.buildLocking(join.Locking) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } joinType, err := joinOpToJoinType(join.JoinType) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } b.recordJoinType(joinType) b.recordJoinAlgorithm(exec.LookupJoin) reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan res.root, err = b.factory.ConstructLookupJoin( @@ -2606,7 +2599,7 @@ func (b *Builder) buildLookupJoin( join.RemoteOnlyLookups, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Apply a post-projection if Cols doesn't contain all input columns. @@ -2714,7 +2707,7 @@ func (b *Builder) handleRemoteInvertedJoinError(join *memo.InvertedJoinExpr) (er func (b *Builder) buildInvertedJoin( join *memo.InvertedJoinExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { enforceHomeRegion := b.evalCtx.SessionData().EnforceHomeRegion && b.IsANSIDML saveDoScanExprCollection := false if enforceHomeRegion { @@ -2730,7 +2723,7 @@ func (b *Builder) buildInvertedJoin( } input, inputCols, err := b.buildRelational(join.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if enforceHomeRegion { @@ -2740,7 +2733,7 @@ func (b *Builder) buildInvertedJoin( // merged. err = b.handleRemoteInvertedJoinError(join) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } md := b.mem.Metadata() @@ -2752,7 +2745,7 @@ func (b *Builder) buildInvertedJoin( for i, c := range join.PrefixKeyCols { prefixEqCols[i], err = getNodeColumnOrdinal(inputCols, c) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } @@ -2771,19 +2764,19 @@ func (b *Builder) buildInvertedJoin( lookupOrdinals, lookupColMap := b.getColumns(lookupCols, join.Table) // allExprCols are the columns used in expressions evaluated by this join. - allExprCols := joinOutputMap(inputCols, lookupColMap) + allExprCols := b.joinOutputMap(inputCols, lookupColMap) allCols := allExprCols if join.IsFirstJoinInPairedJoiner { // allCols needs to include the continuation column since it will be // in the result output by this join. - allCols = allExprCols.Copy() - maxValue, ok := allCols.MaxValue() - if !ok { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("allCols should not be empty") + allCols = b.colOrdsAlloc.Copy(allExprCols) + maxOrd := allCols.MaxOrd() + if maxOrd == -1 { + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("allCols should not be empty") } // Assign the continuation column the next unused value in the map. - allCols.Set(int(join.ContinuationCol), maxValue+1) + allCols.Set(join.ContinuationCol, maxOrd+1) } outputCols = allCols if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { @@ -2792,12 +2785,12 @@ func (b *Builder) buildInvertedJoin( } ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, allExprCols.Len()), - ivarMap: allExprCols.Copy(), + ivh: tree.MakeIndexedVarHelper(nil /* container */, allExprCols.MaxOrd()+1), + ivarMap: b.colOrdsAlloc.Copy(allExprCols), } onExpr, err := b.buildScalar(&ctx, &join.On) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // The inverted filter refers to the inverted column's source column, but it @@ -2808,27 +2801,27 @@ func (b *Builder) buildInvertedJoin( // geospatial type (which would make the expression invalid in terms of // typing). Perhaps we need to pass this information in a more specific way // and not as a generic expression? - ord, _ := ctx.ivarMap.Get(int(invertedColID)) - ctx.ivarMap.Set(int(join.Table.ColumnID(invertedColumn.InvertedSourceColumnOrdinal())), ord) + ord, _ := ctx.ivarMap.Get(invertedColID) + ctx.ivarMap.Set(join.Table.ColumnID(invertedColumn.InvertedSourceColumnOrdinal()), ord) invertedExpr, err := b.buildScalar(&ctx, join.InvertedExpr) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } locking, err := b.buildLocking(join.Locking) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } joinType, err := joinOpToJoinType(join.JoinType) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } b.recordJoinType(joinType) b.recordJoinAlgorithm(exec.InvertedJoin) reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan res.root, err = b.factory.ConstructInvertedJoin( @@ -2845,7 +2838,7 @@ func (b *Builder) buildInvertedJoin( locking, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Apply a post-projection to remove the inverted column. @@ -2854,7 +2847,7 @@ func (b *Builder) buildInvertedJoin( func (b *Builder) buildZigzagJoin( join *memo.ZigzagJoinExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() leftTable := md.Table(join.LeftTable) @@ -2899,23 +2892,23 @@ func (b *Builder) buildZigzagJoin( leftLocking, err := b.buildLocking(join.LeftLocking) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } rightLocking, err := b.buildLocking(join.RightLocking) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - allCols := joinOutputMap(leftColMap, rightColMap) + allCols := b.joinOutputMap(leftColMap, rightColMap) outputCols = allCols ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, leftColMap.Len()+rightColMap.Len()), + ivh: tree.MakeIndexedVarHelper(nil /* container */, leftColMap.MaxOrd()+rightColMap.MaxOrd()+2), ivarMap: allCols, } onExpr, err := b.buildScalar(&ctx, &join.On) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Build the fixed value scalars. @@ -2932,17 +2925,17 @@ func (b *Builder) buildZigzagJoin( leftFixedVals, err := tupleToExprs(join.FixedVals[0].(*memo.TupleExpr)) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } rightFixedVals, err := tupleToExprs(join.FixedVals[1].(*memo.TupleExpr)) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } b.recordJoinAlgorithm(exec.ZigZagJoin) reqOrdering, err := reqOrdering(join, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var res execPlan res.root, err = b.factory.ConstructZigzagJoin( @@ -2962,7 +2955,7 @@ func (b *Builder) buildZigzagJoin( reqOrdering, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Apply a post-projection to retain only the columns we need. @@ -3007,24 +3000,24 @@ func (b *Builder) buildLocking(locking opt.Locking) (opt.Locking, error) { func (b *Builder) buildMax1Row( max1Row *memo.Max1RowExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(max1Row.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructMax1Row(input.root, max1Row.ErrorText) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, inputCols, nil } -func (b *Builder) buildWith(with *memo.WithExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildWith(with *memo.WithExpr) (_ execPlan, outputCols colOrdMap, err error) { value, valuesCols, err := b.buildRelational(with.Binding) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var label bytes.Buffer @@ -3035,7 +3028,7 @@ func (b *Builder) buildWith(with *memo.WithExpr) (_ execPlan, outputCols opt.Col buffer, err := b.factory.ConstructBuffer(value.root, label.String()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // TODO(justin): if the binding here has a spoolNode at its root, we can @@ -3063,23 +3056,23 @@ func (b *Builder) buildWith(with *memo.WithExpr) (_ execPlan, outputCols opt.Col func (b *Builder) buildRecursiveCTE( rec *memo.RecursiveCTEExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { initial, initialCols, err := b.buildRelational(rec.Initial) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Make sure we have the columns in the correct order. initial, initialCols, err = b.ensureColumns(initial, initialCols, rec.Initial, rec.InitialCols, nil /* ordering */) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Renumber the columns so they match the columns expected by the recursive // query. - initialCols = util.FastIntMap{} + initialCols = b.colOrdsAlloc.Alloc() for i, col := range rec.OutCols { - initialCols.Set(int(col), i) + initialCols.Set(col, i) } // To implement exec.RecursiveCTEIterationFn, we create a special Builder. @@ -3102,6 +3095,9 @@ func (b *Builder) buildRecursiveCTE( innerBld := *innerBldTemplate innerBld.factory = ef innerBld.addBuiltWithExpr(rec.WithID, initialCols, bufferRef) + // TODO(mgartner): I think colOrdsAlloc can be reused for each recursive + // iteration. + innerBld.colOrdsAlloc.Init(innerBld.mem.Metadata().MaxColumn()) plan, planCols, err := innerBld.build(rec.Recursive) if err != nil { return nil, err @@ -3122,20 +3118,21 @@ func (b *Builder) buildRecursiveCTE( var ep execPlan ep.root, err = b.factory.ConstructRecursiveCTE(initial.root, fn, label, rec.Deduplicate) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } + outputCols = b.colOrdsAlloc.Alloc() for i, col := range rec.OutCols { - outputCols.Set(int(col), i) + outputCols.Set(col, i) } return ep, outputCols, nil } func (b *Builder) buildWithScan( withScan *memo.WithScanExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { e := b.findBuiltWithExpr(withScan.With) if e == nil { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf( + return execPlan{}, colOrdMap{}, errors.AssertionFailedf( "couldn't find With expression with ID %d", withScan.With, ) } @@ -3148,19 +3145,20 @@ func (b *Builder) buildWithScan( node, err := b.factory.ConstructScanBuffer(e.bufferNode, label.String()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } res := execPlan{root: node} // Apply any necessary projection to produce the InCols in the given order. res, _, err = b.ensureColumns(res, e.outputCols, withScan, withScan.InCols, withScan.ProvidedPhysical().Ordering) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Renumber the columns. + outputCols = b.colOrdsAlloc.Alloc() for i, col := range withScan.OutCols { - outputCols.Set(int(col), i) + outputCols.Set(col, i) } return res, outputCols, nil @@ -3168,10 +3166,10 @@ func (b *Builder) buildWithScan( func (b *Builder) buildProjectSet( projectSet *memo.ProjectSetExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(projectSet.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } zip := projectSet.Zip @@ -3182,19 +3180,19 @@ func (b *Builder) buildProjectSet( zipCols := make(colinfo.ResultColumns, 0, len(zip)) numColsPerGen := make([]int, len(zip)) - n := numOutputColsInMap(inputCols) + n := inputCols.MaxOrd() + 1 for i := range zip { item := &zip[i] exprs[i], err = b.buildScalar(&scalarCtx, item.Fn) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } for _, col := range item.Cols { colMeta := md.ColumnMeta(col) zipCols = append(zipCols, colinfo.ResultColumn{Name: colMeta.Alias, Typ: colMeta.Type}) - outputCols.Set(int(col), n) + inputCols.Set(col, n) n++ } @@ -3204,16 +3202,16 @@ func (b *Builder) buildProjectSet( var ep execPlan ep.root, err = b.factory.ConstructProjectSet(input.root, exprs, zipCols, numColsPerGen) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputCols, nil + return ep, inputCols, nil } -func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols colOrdMap, err error) { udf := c.Proc.(*memo.UDFCallExpr) if udf.Def == nil { - return execPlan{}, opt.ColMap{}, errors.AssertionFailedf("expected non-nil UDF definition") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("expected non-nil UDF definition") } // Build the argument expressions. @@ -3224,7 +3222,7 @@ func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols opt.ColMap for i := range udf.Args { args[i], err = b.buildScalar(&ctx, udf.Args[i]) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } } @@ -3264,9 +3262,9 @@ func (b *Builder) buildCall(c *memo.CallExpr) (_ execPlan, outputCols opt.ColMap var ep execPlan ep.root, err = b.factory.ConstructCall(r) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, opt.ColMap{}, nil + return ep, colOrdMap{}, nil } func (b *Builder) resultColumn(id opt.ColumnID) colinfo.ResultColumn { @@ -3327,7 +3325,7 @@ func (b *Builder) isOffsetMode(boundType treewindow.WindowFrameBoundType) bool { return boundType == treewindow.OffsetPreceding || boundType == treewindow.OffsetFollowing } -func (b *Builder) buildFrame(inputCols opt.ColMap, w *memo.WindowsItem) (*tree.WindowFrame, error) { +func (b *Builder) buildFrame(inputCols colOrdMap, w *memo.WindowsItem) (*tree.WindowFrame, error) { scalarCtx := makeBuildScalarCtx(inputCols) newDef := &tree.WindowFrame{ Mode: w.Frame.Mode, @@ -3371,10 +3369,10 @@ func (b *Builder) buildFrame(inputCols opt.ColMap, w *memo.WindowsItem) (*tree.W return newDef, nil } -func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.ColMap, err error) { +func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(w.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Rearrange the input so that the input has all the passthrough columns @@ -3392,7 +3390,7 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co // way to alleviate this. input, inputCols, err = b.ensureColumns(input, inputCols, w, desiredCols, opt.Ordering{}) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } ctx := makeBuildScalarCtx(inputCols) @@ -3407,7 +3405,7 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co } indexedVar, err := b.indexedVar(&ctx, b.mem.Metadata(), c.ID()) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } orderingExprs[i] = &tree.Order{ Expr: indexedVar, @@ -3420,11 +3418,11 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co i := 0 for col, ok := w.Partition.Next(0); ok; col, ok = w.Partition.Next(col + 1) { - ordinal, _ := inputCols.Get(int(col)) + ordinal, _ := inputCols.Get(col) partitionIdxs[i] = exec.NodeColumnOrdinal(ordinal) indexedVar, err := b.indexedVar(&ctx, b.mem.Metadata(), col) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } partitionExprs[i] = indexedVar i++ @@ -3450,16 +3448,16 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co col := fn.Child(j).(*memo.VariableExpr).Col indexedVar, err := b.indexedVar(&ctx, b.mem.Metadata(), col) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } args[j] = indexedVar - idx, _ := inputCols.Get(int(col)) + idx, _ := inputCols.Get(col) argIdxs[i][j] = exec.NodeColumnOrdinal(idx) } frame, err := b.buildFrame(inputCols, item) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var builtFilter tree.TypedExpr @@ -3467,14 +3465,14 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co if ok { f, ok := filter.(*memo.VariableExpr) if !ok { - return execPlan{}, opt.ColMap{}, + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("expected FILTER expression to be a VariableExpr") } - filterIdxs[i], _ = inputCols.Get(int(f.Col)) + filterIdxs[i], _ = inputCols.Get(f.Col) builtFilter, err = b.buildScalar(&ctx, filter) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } else { filterIdxs[i] = -1 @@ -3487,7 +3485,7 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co } wrappedFn, err := b.wrapFunction(name) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } exprs[i] = tree.NewTypedFuncExpr( wrappedFn, @@ -3505,13 +3503,14 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co // All the passthrough cols will keep their ordinal index. passthrough.ForEach(func(col opt.ColumnID) { - ordinal, _ := inputCols.Get(int(col)) + ordinal, _ := inputCols.Get(col) resultCols[ordinal] = b.resultColumn(col) }) - inputCols.ForEach(func(key, val int) { - if passthrough.Contains(opt.ColumnID(key)) { - outputCols.Set(key, val) + outputCols = b.colOrdsAlloc.Alloc() + inputCols.ForEach(func(col opt.ColumnID, ord int) { + if passthrough.Contains(col) { + outputCols.Set(col, ord) } }) @@ -3523,13 +3522,13 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co windowStart := passthrough.Len() for i := range w.Windows { resultCols[windowStart+i] = b.resultColumn(w.Windows[i].Col) - outputCols.Set(int(w.Windows[i].Col), windowStart+i) + outputCols.Set(w.Windows[i].Col, windowStart+i) outputIdxs[i] = windowStart + i } sqlOrdering, err := sqlOrdering(ord, inputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructWindow(input.root, exec.WindowInfo{ @@ -3542,7 +3541,7 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co Ordering: sqlOrdering, }) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } return ep, outputCols, nil @@ -3550,23 +3549,24 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols opt.Co func (b *Builder) buildSequenceSelect( seqSel *memo.SequenceSelectExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { seq := b.mem.Metadata().Sequence(seqSel.Sequence) var ep execPlan ep.root, err = b.factory.ConstructSequenceSelect(seq) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } + outputCols = b.colOrdsAlloc.Alloc() for i, c := range seqSel.Cols { - outputCols.Set(int(c), i) + outputCols.Set(c, i) } return ep, outputCols, nil } func (b *Builder) applySaveTable( - input execPlan, inputCols opt.ColMap, e memo.RelExpr, saveTableName string, + input execPlan, inputCols colOrdMap, e memo.RelExpr, saveTableName string, ) (execPlan, error) { name := tree.NewTableNameWithSchema(tree.Name(opt.SaveTablesDatabase), catconstants.PublicSchemaName, tree.Name(saveTableName)) @@ -3576,7 +3576,7 @@ func (b *Builder) applySaveTable( colNames := make([]string, outputCols.Len()) colNameGen := memo.NewColumnNameGenerator(e) for col, ok := outputCols.Next(0); ok; col, ok = outputCols.Next(col + 1) { - ord, _ := inputCols.Get(int(col)) + ord, _ := inputCols.Get(col) colNames[ord] = colNameGen.GenerateName(col) } @@ -3590,15 +3590,16 @@ func (b *Builder) applySaveTable( func (b *Builder) buildOpaque( opaque *memo.OpaqueRelPrivate, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { var ep execPlan ep.root, err = b.factory.ConstructOpaque(opaque.Metadata) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } + outputCols = b.colOrdsAlloc.Alloc() for i, c := range opaque.Columns { - outputCols.Set(int(c), i) + outputCols.Set(c, i) } return ep, outputCols, nil @@ -3606,7 +3607,7 @@ func (b *Builder) buildOpaque( func (b *Builder) buildBarrier( barrier *memo.BarrierExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { // BarrierExpr is only used as an optimization barrier. In the execution plan, // it is replaced with its input. return b.buildRelational(barrier.Input) @@ -3616,12 +3617,12 @@ func (b *Builder) buildBarrier( // to produce the given list of columns. If the input plan already produces // the columns (in the same order), returns needProj=false. func (b *Builder) needProjection( - inputCols opt.ColMap, colList opt.ColList, + inputCols colOrdMap, colList opt.ColList, ) (_ []exec.NodeColumnOrdinal, needProj bool, err error) { - if numOutputColsInMap(inputCols) == len(colList) { + if inputCols.MaxOrd()+1 == len(colList) { identity := true for i, col := range colList { - if ord, ok := inputCols.Get(int(col)); !ok || ord != i { + if ord, ok := inputCols.Get(col); !ok || ord != i { identity = false break } @@ -3647,14 +3648,14 @@ func (b *Builder) needProjection( // given list of columns; colNames is optional. func (b *Builder) ensureColumns( input execPlan, - inputCols opt.ColMap, + inputCols colOrdMap, inputExpr memo.RelExpr, colList opt.ColList, provided opt.Ordering, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { cols, needProj, err := b.needProjection(inputCols, colList) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if !needProj { return input, inputCols, nil @@ -3663,12 +3664,13 @@ func (b *Builder) ensureColumns( // we need to explicitly annotate the latter with estimates since the code // in buildRelational() will attach them to the project. b.maybeAnnotateWithEstimates(input.root, inputExpr) + outputCols = b.colOrdsAlloc.Alloc() for i, col := range colList { - outputCols.Set(int(col), i) + outputCols.Set(col, i) } sqlOrdering, err := sqlOrdering(provided, outputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } reqOrdering := exec.OutputOrdering(sqlOrdering) var res execPlan @@ -3679,18 +3681,19 @@ func (b *Builder) ensureColumns( // applyPresentation adds a projection to a plan to satisfy a required // Presentation property. func (b *Builder) applyPresentation( - input execPlan, inputCols opt.ColMap, pres physical.Presentation, -) (_ execPlan, outputCols opt.ColMap, err error) { + input execPlan, inputCols colOrdMap, pres physical.Presentation, +) (_ execPlan, outputCols colOrdMap, err error) { cols := make([]exec.NodeColumnOrdinal, len(pres)) colNames := make([]string, len(pres)) var res execPlan + outputCols = b.colOrdsAlloc.Alloc() for i := range pres { ord, err := getNodeColumnOrdinal(inputCols, pres[i].ID) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } cols[i] = ord - outputCols.Set(int(pres[i].ID), i) + outputCols.Set(pres[i].ID, i) colNames[i] = pres[i].Alias } res.root, err = b.factory.ConstructSerializingProject(input.root, cols, colNames) diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index d5dfbe812884..699ab48dc925 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -42,7 +42,7 @@ type buildScalarCtx struct { // ivarMap is a map from opt.ColumnID to the index of an IndexedVar. // If a ColumnID is not in the map, it cannot appear in the expression. - ivarMap opt.ColMap + ivarMap colOrdMap } type buildFunc func(b *Builder, ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.TypedExpr, error) @@ -115,10 +115,10 @@ func (b *Builder) buildScalar(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree. } func (b *Builder) buildScalarWithMap( - colMap opt.ColMap, scalar opt.ScalarExpr, + colMap colOrdMap, scalar opt.ScalarExpr, ) (tree.TypedExpr, error) { ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, numOutputColsInMap(colMap)), + ivh: tree.MakeIndexedVarHelper(nil /* container */, colMap.MaxOrd()+1), ivarMap: colMap, } return b.buildScalar(&ctx, scalar) @@ -147,7 +147,7 @@ func (b *Builder) buildVariable( func (b *Builder) indexedVar( ctx *buildScalarCtx, md *opt.Metadata, colID opt.ColumnID, ) (tree.TypedExpr, error) { - idx, ok := ctx.ivarMap.Get(int(colID)) + idx, ok := ctx.ivarMap.Get(colID) if !ok { return nil, errors.AssertionFailedf("cannot map variable %d to an indexed var", redact.Safe(colID)) } @@ -583,9 +583,9 @@ func (b *Builder) buildAny(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ } // Construct tuple type of columns in the row. - contents := make([]*types.T, numOutputColsInMap(planCols)) - planCols.ForEach(func(key, val int) { - contents[val] = b.mem.Metadata().ColumnMeta(opt.ColumnID(key)).Type + contents := make([]*types.T, planCols.MaxOrd()+1) + planCols.ForEach(func(col opt.ColumnID, ord int) { + contents[ord] = b.mem.Metadata().ColumnMeta(col).Type }) typs := types.MakeTuple(contents) subqueryExpr := b.addSubquery( diff --git a/pkg/sql/opt/exec/execbuilder/statement.go b/pkg/sql/opt/exec/execbuilder/statement.go index 25ce4314e0d3..48171d6aea04 100644 --- a/pkg/sql/opt/exec/execbuilder/statement.go +++ b/pkg/sql/opt/exec/execbuilder/statement.go @@ -29,33 +29,33 @@ import ( func (b *Builder) buildCreateTable( ct *memo.CreateTableExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { schema := b.mem.Metadata().Schema(ct.Schema) if !ct.Syntax.As() { root, err := b.factory.ConstructCreateTable(schema, ct.Syntax) - return execPlan{root: root}, opt.ColMap{}, err + return execPlan{root: root}, colOrdMap{}, err } // Construct AS input to CREATE TABLE. input, inputCols, err := b.buildRelational(ct.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // Impose ordering and naming on input columns, so that they match the // order and names of the table columns into which values will be // inserted. input, _, err = b.applyPresentation(input, inputCols, ct.InputCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } root, err := b.factory.ConstructCreateTableAs(input.root, schema, ct.Syntax) - return execPlan{root: root}, opt.ColMap{}, err + return execPlan{root: root}, colOrdMap{}, err } func (b *Builder) buildCreateView( cv *memo.CreateViewExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() schema := md.Schema(cv.Schema) cols := make(colinfo.ResultColumns, len(cv.Columns)) @@ -71,12 +71,12 @@ func (b *Builder) buildCreateView( cv.Deps, cv.TypeDeps, ) - return execPlan{root: root}, opt.ColMap{}, err + return execPlan{root: root}, colOrdMap{}, err } func (b *Builder) buildCreateFunction( cf *memo.CreateFunctionExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() schema := md.Schema(cf.Schema) root, err := b.factory.ConstructCreateFunction( @@ -85,12 +85,12 @@ func (b *Builder) buildCreateFunction( cf.Deps, cf.TypeDeps, ) - return execPlan{root: root}, opt.ColMap{}, err + return execPlan{root: root}, colOrdMap{}, err } func (b *Builder) buildExplainOpt( explain *memo.ExplainExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { fmtFlags := memo.ExprFmtHideAll switch { case explain.Options.Flags[tree.ExplainFlagVerbose]: @@ -145,21 +145,21 @@ func (b *Builder) buildExplainOpt( var err error envOpts, err = b.getEnvData() if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } var ep execPlan ep.root, err = b.factory.ConstructExplainOpt(planText.String(), envOpts) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(explain.ColList), nil + return ep, b.outputColsFromList(explain.ColList), nil } func (b *Builder) buildExplain( explainExpr *memo.ExplainExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { if explainExpr.Options.Mode == tree.ExplainOpt { return b.buildExplainOpt(explainExpr) } @@ -191,34 +191,34 @@ func (b *Builder) buildExplain( }, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(explainExpr.ColList), nil + return ep, b.outputColsFromList(explainExpr.ColList), nil } func (b *Builder) buildShowTrace( show *memo.ShowTraceForSessionExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { var ep execPlan ep.root, err = b.factory.ConstructShowTrace(show.TraceType, show.Compact) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(show.ColList), nil + return ep, b.outputColsFromList(show.ColList), nil } func (b *Builder) buildAlterTableSplit( split *memo.AlterTableSplitExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(split.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } scalarCtx := buildScalarCtx{} expiration, err := b.buildScalar(&scalarCtx, split.Expiration) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } table := b.mem.Metadata().Table(split.Table) var ep execPlan @@ -228,17 +228,17 @@ func (b *Builder) buildAlterTableSplit( expiration, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(split.Columns), nil + return ep, b.outputColsFromList(split.Columns), nil } func (b *Builder) buildAlterTableUnsplit( unsplit *memo.AlterTableUnsplitExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(unsplit.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } table := b.mem.Metadata().Table(unsplit.Table) var ep execPlan @@ -247,29 +247,29 @@ func (b *Builder) buildAlterTableUnsplit( input.root, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(unsplit.Columns), nil + return ep, b.outputColsFromList(unsplit.Columns), nil } func (b *Builder) buildAlterTableUnsplitAll( unsplitAll *memo.AlterTableUnsplitAllExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { table := b.mem.Metadata().Table(unsplitAll.Table) var ep execPlan ep.root, err = b.factory.ConstructAlterTableUnsplitAll(table.Index(unsplitAll.Index)) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(unsplitAll.Columns), nil + return ep, b.outputColsFromList(unsplitAll.Columns), nil } func (b *Builder) buildAlterTableRelocate( relocate *memo.AlterTableRelocateExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(relocate.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } table := b.mem.Metadata().Table(relocate.Table) var ep execPlan @@ -279,26 +279,26 @@ func (b *Builder) buildAlterTableRelocate( relocate.SubjectReplicas, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(relocate.Columns), nil + return ep, b.outputColsFromList(relocate.Columns), nil } func (b *Builder) buildAlterRangeRelocate( relocate *memo.AlterRangeRelocateExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(relocate.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } scalarCtx := buildScalarCtx{} toStoreID, err := b.buildScalar(&scalarCtx, relocate.ToStoreID) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } fromStoreID, err := b.buildScalar(&scalarCtx, relocate.FromStoreID) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructAlterRangeRelocate( @@ -308,23 +308,23 @@ func (b *Builder) buildAlterRangeRelocate( fromStoreID, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(relocate.Columns), nil + return ep, b.outputColsFromList(relocate.Columns), nil } func (b *Builder) buildControlJobs( ctl *memo.ControlJobsExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(ctl.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } scalarCtx := buildScalarCtx{} reason, err := b.buildScalar(&scalarCtx, ctl.Reason) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan @@ -334,18 +334,18 @@ func (b *Builder) buildControlJobs( reason, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // ControlJobs returns no columns. - return ep, opt.ColMap{}, nil + return ep, colOrdMap{}, nil } func (b *Builder) buildControlSchedules( ctl *memo.ControlSchedulesExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(ctl.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructControlSchedules( @@ -353,85 +353,85 @@ func (b *Builder) buildControlSchedules( input.root, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // ControlSchedules returns no columns. - return ep, opt.ColMap{}, nil + return ep, colOrdMap{}, nil } func (b *Builder) buildShowCompletions( ctl *memo.ShowCompletionsExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { var ep execPlan ep.root, err = b.factory.ConstructShowCompletions( ctl.Command, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(ctl.Columns), nil + return ep, b.outputColsFromList(ctl.Columns), nil } func (b *Builder) buildCancelQueries( cancel *memo.CancelQueriesExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(cancel.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan ep.root, err = b.factory.ConstructCancelQueries(input.root, cancel.IfExists) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if !b.disableTelemetry { telemetry.Inc(sqltelemetry.CancelQueriesUseCounter) } // CancelQueries returns no columns. - return ep, opt.ColMap{}, nil + return ep, colOrdMap{}, nil } func (b *Builder) buildCancelSessions( cancel *memo.CancelSessionsExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, _, err := b.buildRelational(cancel.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } node, err := b.factory.ConstructCancelSessions(input.root, cancel.IfExists) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } if !b.disableTelemetry { telemetry.Inc(sqltelemetry.CancelSessionsUseCounter) } // CancelSessions returns no columns. - return execPlan{root: node}, opt.ColMap{}, nil + return execPlan{root: node}, colOrdMap{}, nil } func (b *Builder) buildCreateStatistics( c *memo.CreateStatisticsExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { node, err := b.factory.ConstructCreateStatistics(c.Syntax) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } // CreateStatistics returns no columns. - return execPlan{root: node}, opt.ColMap{}, nil + return execPlan{root: node}, colOrdMap{}, nil } func (b *Builder) buildExport( export *memo.ExportExpr, -) (_ execPlan, outputCols opt.ColMap, err error) { +) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildRelational(export.Input) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } scalarCtx := buildScalarCtx{} fileName, err := b.buildScalar(&scalarCtx, export.FileName) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } opts := make([]exec.KVOption, len(export.Options)) @@ -440,12 +440,12 @@ func (b *Builder) buildExport( var err error opts[i].Value, err = b.buildScalar(&scalarCtx, o.Value) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } } notNullColsSet, err := getNodeColumnOrdinalSet(inputCols, export.Input.Relational().NotNullCols) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } var ep execPlan @@ -457,17 +457,17 @@ func (b *Builder) buildExport( notNullColsSet, ) if err != nil { - return execPlan{}, opt.ColMap{}, err + return execPlan{}, colOrdMap{}, err } - return ep, outputColsFromList(export.Columns), nil + return ep, b.outputColsFromList(export.Columns), nil } // planWithColumns creates an execPlan for a node which has a fixed output // schema. -func outputColsFromList(cols opt.ColList) opt.ColMap { - var outputCols opt.ColMap +func (b *Builder) outputColsFromList(cols opt.ColList) colOrdMap { + outputCols := b.colOrdsAlloc.Alloc() for i, c := range cols { - outputCols.Set(int(c), i) + outputCols.Set(c, i) } return outputCols } diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 67e459212991..899a5fb114e6 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -779,6 +779,11 @@ func (md *Metadata) NumColumns() int { return len(md.cols) } +// MaxColumn returns the maximum column ID tracked by this Metadata instance. +func (md *Metadata) MaxColumn() ColumnID { + return ColumnID(len(md.cols)) +} + // ColumnMeta looks up the metadata for the column associated with the given // column id. The same column can be added multiple times to the query metadata // and associated with multiple column ids. From be0922896a7abfaeac430898d4ab90d309ca9e8f Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 9 Feb 2024 17:41:22 -0500 Subject: [PATCH 5/6] opt/execbuilder: reuse allocated colOrdMaps This commit extends `colOrdOrdMapAllocator` with a `Free` method. Freed maps will be reused in future calls to `Alloc` instead of allocating a new map. The build functions of the major relational expressions have been updated to free maps when they are no longer needed. This reduces the number of maps allocated, especially for complex queries with many execution nodes. Informs #117546 Release note: None --- pkg/sql/opt/exec/execbuilder/col_ord_map.go | 33 +- .../opt/exec/execbuilder/col_ord_map_test.go | 2 +- pkg/sql/opt/exec/execbuilder/mutation.go | 5 +- pkg/sql/opt/exec/execbuilder/relational.go | 283 ++++++++++++++---- 4 files changed, 251 insertions(+), 72 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map.go b/pkg/sql/opt/exec/execbuilder/col_ord_map.go index d636ef897591..3b7386497e72 100644 --- a/pkg/sql/opt/exec/execbuilder/col_ord_map.go +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map.go @@ -17,10 +17,16 @@ import ( "github.com/cockroachdb/errors" ) -// colOrdMapAllocator is used to allocate colOrdMaps. +// colOrdMapAllocator is used to allocate colOrdMaps. It must be initialized +// with Init before use. // +// Allocated maps can be returned to the allocator via the Free method. Freed +// maps will be returned in future calls to Alloc. +// +// WARNING: Do not mix-and-match maps allocated by separate allocators. type colOrdMapAllocator struct { maxCol opt.ColumnID + freed []colOrdMap } // Init initialized the allocator that can allocate maps that support column IDs @@ -29,10 +35,16 @@ func (a *colOrdMapAllocator) Init(maxCol opt.ColumnID) { a.maxCol = maxCol } -// Alloc returns an empty colOrdMap. It will return a previously Free'd +// Alloc returns an empty colOrdMap. It will return a previously freed // colOrdMap, if one is available. func (a *colOrdMapAllocator) Alloc() colOrdMap { - return newColOrdMap(a.maxCol) + if len(a.freed) == 0 { + // There are no freed maps, so allocate a new one. + return newColOrdMap(a.maxCol) + } + m := a.freed[len(a.freed)-1] + a.freed = a.freed[:len(a.freed)-1] + return m } // Copy returns a copy of the given colOrdMap. @@ -42,6 +54,17 @@ func (a *colOrdMapAllocator) Copy(from colOrdMap) colOrdMap { return m } +// Free returns the given map to the allocator for future reuse. +// +// WARNING: Do not use a map once it has been freed. +// WARNING: Do not free a map more than once. +// WARNING: Do not free a map that was allocated by a different allocator. +func (a *colOrdMapAllocator) Free(m colOrdMap) { + // Check that the map has not already been freed. + m.Clear() + a.freed = append(a.freed, m) +} + // colOrdMap is a map from column IDs to ordinals. // // The map is implemented as a slice of integers, with the slice's indexes @@ -142,8 +165,8 @@ func (m *colOrdMap) CopyFrom(other colOrdMap) { copy(m.ords, other.ords) } -// clear clears the map. The allocated memory is retained for future reuse. -func (m *colOrdMap) clear() { +// Clear clears the map. The allocated memory is retained for future reuse. +func (m colOrdMap) Clear() { for i := range m.ords { m.ords[i] = 0 } diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go b/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go index 63ba8891de08..93e3b2723fca 100644 --- a/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go @@ -46,7 +46,7 @@ func TestColOrdMap(t *testing.T) { switch { case n < 5: oracle = make(map[opt.ColumnID]int) - m.clear() + m.Clear() validate(t, m, oracle) case n < 15: cpy := newColOrdMap(maxCol) diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 27e105131947..b8deb098b301 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -60,7 +60,10 @@ func (b *Builder) buildMutationInput( } } - input, inputCols, err = b.ensureColumns(input, inputCols, inputExpr, colList, inputExpr.ProvidedPhysical().Ordering) + input, inputCols, err = b.ensureColumns( + input, inputCols, inputExpr, colList, + inputExpr.ProvidedPhysical().Ordering, true, /* reuseInputCols */ + ) if err != nil { return execPlan{}, colOrdMap{}, err } diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 7f5fde60c93c..96cca1d756b9 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -1004,7 +1004,8 @@ func (b *Builder) buildInvertedFilter( return b.applySimpleProject(res, inputCols, invFilter, invFilter.Relational().OutputCols, invFilter.ProvidedPhysical().Ordering) } -// applySimpleProject adds a simple projection on top of an existing plan. +// applySimpleProject adds a simple projection on top of an existing plan. The +// returned outputCols are always a new map distinct from inputCols. func (b *Builder) applySimpleProject( input execPlan, inputCols colOrdMap, @@ -1046,6 +1047,10 @@ func (b *Builder) buildProject( ) (_ execPlan, outputCols colOrdMap, err error) { md := b.mem.Metadata() input, inputCols, err := b.buildRelational(prj.Input) + // The input column map is only used for the lifetime of this function, so + // free the map afterward. In the case of a simple project, + // applySimpleProject will create a new map. + defer b.colOrdsAlloc.Free(inputCols) if err != nil { return execPlan{}, colOrdMap{}, err } @@ -1239,8 +1244,10 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols colO } // The right plan will always produce the columns in the presentation, in - // the same order. + // the same order. This map is only used for the lifetime of this function, + // so free the map afterward. rightOutputCols := b.colOrdsAlloc.Alloc() + defer b.colOrdsAlloc.Free(rightOutputCols) for i := range rightRequiredProps.Presentation { rightOutputCols.Set(rightRequiredProps.Presentation[i].ID, i) } @@ -1260,8 +1267,12 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (_ execPlan, outputCols colO if !joinType.ShouldIncludeRightColsInOutput() { outputCols = leftCols + // allCols is no longer used, so it can be freed. + b.colOrdsAlloc.Free(allCols) } else { outputCols = allCols + // leftCols is no longer used, so it can be freed. + b.colOrdsAlloc.Free(leftCols) } var ep execPlan @@ -1369,16 +1380,36 @@ func (b *Builder) buildHashJoin(join memo.RelExpr) (_ execPlan, outputCols colOr telemetry.Inc(opt.JoinTypeToUseCounter(join.Op())) } - left, right, onExpr, leftCols, rightCols, outputCols, err := b.initJoinBuild( + left, right, onExpr, leftCols, rightCols, allCols, err := b.initJoinBuild( leftExpr, rightExpr, memo.ExtractRemainingJoinFilters(*filters, leftEq, rightEq), - joinType, ) if err != nil { return execPlan{}, colOrdMap{}, err } + switch { + case !joinType.ShouldIncludeLeftColsInOutput(): + outputCols = rightCols + // leftCols and allCols are only used for the lifetime of the function, + // so they can be freed afterward. + defer b.colOrdsAlloc.Free(leftCols) + defer b.colOrdsAlloc.Free(allCols) + case !joinType.ShouldIncludeRightColsInOutput(): + outputCols = leftCols + // rightCols and allCols are only used for the lifetime of the function, + // so they can be freed afterward. + defer b.colOrdsAlloc.Free(rightCols) + defer b.colOrdsAlloc.Free(allCols) + default: + outputCols = allCols + // leftCols and rightCols are only used for the lifetime of the + // function, so they can be freed afterward. + defer b.colOrdsAlloc.Free(leftCols) + defer b.colOrdsAlloc.Free(rightCols) + } + // Convert leftEq/rightEq to ordinals. eqColsBuf := make([]exec.NodeColumnOrdinal, 2*len(leftEq)) leftEqOrdinals := eqColsBuf[:len(leftEq):len(leftEq)] @@ -1451,12 +1482,34 @@ func (b *Builder) buildMergeJoin( } } - left, right, onExpr, leftCols, rightCols, outputCols, err := b.initJoinBuild( - leftExpr, rightExpr, join.On, joinType, + left, right, onExpr, leftCols, rightCols, allCols, err := b.initJoinBuild( + leftExpr, rightExpr, join.On, ) if err != nil { return execPlan{}, colOrdMap{}, err } + + switch { + case !joinType.ShouldIncludeLeftColsInOutput(): + outputCols = rightCols + // leftCols and allCols are only used for the lifetime of the function, + // so they can be freed afterward. + defer b.colOrdsAlloc.Free(leftCols) + defer b.colOrdsAlloc.Free(allCols) + case !joinType.ShouldIncludeRightColsInOutput(): + outputCols = leftCols + // rightCols and allCols are only used for the lifetime of the function, + // so they can be freed afterward. + defer b.colOrdsAlloc.Free(rightCols) + defer b.colOrdsAlloc.Free(allCols) + default: + outputCols = allCols + // leftCols and rightCols are only used for the lifetime of the + // function, so they can be freed afterward. + defer b.colOrdsAlloc.Free(leftCols) + defer b.colOrdsAlloc.Free(rightCols) + } + leftOrd, err := sqlOrdering(leftEq, leftCols) if err != nil { return execPlan{}, colOrdMap{}, err @@ -1489,14 +1542,11 @@ func (b *Builder) buildMergeJoin( // initJoinBuild builds the inputs to the join as well as the ON expression. func (b *Builder) initJoinBuild( - leftChild memo.RelExpr, - rightChild memo.RelExpr, - filters memo.FiltersExpr, - joinType descpb.JoinType, + leftChild memo.RelExpr, rightChild memo.RelExpr, filters memo.FiltersExpr, ) ( leftPlan, rightPlan execPlan, onExpr tree.TypedExpr, - leftCols, rightCols, outputCols colOrdMap, + leftCols, rightCols, allCols colOrdMap, _ error, ) { leftPlan, leftCols, err := b.buildRelational(leftChild) @@ -1508,7 +1558,7 @@ func (b *Builder) initJoinBuild( return execPlan{}, execPlan{}, nil, colOrdMap{}, colOrdMap{}, colOrdMap{}, err } - allCols := b.joinOutputMap(leftCols, rightCols) + allCols = b.joinOutputMap(leftCols, rightCols) if len(filters) != 0 { onExpr, err = b.buildScalarWithMap(allCols, &filters) @@ -1517,12 +1567,6 @@ func (b *Builder) initJoinBuild( } } - if !joinType.ShouldIncludeLeftColsInOutput() { - return leftPlan, rightPlan, onExpr, leftCols, rightCols, rightCols, nil - } - if !joinType.ShouldIncludeRightColsInOutput() { - return leftPlan, rightPlan, onExpr, leftCols, rightCols, leftCols, nil - } return leftPlan, rightPlan, onExpr, leftCols, rightCols, allCols, nil } @@ -1565,6 +1609,9 @@ func joinOpToJoinType(op opt.Operator) (descpb.JoinType, error) { func (b *Builder) buildGroupBy(groupBy memo.RelExpr) (_ execPlan, outputCols colOrdMap, err error) { input, inputCols, err := b.buildGroupByInput(groupBy) + // The input column map is only used for the lifetime of this function, so + // free the map afterward. + defer b.colOrdsAlloc.Free(inputCols) if err != nil { return execPlan{}, colOrdMap{}, err } @@ -1724,7 +1771,10 @@ func (b *Builder) buildDistinct( if inputCols.MaxOrd()+1 == outCols.Len() { return ep, inputCols, nil } - return b.ensureColumns(ep, inputCols, distinct, outCols.ToList(), distinct.ProvidedPhysical().Ordering) + return b.ensureColumns( + ep, inputCols, distinct, outCols.ToList(), + distinct.ProvidedPhysical().Ordering, true, /* reuseInputCols */ + ) } func (b *Builder) buildGroupByInput( @@ -1777,10 +1827,14 @@ func (b *Builder) buildGroupByInput( cols = append(cols, exec.NodeColumnOrdinal(ordinal)) } + // The input column map is no longer used, so free it. + b.colOrdsAlloc.Free(inputCols) + reqOrdering, err := reqOrdering(groupByInput, outputCols) if err != nil { return execPlan{}, colOrdMap{}, err } + input.root, err = b.factory.ConstructSimpleProject(input.root, cols, reqOrdering) if err != nil { return execPlan{}, colOrdMap{}, err @@ -1821,15 +1875,26 @@ func (b *Builder) buildSetOp(set memo.RelExpr) (_ execPlan, outputCols colOrdMap // Note that (unless this is part of a larger query) the presentation property // will ensure that the columns are presented correctly in the output (i.e. in // the order `b, c, a`). - left, leftCols, err = b.ensureColumns(left, leftCols, leftExpr, private.LeftCols, leftExpr.ProvidedPhysical().Ordering) + left, leftCols, err = b.ensureColumns( + left, leftCols, leftExpr, private.LeftCols, + leftExpr.ProvidedPhysical().Ordering, true, /* reuseInputCols */ + ) if err != nil { return execPlan{}, colOrdMap{}, err } - right, rightCols, err = b.ensureColumns(right, rightCols, rightExpr, private.RightCols, rightExpr.ProvidedPhysical().Ordering) + right, rightCols, err = b.ensureColumns( + right, rightCols, rightExpr, private.RightCols, + rightExpr.ProvidedPhysical().Ordering, true, /* reuseInputCols */ + ) if err != nil { return execPlan{}, colOrdMap{}, err } + // The left and right column maps are only used for the lifetime of this + // function, so free them afterward. + defer b.colOrdsAlloc.Free(leftCols) + defer b.colOrdsAlloc.Free(rightCols) + var typ tree.UnionType var all bool switch set.Op() { @@ -2196,11 +2261,9 @@ func (b *Builder) buildOrdinality( // We have one additional ordinality column, which is ordered at the end of // the list. - // TODO(mgartner): Is the copy here necessary? - outputCols = b.colOrdsAlloc.Copy(inputCols) - outputCols.Set(ord.ColID, outputCols.MaxOrd()+1) + inputCols.Set(ord.ColID, inputCols.MaxOrd()+1) - return ep, outputCols, nil + return ep, inputCols, nil } func (b *Builder) buildIndexJoin( @@ -2226,6 +2289,9 @@ func (b *Builder) buildIndexJoin( } } + // The input column map is no longer used, so it can be freed. + b.colOrdsAlloc.Free(inputCols) + var needed exec.TableColumnOrdinalSet needed, outputCols = b.getColumns(join.Cols, join.Table) @@ -2512,30 +2578,60 @@ func (b *Builder) buildLookupJoin( } lookupOrdinals, lookupColMap := b.getColumns(lookupCols, join.Table) - // allExprCols are the columns used in expressions evaluated by this join. - allExprCols := b.joinOutputMap(inputCols, lookupColMap) - allCols := allExprCols - if join.IsFirstJoinInPairedJoiner { - // allCols needs to include the continuation column since it will be - // in the result output by this join. - allCols = b.colOrdsAlloc.Copy(allExprCols) - maxOrd := allCols.MaxOrd() + + // leftAndRightCols are the columns used in expressions evaluated by this + // join. + leftAndRightCols := b.joinOutputMap(inputCols, lookupColMap) + + // lookupColMap is no longer used, so it can be freed. + b.colOrdsAlloc.Free(lookupColMap) + + // Create the output column mapping. + switch { + case join.IsFirstJoinInPairedJoiner: + // For the first join in a paired join, outputCols needs to include the + // continuation column since it will be in the result output by this + // join. The first join in a paired join is always a left-join or an + // inner join, so the output columns always include the left and right + // columns. + if join.JoinType != opt.LeftJoinOp && join.JoinType != opt.InnerJoinOp { + return execPlan{}, colOrdMap{}, errors.AssertionFailedf( + "unexpected join type %s for lower join", join.JoinType, + ) + } + outputCols = b.colOrdsAlloc.Copy(leftAndRightCols) + + maxOrd := outputCols.MaxOrd() if maxOrd == -1 { - return execPlan{}, colOrdMap{}, errors.AssertionFailedf("allCols should not be empty") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("outputCols should not be empty") } // Assign the continuation column the next unused value in the map. - allCols.Set(join.ContinuationCol, maxOrd+1) - } - outputCols = allCols - if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { - // For semi and anti join, only the left columns are output. + outputCols.Set(join.ContinuationCol, maxOrd+1) + + // leftAndRightCols is only needed for the lifetime of the function, so + // free it afterward. + defer b.colOrdsAlloc.Free(leftAndRightCols) + + // inputCols is no longer used, so it can be freed. + b.colOrdsAlloc.Free(inputCols) + + case join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp: + // For semi and anti joins, only the left columns are output. outputCols = inputCols - } - ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, allExprCols.MaxOrd()+1), - ivarMap: allExprCols, + // leftAndRightCols is only needed for the lifetime of the function, so + // free it afterward. + defer b.colOrdsAlloc.Free(leftAndRightCols) + + default: + // For all other joins, the left and right columns are output. + outputCols = leftAndRightCols + + // inputCols is no longer used, so it can be freed. + b.colOrdsAlloc.Free(inputCols) } + + ctx := makeBuildScalarCtx(leftAndRightCols) var lookupExpr, remoteLookupExpr tree.TypedExpr if len(join.LookupExpr) > 0 { var err error @@ -2612,6 +2708,9 @@ func (b *Builder) buildLookupJoin( if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { outCols = join.Cols.Intersection(joinInputCols) } + // applySimpleProject creates a new map for outputCols, so outputCols + // can be freed. + defer b.colOrdsAlloc.Free(outputCols) return b.applySimpleProject(res, outputCols, join, outCols, join.ProvidedPhysical().Ordering) } return res, outputCols, nil @@ -2763,30 +2862,60 @@ func (b *Builder) buildInvertedJoin( lookupCols.Add(invertedColID) lookupOrdinals, lookupColMap := b.getColumns(lookupCols, join.Table) - // allExprCols are the columns used in expressions evaluated by this join. - allExprCols := b.joinOutputMap(inputCols, lookupColMap) + // leftAndRightCols are the columns used in expressions evaluated by this join. + leftAndRightCols := b.joinOutputMap(inputCols, lookupColMap) + + // Create the output column mapping. + switch { + case join.IsFirstJoinInPairedJoiner: + // For the first join in a paired join, outputCols needs to include the + // continuation column since it will be in the result output by this + // join. The first join in a paired join is always a left-join or an + // inner join, so the output columns always include the left and right + // columns. + if join.JoinType != opt.LeftJoinOp && join.JoinType != opt.InnerJoinOp { + return execPlan{}, colOrdMap{}, errors.AssertionFailedf( + "unexpected join type %s for lower join", join.JoinType, + ) + } + outputCols = b.colOrdsAlloc.Copy(leftAndRightCols) - allCols := allExprCols - if join.IsFirstJoinInPairedJoiner { - // allCols needs to include the continuation column since it will be - // in the result output by this join. - allCols = b.colOrdsAlloc.Copy(allExprCols) - maxOrd := allCols.MaxOrd() + maxOrd := outputCols.MaxOrd() if maxOrd == -1 { - return execPlan{}, colOrdMap{}, errors.AssertionFailedf("allCols should not be empty") + return execPlan{}, colOrdMap{}, errors.AssertionFailedf("outputCols should not be empty") } // Assign the continuation column the next unused value in the map. - allCols.Set(join.ContinuationCol, maxOrd+1) - } - outputCols = allCols - if join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp { + // TODO(mgartner): It's currently safe to use maxOrd like this, but it + // is not robust because it's not guaranteed to be the max ordinal, only + // an approximate upper-bound. + outputCols.Set(join.ContinuationCol, maxOrd+1) + + // leftAndRightCols is only needed for the lifetime of the function, so free + // it afterward. + defer b.colOrdsAlloc.Free(leftAndRightCols) + + // inputCols is no longer used, so it can be freed. + b.colOrdsAlloc.Free(inputCols) + + case join.JoinType == opt.SemiJoinOp || join.JoinType == opt.AntiJoinOp: // For semi and anti join, only the left columns are output. outputCols = inputCols + + // leftAndRightCols is only needed for the lifetime of the function, so free + // it afterward. + defer b.colOrdsAlloc.Free(leftAndRightCols) + + default: + // For all other joins, the left and right columns are output. + outputCols = leftAndRightCols + + // inputCols is no longer used, so it can be freed. + b.colOrdsAlloc.Free(inputCols) } ctx := buildScalarCtx{ - ivh: tree.MakeIndexedVarHelper(nil /* container */, allExprCols.MaxOrd()+1), - ivarMap: b.colOrdsAlloc.Copy(allExprCols), + ivh: tree.MakeIndexedVarHelper(nil /* container */, leftAndRightCols.MaxOrd()+1), + ivarMap: b.colOrdsAlloc.Copy(leftAndRightCols), } onExpr, err := b.buildScalar(&ctx, &join.On) if err != nil { @@ -2887,6 +3016,8 @@ func (b *Builder) buildZigzagJoin( rightCols.Add(join.RightTable.IndexColumnID(rightIndex, i)) } + // TODO(mgartner): Free leftColMap, rightColMap, and other ordColMaps below + // when possible. leftOrdinals, leftColMap := b.getColumns(leftCols, join.LeftTable) rightOrdinals, rightColMap := b.getColumns(rightCols, join.RightTable) @@ -3063,14 +3194,17 @@ func (b *Builder) buildRecursiveCTE( } // Make sure we have the columns in the correct order. - initial, initialCols, err = b.ensureColumns(initial, initialCols, rec.Initial, rec.InitialCols, nil /* ordering */) + initial, initialCols, err = b.ensureColumns( + initial, initialCols, rec.Initial, rec.InitialCols, + nil /* ordering */, true, /* reuseInputCols */ + ) if err != nil { return execPlan{}, colOrdMap{}, err } // Renumber the columns so they match the columns expected by the recursive // query. - initialCols = b.colOrdsAlloc.Alloc() + initialCols.Clear() for i, col := range rec.OutCols { initialCols.Set(col, i) } @@ -3103,7 +3237,10 @@ func (b *Builder) buildRecursiveCTE( return nil, err } // Ensure columns are output in the same order. - plan, planCols, err = innerBld.ensureColumns(plan, planCols, rec.Recursive, rec.RecursiveCols, opt.Ordering{}) + plan, _, err = innerBld.ensureColumns( + plan, planCols, rec.Recursive, rec.RecursiveCols, + opt.Ordering{}, true, /* reuseInputCols */ + ) if err != nil { return nil, err } @@ -3150,7 +3287,10 @@ func (b *Builder) buildWithScan( res := execPlan{root: node} // Apply any necessary projection to produce the InCols in the given order. - res, _, err = b.ensureColumns(res, e.outputCols, withScan, withScan.InCols, withScan.ProvidedPhysical().Ordering) + res, _, err = b.ensureColumns( + res, e.outputCols, withScan, withScan.InCols, + withScan.ProvidedPhysical().Ordering, false, /* reuseInputCols */ + ) if err != nil { return execPlan{}, colOrdMap{}, err } @@ -3167,7 +3307,9 @@ func (b *Builder) buildWithScan( func (b *Builder) buildProjectSet( projectSet *memo.ProjectSetExpr, ) (_ execPlan, outputCols colOrdMap, err error) { + // TODO(mgartner): Free inputCols and other ordColMaps below when possible. input, inputCols, err := b.buildRelational(projectSet.Input) + if err != nil { return execPlan{}, colOrdMap{}, err } @@ -3370,6 +3512,7 @@ func (b *Builder) buildFrame(inputCols colOrdMap, w *memo.WindowsItem) (*tree.Wi } func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols colOrdMap, err error) { + // TODO(mgartner): Free inputCols and other ordColMaps below when possible. input, inputCols, err := b.buildRelational(w.Input) if err != nil { return execPlan{}, colOrdMap{}, err @@ -3388,7 +3531,9 @@ func (b *Builder) buildWindow(w *memo.WindowExpr) (_ execPlan, outputCols colOrd // TODO(justin): this call to ensureColumns is kind of unfortunate because it // can result in an extra render beneath each window function. Figure out a // way to alleviate this. - input, inputCols, err = b.ensureColumns(input, inputCols, w, desiredCols, opt.Ordering{}) + input, inputCols, err = b.ensureColumns( + input, inputCols, w, desiredCols, opt.Ordering{}, true, /* reuseInputCols */ + ) if err != nil { return execPlan{}, colOrdMap{}, err } @@ -3645,13 +3790,16 @@ func (b *Builder) needProjection( } // ensureColumns applies a projection as necessary to make the output match the -// given list of columns; colNames is optional. +// given list of columns; colNames is optional. If reuseInputCols is true, then +// inputCols will be reused to build outputCols, and the caller must no longer +// use inputCols. func (b *Builder) ensureColumns( input execPlan, inputCols colOrdMap, inputExpr memo.RelExpr, colList opt.ColList, provided opt.Ordering, + reuseInputCols bool, ) (_ execPlan, outputCols colOrdMap, err error) { cols, needProj, err := b.needProjection(inputCols, colList) if err != nil { @@ -3664,7 +3812,12 @@ func (b *Builder) ensureColumns( // we need to explicitly annotate the latter with estimates since the code // in buildRelational() will attach them to the project. b.maybeAnnotateWithEstimates(input.root, inputExpr) - outputCols = b.colOrdsAlloc.Alloc() + if reuseInputCols { + outputCols = inputCols + outputCols.Clear() + } else { + outputCols = b.colOrdsAlloc.Alloc() + } for i, col := range colList { outputCols.Set(col, i) } From 3f4d099d2bd0099550a33e405acbd28438e86665 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Mon, 12 Feb 2024 12:53:48 -0500 Subject: [PATCH 6/6] opt/execbuilder: faster maximum ordinal method for colOrdMap This commit makes `colOrdMap.MaxOrd()` a constant-time operation in most cases. See the newly added comments for more details. Release note: None --- pkg/sql/opt/exec/execbuilder/col_ord_map.go | 66 +++++++++++++++---- .../opt/exec/execbuilder/col_ord_map_test.go | 41 ++++++++++-- pkg/sql/opt/exec/execbuilder/relational.go | 3 - 3 files changed, 90 insertions(+), 20 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map.go b/pkg/sql/opt/exec/execbuilder/col_ord_map.go index 3b7386497e72..9ebfc7dcacd0 100644 --- a/pkg/sql/opt/exec/execbuilder/col_ord_map.go +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map.go @@ -91,6 +91,11 @@ type colOrdMap struct { // 1, which allows the map to store the zero ordinal and have the zero value // in the map represent an unset column ID. ords []int32 + // maxOrd is the maximum ordinal in the map and is used by MaxOrd. If it is + // 0, then the set is empty. If it is -1, then the current maximum ordinal + // is "unknown" and MaxOrd must scan the entire map to find the maximum + // ordinal. See Set and MaxOrd for more details. + maxOrd int } // newColOrdMap returns a new column mapping that can store column IDs less than @@ -111,6 +116,20 @@ func (m *colOrdMap) Set(col opt.ColumnID, ord int) { if ord > math.MaxInt32 { panic(errors.AssertionFailedf("ordinal %d exceeds max ordinal %d", ord-1, math.MaxInt32-1)) } + switch { + case m.maxIsUnknown(): + // If the maximum ordinal is currently unknown, then leave it as-is. + case m.ords[col] > 0 && m.ords[col] == int32(m.maxOrd) && ord < m.maxOrd: + // If we are overriding an ordinal that was previously the maximum + // ordinal with a smaller ordinal, then we don't know what the new + // maximum ordinal is. So we set the maximum ordinal as "unknown". This + // makes MaxOrd scan the map to find the maximum ordinal. See MaxOrd for + // more details. + m.setUnknownMax() + default: + // Otherwise, set the known maximum ordinal. + m.maxOrd = max(m.maxOrd, ord) + } m.ords[col] = int32(ord) } @@ -128,19 +147,34 @@ func (m colOrdMap) Get(col opt.ColumnID) (ord int, ok bool) { return ord - 1, true } -// MaxOrd returns the maximum ordinal stored in the map, or -1 if the map is -// empty. +// MaxOrd returns the maximum ordinal in the map, or -1 if the map is empty. +// +// In most cases, MaxOrd has constant time complexity. If the map had a previous +// maximum ordinal that was overwritten by a smaller ordinal, then MaxOrd has +// linear time complexity with respect to the size of the map. In this case, the +// result is memoized, so future calls to MaxOrd are constant time until the +// maximum ordinal is overwritten by a smaller ordinal again. +// +// The case with linear time complexity should be rare in practice. The benefit +// of this approach is that it allows for constant time complexity in most cases +// with only constant additional space. Making MaxOrd constant-time in all cases +// would require non-constant additional space to keep a history of all previous +// maximum ordinals. func (m colOrdMap) MaxOrd() int { - maxOrd := -1 - for _, ord := range m.ords { - if ord == 0 { - continue - } + // If the maximum ordinal is known, return it. + if !m.maxIsUnknown() { // Reverse the bias when fetching the max ordinal from the map. - ord-- - maxOrd = max(maxOrd, int(ord)) + return m.maxOrd - 1 } - return maxOrd + // Otherwise, maxOrd is negative, meaning that a previous maximum ordinal + // was overwritten by a smaller ordinal. So we have to search for the + // maximum ordinal in the set. + m.maxOrd = 0 + for _, ord := range m.ords { + m.maxOrd = max(m.maxOrd, int(ord)) + } + // Reverse the bias when fetching the max ordinal from the map. + return m.maxOrd - 1 } // ForEach calls the given function for each column ID and ordinal pair in the @@ -163,11 +197,21 @@ func (m *colOrdMap) CopyFrom(other colOrdMap) { len(m.ords), len(other.ords))) } copy(m.ords, other.ords) + m.maxOrd = other.maxOrd } // Clear clears the map. The allocated memory is retained for future reuse. -func (m colOrdMap) Clear() { +func (m *colOrdMap) Clear() { for i := range m.ords { m.ords[i] = 0 } + m.maxOrd = 0 +} + +func (m colOrdMap) maxIsUnknown() bool { + return m.maxOrd == -1 +} + +func (m *colOrdMap) setUnknownMax() { + m.maxOrd = -1 } diff --git a/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go b/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go index 93e3b2723fca..e787b632fd6d 100644 --- a/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go +++ b/pkg/sql/opt/exec/execbuilder/col_ord_map_test.go @@ -11,6 +11,7 @@ package execbuilder import ( + "math" "testing" "github.com/cockroachdb/cockroach/pkg/sql/opt" @@ -24,17 +25,12 @@ func TestColOrdMap(t *testing.T) { const maxCol = 100 m := newColOrdMap(maxCol) oracle := make(map[opt.ColumnID]int) - - if m.MaxOrd() != -1 { - t.Errorf("expected empty map to have MaxOrd of -1, got %d", m.MaxOrd()) - } - rng, _ := randutil.NewTestRand() const numOps = 1000 for i := 0; i < numOps; i++ { col := opt.ColumnID(rng.Intn(maxCol + 1)) - ord := int(rng.Int31()) + ord := rng.Intn(math.MaxInt32) oracle[col] = ord m.Set(col, ord) @@ -77,3 +73,36 @@ func validate(t *testing.T, m colOrdMap, oracle map[opt.ColumnID]int) { } }) } + +func TestMaxOrd(t *testing.T) { + defer leaktest.AfterTest(t)() + + const maxCol = 100 + m := newColOrdMap(maxCol) + + assertMax := func(expectedMax int) { + maxOrd := m.MaxOrd() + if maxOrd != expectedMax { + t.Errorf("expected empty map to have MaxOrd of %d, got %d", expectedMax, maxOrd) + } + } + + // An empty map has a max ordinal of -1. + assertMax(-1) + + m.Set(1, 2) + assertMax(2) + + m.Set(1, 3) + m.Set(2, 3) + assertMax(3) + + m.Set(1, 1) + assertMax(3) + + m.Set(2, 1) + assertMax(1) + + m.Set(1, 0) + assertMax(1) +} diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 96cca1d756b9..97607d1cdc61 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -2885,9 +2885,6 @@ func (b *Builder) buildInvertedJoin( return execPlan{}, colOrdMap{}, errors.AssertionFailedf("outputCols should not be empty") } // Assign the continuation column the next unused value in the map. - // TODO(mgartner): It's currently safe to use maxOrd like this, but it - // is not robust because it's not guaranteed to be the max ordinal, only - // an approximate upper-bound. outputCols.Set(join.ContinuationCol, maxOrd+1) // leftAndRightCols is only needed for the lifetime of the function, so free