diff --git a/pkg/sql/opt/cat/column.go b/pkg/sql/opt/cat/column.go index 7c0ce872af23..8a8168968da4 100644 --- a/pkg/sql/opt/cat/column.go +++ b/pkg/sql/opt/cat/column.go @@ -15,6 +15,21 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) +// ColumnKind differentiates between different kinds of table columns. +type ColumnKind uint8 + +const ( + // Ordinary columns are "regular" table columns (including hidden columns + // like `rowid`). + Ordinary ColumnKind = iota + // WriteOnly columns are mutation columns that have to be updated on writes + // (inserts, updates, deletes) and cannot be otherwise accessed. + WriteOnly + // DeleteOnly columns are mutation columns that have to be updated only on + // deletes and cannot be otherwise accessed. + DeleteOnly +) + // Column is an interface to a table column, exposing only the information // needed by the query optimizer. type Column interface { @@ -85,5 +100,6 @@ type Column interface { // IsMutationColumn is a convenience function that returns true if the column at // the given ordinal position is a mutation column. func IsMutationColumn(table Table, ord int) bool { - return ord >= table.ColumnCount() + kind := table.ColumnKind(ord) + return kind == WriteOnly || kind == DeleteOnly } diff --git a/pkg/sql/opt/cat/table.go b/pkg/sql/opt/cat/table.go index 2d3191f768be..69fec39dff9b 100644 --- a/pkg/sql/opt/cat/table.go +++ b/pkg/sql/opt/cat/table.go @@ -41,21 +41,9 @@ type Table interface { // information_schema tables. IsVirtualTable() bool - // ColumnCount returns the number of public columns in the table. Public - // columns are not currently being added or dropped from the table. This - // method should be used when mutation columns can be ignored (the common - // case). - ColumnCount() int - - // WritableColumnCount returns the number of public and write-only columns in - // the table. Although write-only columns are not visible, any inserts and - // updates must still set them. WritableColumnCount is always >= ColumnCount. - WritableColumnCount() int - - // DeletableColumnCount returns the number of public, write-only, and - // delete- only columns in the table. DeletableColumnCount is always >= - // WritableColumnCount. - DeletableColumnCount() int + // AllColumnCount returns the number of columns in the table. This includes + // public columns, write-only columns, etc. + AllColumnCount() int // Column returns a Column interface to the column at the ith ordinal // position within the table, where i < ColumnCount. Note that the Columns @@ -70,6 +58,12 @@ type Table interface { // by deletable columns. Column(i int) Column + // ColumnKind returns the column kind. + // Note: this is not a method in Column for the efficiency of the + // Column implementation (which can't access this information without using + // extra objects). + ColumnKind(i int) ColumnKind + // IndexCount returns the number of public indexes defined on this table. // Public indexes are not currently being added or dropped from the table. // This method should be used when mutation columns can be ignored (the common diff --git a/pkg/sql/opt/cat/utils.go b/pkg/sql/opt/cat/utils.go index 00370e9b00e6..304246c6941b 100644 --- a/pkg/sql/opt/cat/utils.go +++ b/pkg/sql/opt/cat/utils.go @@ -121,40 +121,6 @@ func ResolveTableIndex( return found, foundTabName, nil } -// ConvertColumnIDsToOrdinals converts a list of ColumnIDs (such as from a -// tree.TableRef), to a list of ordinal positions of columns within the given -// table. See tree.Table for more information on column ordinals. -func ConvertColumnIDsToOrdinals(tab Table, columns []tree.ColumnID) (ordinals []int) { - ordinals = make([]int, len(columns)) - for i, c := range columns { - ord := 0 - cnt := tab.ColumnCount() - for ord < cnt { - if tab.Column(ord).ColID() == StableID(c) { - break - } - ord++ - } - if ord >= cnt { - panic(pgerror.Newf(pgcode.UndefinedColumn, - "column [%d] does not exist", c)) - } - ordinals[i] = ord - } - return ordinals -} - -// FindTableColumnByName returns the ordinal of the non-mutation column having -// the given name, if one exists in the given table. Otherwise, it returns -1. -func FindTableColumnByName(tab Table, name tree.Name) int { - for ord, n := 0, tab.ColumnCount(); ord < n; ord++ { - if tab.Column(ord).ColName() == name { - return ord - } - } - return -1 -} - // FormatTable nicely formats a catalog table using a treeprinter for debugging // and testing. func FormatTable(cat Catalog, tab Table, tp treeprinter.Node) { @@ -164,7 +130,7 @@ func FormatTable(cat Catalog, tab Table, tp treeprinter.Node) { } var buf bytes.Buffer - for i := 0; i < tab.DeletableColumnCount(); i++ { + for i := 0; i < tab.AllColumnCount(); i++ { buf.Reset() formatColumn(tab.Column(i), IsMutationColumn(tab, i), &buf) child.Child(buf.String()) diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 12831414eb4b..5bfbcb373ac6 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -702,7 +702,7 @@ func mutationOutputColMap(mutation memo.RelExpr) opt.ColMap { var colMap opt.ColMap ord := 0 - for i, n := 0, tab.DeletableColumnCount(); i < n; i++ { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { colID := private.Table.ColumnID(i) if outCols.Contains(colID) { colMap.Set(int(colID), ord) diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 2ad28a94d993..06f6ffe043ef 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -411,7 +411,7 @@ func (b *Builder) getColumns( var needed exec.TableColumnOrdinalSet var output opt.ColMap - columnCount := b.mem.Metadata().Table(tableID).DeletableColumnCount() + columnCount := b.mem.Metadata().Table(tableID).AllColumnCount() n := 0 for i := 0; i < columnCount; i++ { colID := tableID.ColumnID(i) diff --git a/pkg/sql/opt/memo/check_expr.go b/pkg/sql/opt/memo/check_expr.go index c25e710cd00b..657825755321 100644 --- a/pkg/sql/opt/memo/check_expr.go +++ b/pkg/sql/opt/memo/check_expr.go @@ -14,6 +14,7 @@ package memo import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -172,14 +173,14 @@ func (m *Memo) CheckExpr(e opt.Expr) { case *InsertExpr: tab := m.Metadata().Table(t.Table) - m.checkColListLen(t.InsertCols, tab.DeletableColumnCount(), "InsertCols") + m.checkColListLen(t.InsertCols, tab.AllColumnCount(), "InsertCols") m.checkColListLen(t.FetchCols, 0, "FetchCols") m.checkColListLen(t.UpdateCols, 0, "UpdateCols") // Ensure that insert columns include all columns except for delete-only // mutation columns (which do not need to be part of INSERT). - for i, n := 0, tab.WritableColumnCount(); i < n; i++ { - if t.InsertCols[i] == 0 { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { + if tab.ColumnKind(i) != cat.DeleteOnly && t.InsertCols[i] == 0 { panic(errors.AssertionFailedf("insert values not provided for all table columns")) } } @@ -189,8 +190,8 @@ func (m *Memo) CheckExpr(e opt.Expr) { case *UpdateExpr: tab := m.Metadata().Table(t.Table) m.checkColListLen(t.InsertCols, 0, "InsertCols") - m.checkColListLen(t.FetchCols, tab.DeletableColumnCount(), "FetchCols") - m.checkColListLen(t.UpdateCols, tab.DeletableColumnCount(), "UpdateCols") + m.checkColListLen(t.FetchCols, tab.AllColumnCount(), "FetchCols") + m.checkColListLen(t.UpdateCols, tab.AllColumnCount(), "UpdateCols") m.checkMutationExpr(t, &t.MutationPrivate) case *ZigzagJoinExpr: @@ -249,8 +250,10 @@ func (m *Memo) checkMutationExpr(rel RelExpr, private *MutationPrivate) { // Output columns should never include mutation columns. tab := m.Metadata().Table(private.Table) var mutCols opt.ColSet - for i, n := tab.ColumnCount(), tab.DeletableColumnCount(); i < n; i++ { - mutCols.Add(private.Table.ColumnID(i)) + for i, n := 0, tab.AllColumnCount(); i < n; i++ { + if cat.IsMutationColumn(tab, i) { + mutCols.Add(private.Table.ColumnID(i)) + } } if rel.Relational().OutputCols.Intersects(mutCols) { panic(errors.AssertionFailedf("output columns cannot include mutation columns")) diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index 0666a44f2548..e3ebc98eee7a 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -640,7 +640,7 @@ func (m *MutationPrivate) MapToInputCols(tabCols opt.ColSet) opt.ColSet { // AddEquivTableCols adds an FD to the given set that declares an equivalence // between each table column and its corresponding input column. func (m *MutationPrivate) AddEquivTableCols(md *opt.Metadata, fdset *props.FuncDepSet) { - for i, n := 0, md.Table(m.Table).DeletableColumnCount(); i < n; i++ { + for i, n := 0, md.Table(m.Table).AllColumnCount(); i < n; i++ { t := m.Table.ColumnID(i) id := m.MapToInputID(t) if id != 0 { diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 1405f7fda2e4..973e4462f802 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -1268,7 +1268,7 @@ func (b *logicalPropsBuilder) buildMutationProps(mutation RelExpr, rel *props.Re // Output Columns // -------------- // Only non-mutation columns are output columns. - for i, n := 0, tab.ColumnCount(); i < n; i++ { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { if private.IsColumnOutput(i) { colID := private.Table.ColumnID(i) rel.OutputCols.Add(colID) @@ -1289,7 +1289,7 @@ func (b *logicalPropsBuilder) buildMutationProps(mutation RelExpr, rel *props.Re // null or the corresponding insert and fetch/update columns are not null. In // other words, if either the source or destination column is not null, then // the column must be not null. - for i, n := 0, tab.ColumnCount(); i < n; i++ { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { tabColID := private.Table.ColumnID(i) if !rel.OutputCols.Contains(tabColID) { continue @@ -1562,7 +1562,7 @@ func MakeTableFuncDep(md *opt.Metadata, tabID opt.TableID) *props.FuncDepSet { // Make now and annotate the metadata table with it for next time. var allCols opt.ColSet tab := md.Table(tabID) - for i := 0; i < tab.DeletableColumnCount(); i++ { + for i := 0; i < tab.AllColumnCount(); i++ { allCols.Add(tabID.ColumnID(i)) } @@ -1812,16 +1812,16 @@ func ensureInputPropsForIndex( } } -// tableNotNullCols returns the set of not-NULL columns from the given table. +// tableNotNullCols returns the set of not-NULL non-mutation columns from the given table. func tableNotNullCols(md *opt.Metadata, tabID opt.TableID) opt.ColSet { cs := opt.ColSet{} tab := md.Table(tabID) // Only iterate over non-mutation columns, since even non-null mutation // columns can be null during backfill. - for i := 0; i < tab.ColumnCount(); i++ { + for i := 0; i < tab.AllColumnCount(); i++ { // Non-null mutation columns can be null during backfill. - if !tab.Column(i).IsNullable() { + if !cat.IsMutationColumn(tab, i) && !tab.Column(i).IsNullable() { cs.Add(tabID.ColumnID(i)) } } diff --git a/pkg/sql/opt/memo/multiplicity_builder.go b/pkg/sql/opt/memo/multiplicity_builder.go index 7952a55163fa..5c9c3d8f9634 100644 --- a/pkg/sql/opt/memo/multiplicity_builder.go +++ b/pkg/sql/opt/memo/multiplicity_builder.go @@ -107,7 +107,7 @@ func deriveUnfilteredCols(in RelExpr) opt.ColSet { md := t.Memo().Metadata() baseTable := md.Table(t.Table) if t.IsUnfiltered(md) { - for i, cnt := 0, baseTable.ColumnCount(); i < cnt; i++ { + for i, cnt := 0, baseTable.AllColumnCount(); i < cnt; i++ { unfilteredCols.Add(t.Table.ColumnID(i)) } } diff --git a/pkg/sql/opt/memo/multiplicity_builder_test.go b/pkg/sql/opt/memo/multiplicity_builder_test.go index bcfd9c97a136..697927c5a48a 100644 --- a/pkg/sql/opt/memo/multiplicity_builder_test.go +++ b/pkg/sql/opt/memo/multiplicity_builder_test.go @@ -329,7 +329,7 @@ func (ob *testOpBuilder) makeScan(tableName tree.Name) (scan RelExpr, vars []*Va tab := ob.cat.Table(tn) tabID := ob.mem.Metadata().AddTable(tab, tn) var cols opt.ColSet - for i := 0; i < tab.ColumnCount(); i++ { + for i := 0; i < tab.AllColumnCount(); i++ { col := tabID.ColumnID(i) cols.Add(col) newVar := ob.mem.MemoizeVariable(col) diff --git a/pkg/sql/opt/memo/statistics_builder_test.go b/pkg/sql/opt/memo/statistics_builder_test.go index 093ef2caeaf1..b477f809b494 100644 --- a/pkg/sql/opt/memo/statistics_builder_test.go +++ b/pkg/sql/opt/memo/statistics_builder_test.go @@ -99,7 +99,7 @@ func TestGetStatsFromConstraint(t *testing.T) { t.Helper() var cols opt.ColSet - for i := 0; i < tab.ColumnCount(); i++ { + for i := 0; i < tab.AllColumnCount(); i++ { cols.Add(tabID.ColumnID(i)) } diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index d35f3216208f..cf364950cdd2 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -336,7 +336,7 @@ func (md *Metadata) AddTable(tab cat.Table, alias *tree.TableName) TableID { } md.tables = append(md.tables, TableMeta{MetaID: tabID, Table: tab, Alias: *alias}) - colCount := tab.DeletableColumnCount() + colCount := tab.AllColumnCount() if md.cols == nil { md.cols = make([]ColumnMeta, 0, colCount) } diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index f1a61ccc8ccf..64d3dff51d49 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -194,7 +194,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope } ins.Columns = make(tree.NameList, len(refColumns)) - for i, ord := range cat.ConvertColumnIDsToOrdinals(tab, refColumns) { + for i, ord := range resolveNumericColumnRefs(tab, refColumns) { ins.Columns[i] = tab.Column(ord).ColName() } } @@ -288,7 +288,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // Wrap the input in one LEFT OUTER JOIN per UNIQUE index, and filter out // rows that have conflicts. See the buildInputForDoNothing comment for // more details. - conflictOrds := mb.mapColumnNamesToOrdinals(ins.OnConflict.Columns) + conflictOrds := mb.mapPublicColumnNamesToOrdinals(ins.OnConflict.Columns) mb.buildInputForDoNothing(inScope, conflictOrds) // Since buildInputForDoNothing filters out rows with conflicts, always @@ -321,7 +321,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope default: // Left-join each input row to the target table, using the conflict columns // as the join condition. - conflictOrds := mb.mapColumnNamesToOrdinals(ins.OnConflict.Columns) + conflictOrds := mb.mapPublicColumnNamesToOrdinals(ins.OnConflict.Columns) mb.buildInputForUpsert(inScope, conflictOrds, ins.OnConflict.Where) // Derive the columns that will be updated from the SET expressions. @@ -366,7 +366,7 @@ func (mb *mutationBuilder) needExistingRows() bool { // TODO(andyk): This is not true in the case of composite key encodings. See // issue #34518. keyOrds := getIndexLaxKeyOrdinals(mb.tab.Index(cat.PrimaryIndex)) - for i, n := 0, mb.tab.DeletableColumnCount(); i < n; i++ { + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { if keyOrds.Contains(i) { // #1: Don't consider key columns. continue @@ -512,9 +512,9 @@ func (mb *mutationBuilder) addTargetTableColsForInsert(maxCols int) { // Only consider non-mutation columns, since mutation columns are hidden from // the SQL user. numCols := 0 - for i, n := 0, mb.tab.ColumnCount(); i < n && numCols < maxCols; i++ { - // Skip hidden columns. - if mb.tab.Column(i).IsHidden() { + for i, n := 0, mb.tab.AllColumnCount(); i < n && numCols < maxCols; i++ { + // Skip mutation or hidden columns. + if cat.IsMutationColumn(mb.tab, i) || mb.tab.Column(i).IsHidden() { continue } @@ -559,12 +559,13 @@ func (mb *mutationBuilder) buildInputForInsert(inScope *scope, inputRows *tree.S desiredTypes[i] = mb.md.ColumnMeta(colID).Type } } else { - // Do not target mutation columns. - desiredTypes = make([]*types.T, 0, mb.tab.ColumnCount()) - for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - tabCol := mb.tab.Column(i) - if !tabCol.IsHidden() { - desiredTypes = append(desiredTypes, tabCol.DatumType()) + desiredTypes = make([]*types.T, 0, mb.tab.AllColumnCount()) + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { + if !cat.IsMutationColumn(mb.tab, i) { + tabCol := mb.tab.Column(i) + if !tabCol.IsHidden() { + desiredTypes = append(desiredTypes, tabCol.DatumType()) + } } } } @@ -758,7 +759,7 @@ func (mb *mutationBuilder) buildInputForDoNothing(inScope *scope, conflictOrds u conflictCols, mb.outScope, true /* nullsAreDistinct */, "" /* errorOnDup */) } - mb.targetColList = make(opt.ColList, 0, mb.tab.DeletableColumnCount()) + mb.targetColList = make(opt.ColList, 0, mb.tab.AllColumnCount()) mb.targetColSet = opt.ColSet{} } @@ -872,7 +873,7 @@ func (mb *mutationBuilder) buildInputForUpsert( mb.b.buildWhere(where, mb.outScope) } - mb.targetColList = make(opt.ColList, 0, mb.tab.DeletableColumnCount()) + mb.targetColList = make(opt.ColList, 0, mb.tab.AllColumnCount()) mb.targetColSet = opt.ColSet{} } @@ -903,7 +904,7 @@ func (mb *mutationBuilder) setUpsertCols(insertCols tree.NameList) { for _, name := range insertCols { // Table column must exist, since existence of insertCols has already // been checked previously. - ord := cat.FindTableColumnByName(mb.tab, name) + ord := findPublicTableColumnByName(mb.tab, name) mb.updateOrds[ord] = mb.insertOrds[ord] } } else { @@ -911,8 +912,10 @@ func (mb *mutationBuilder) setUpsertCols(insertCols tree.NameList) { } // Never update mutation columns. - for i, n := mb.tab.ColumnCount(), mb.tab.DeletableColumnCount(); i < n; i++ { - mb.updateOrds[i] = -1 + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { + if cat.IsMutationColumn(mb.tab, i) { + mb.updateOrds[i] = -1 + } } // Never update primary key columns. @@ -968,7 +971,7 @@ func (mb *mutationBuilder) projectUpsertColumns() { // Add a new column for each target table column that needs to be upserted. // This can include mutation columns. - for i, n := 0, mb.tab.DeletableColumnCount(); i < n; i++ { + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { insertScopeOrd := mb.insertOrds[i] updateScopeOrd := mb.updateOrds[i] if updateScopeOrd == -1 { @@ -1048,15 +1051,16 @@ func (mb *mutationBuilder) ensureUniqueConflictCols(conflictOrds util.FastIntSet "there is no unique or exclusion constraint matching the ON CONFLICT specification")) } -// mapColumnNamesToOrdinals returns the set of ordinal positions within the -// target table that correspond to the given names. -func (mb *mutationBuilder) mapColumnNamesToOrdinals(names tree.NameList) util.FastIntSet { +// mapPublicColumnNamesToOrdinals returns the set of ordinal positions within +// the target table that correspond to the given names. Mutation columns are +// ignored. +func (mb *mutationBuilder) mapPublicColumnNamesToOrdinals(names tree.NameList) util.FastIntSet { var ords util.FastIntSet for _, name := range names { found := false - for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { tabCol := mb.tab.Column(i) - if tabCol.ColName() == name { + if tabCol.ColName() == name && !cat.IsMutationColumn(mb.tab, i) { ords.Add(i) found = true break diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 78a9de4e9d89..d8c9b16ce7b1 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -169,10 +169,11 @@ func (mb *mutationBuilder) init(b *Builder, opName string, tab cat.Table, alias mb.opName = opName mb.tab = tab mb.alias = alias - mb.targetColList = make(opt.ColList, 0, tab.DeletableColumnCount()) + + n := tab.AllColumnCount() + mb.targetColList = make(opt.ColList, 0, n) // Allocate segmented array of scope column ordinals. - n := tab.DeletableColumnCount() numPartialIndexes := partialIndexCount(tab) scopeOrds := make([]scopeOrdinal, n*4+tab.CheckCount()+2*numPartialIndexes) for i := range scopeOrds { @@ -399,7 +400,7 @@ func (mb *mutationBuilder) addTargetColsByName(names tree.NameList) { for _, name := range names { // Determine the ordinal position of the named column in the table and // add it as a target column. - if ord := cat.FindTableColumnByName(mb.tab, name); ord != -1 { + if ord := findPublicTableColumnByName(mb.tab, name); ord != -1 { mb.addTargetCol(ord) continue } @@ -545,9 +546,12 @@ func (mb *mutationBuilder) addSynthesizedCols( ) { var projectionsScope *scope - // Skip delete-only mutation columns, since they are ignored by all mutation - // operators that synthesize columns. - for i, n := 0, mb.tab.WritableColumnCount(); i < n; i++ { + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { + if mb.tab.ColumnKind(i) == cat.DeleteOnly { + // Skip delete-only mutation columns, since they are ignored by all + // mutation operators that synthesize columns. + continue + } // Skip columns that are already specified. if scopeOrds[i] != -1 { continue @@ -775,7 +779,7 @@ func (mb *mutationBuilder) addPartialIndexCols(aliasPrefix string, ords []scopeO func (mb *mutationBuilder) disambiguateColumns() { // Determine the set of scope columns that will have their names preserved. var preserve util.FastIntSet - for i, n := 0, mb.tab.DeletableColumnCount(); i < n; i++ { + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { scopeOrd := mb.mapToReturnScopeOrd(i) if scopeOrd != -1 { preserve.Add(int(scopeOrd)) @@ -825,11 +829,12 @@ func (mb *mutationBuilder) makeMutationPrivate(needResults bool) *memo.MutationP } if needResults { - // Only non-mutation columns are output columns. ReturnCols needs to have - // DeletableColumnCount entries, but only the first ColumnCount entries - // can be defined (i.e. >= 0). - private.ReturnCols = make(opt.ColList, mb.tab.DeletableColumnCount()) - for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { + private.ReturnCols = make(opt.ColList, mb.tab.AllColumnCount()) + for i, n := 0, mb.tab.AllColumnCount(); i < n; i++ { + if cat.IsMutationColumn(mb.tab, i) { + // Only non-mutation columns are output columns. + continue + } scopeOrd := mb.mapToReturnScopeOrd(i) if scopeOrd == -1 { panic(errors.AssertionFailedf("column %d is not available in the mutation input", i)) @@ -936,7 +941,7 @@ func (mb *mutationBuilder) checkNumCols(expected, actual int) { // reuse. func (mb *mutationBuilder) parseDefaultOrComputedExpr(colID opt.ColumnID) tree.Expr { if mb.parsedExprs == nil { - mb.parsedExprs = make([]tree.Expr, mb.tab.DeletableColumnCount()) + mb.parsedExprs = make([]tree.Expr, mb.tab.AllColumnCount()) } // Return expression from cache, if it was already parsed previously. diff --git a/pkg/sql/opt/optbuilder/scope.go b/pkg/sql/opt/optbuilder/scope.go index 226a54c4ab2e..478208755930 100644 --- a/pkg/sql/opt/optbuilder/scope.go +++ b/pkg/sql/opt/optbuilder/scope.go @@ -17,6 +17,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" @@ -221,14 +222,17 @@ func (s *scope) appendColumnsFromScope(src *scope) { } } -// appendColumnsFromTable adds all columns from the given table metadata to this -// scope. +// appendColumnsFromTable adds all non-mutation columns from the given table +// metadata to this scope. func (s *scope) appendColumnsFromTable(tabMeta *opt.TableMeta, alias *tree.TableName) { tab := tabMeta.Table if s.cols == nil { - s.cols = make([]scopeColumn, 0, tab.ColumnCount()) + s.cols = make([]scopeColumn, 0, tab.AllColumnCount()) } - for i, n := 0, tab.ColumnCount(); i < n; i++ { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { + if cat.IsMutationColumn(tab, i) { + continue + } tabCol := tab.Column(i) s.cols = append(s.cols, scopeColumn{ name: tabCol.ColName(), diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 35d21f88cc03..e9b9744c8779 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -393,7 +393,7 @@ func (b *Builder) buildScanFromTableRef( panic(pgerror.Newf(pgcode.Syntax, "an explicit list of column IDs must include at least one column")) } - ordinals = cat.ConvertColumnIDsToOrdinals(tab, ref.Columns) + ordinals = resolveNumericColumnRefs(tab, ref.Columns) } tn := tree.MakeUnqualifiedTableName(tab.Name()) @@ -442,29 +442,9 @@ func (b *Builder) buildScan( tabMeta.IgnoreForeignKeys = true } - colCount := len(ordinals) - if colCount == 0 { - // If scanning mutation columns, then include writable and deletable - // columns in the output, in addition to public columns. - if scanMutationCols { - colCount = tab.DeletableColumnCount() - } else { - colCount = tab.ColumnCount() - } - } - - getOrdinal := func(i int) int { - if ordinals == nil { - return i - } - return ordinals[i] - } - - var tabColIDs opt.ColSet outScope = inScope.push() - outScope.cols = make([]scopeColumn, 0, colCount) - for i := 0; i < colCount; i++ { - ord := getOrdinal(i) + var tabColIDs opt.ColSet + addCol := func(ord int) { col := tab.Column(ord) colID := tabID.ColumnID(ord) tabColIDs.Add(colID) @@ -480,6 +460,23 @@ func (b *Builder) buildScan( }) } + if ordinals == nil { + // If no ordinals are requested, then add in all of the table columns + // (including mutation columns if scanMutationCols is true). + outScope.cols = make([]scopeColumn, 0, tab.AllColumnCount()) + for i, n := 0, tab.AllColumnCount(); i < n; i++ { + if scanMutationCols || !cat.IsMutationColumn(tab, i) { + addCol(i) + } + } + } else { + // Otherwise, just add the ordinals. + outScope.cols = make([]scopeColumn, 0, len(ordinals)) + for _, ord := range ordinals { + addCol(ord) + } + } + if tab.IsVirtualTable() { if indexFlags != nil { panic(pgerror.Newf(pgcode.Syntax, @@ -536,7 +533,11 @@ func (b *Builder) buildScan( // We will track the ColumnID to Ord mapping so Ords can be added // when a column is referenced. for i, col := range outScope.cols { - dep.ColumnIDToOrd[col.id] = getOrdinal(i) + if ordinals == nil { + dep.ColumnIDToOrd[col.id] = i + } else { + dep.ColumnIDToOrd[col.id] = ordinals[i] + } } if private.Flags.ForceIndex { dep.SpecificIndex = true @@ -575,10 +576,11 @@ func (b *Builder) addCheckConstraintsForTable(tabMeta *opt.TableMeta) { tableScope := b.allocScope() tableScope.appendColumnsFromTable(tabMeta, &tabMeta.Alias) - // Find the non-nullable table columns. + // Find the non-nullable table columns. Mutation columns can be NULL during + // backfill, so they should be excluded. var notNullCols opt.ColSet - for i := 0; i < tab.ColumnCount(); i++ { - if !tab.Column(i).IsNullable() { + for i := 0; i < tab.AllColumnCount(); i++ { + if !tab.Column(i).IsNullable() && !cat.IsMutationColumn(tab, i) { notNullCols.Add(tabMeta.MetaID.ColumnID(i)) } } @@ -626,11 +628,16 @@ func (b *Builder) addCheckConstraintsForTable(tabMeta *opt.TableMeta) { func (b *Builder) addComputedColsForTable(tabMeta *opt.TableMeta) { var tableScope *scope tab := tabMeta.Table - for i, n := 0, tab.ColumnCount(); i < n; i++ { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { tabCol := tab.Column(i) if !tabCol.IsComputed() { continue } + if cat.IsMutationColumn(tab, i) { + // Mutation columns can be NULL during backfill, so they won't equal the + // computed column expression value (in general). + continue + } expr, err := parser.ParseExpr(tabCol.ComputedExprStr()) if err != nil { panic(err) diff --git a/pkg/sql/opt/optbuilder/testdata/select b/pkg/sql/opt/optbuilder/testdata/select index c1bc7799dcb6..95d8e194a536 100644 --- a/pkg/sql/opt/optbuilder/testdata/select +++ b/pkg/sql/opt/optbuilder/testdata/select @@ -17,6 +17,15 @@ exec-ddl CREATE TABLE tab55 (a INT PRIMARY KEY, b INT NOT NULL, CONSTRAINT foo CHECK (a+b < 10)) ---- +exec-ddl +CREATE TABLE tab56 ( + a INT PRIMARY KEY, + b INT, + "c:write-only" INT, + "d:delete-only" INT +) +---- + # SELECT with no table. build @@ -1095,6 +1104,22 @@ scan t └── check constraint expressions └── (a:1 + b:2) < 10 +# Test that we error out if we refer to a mutation column, +build +SELECT * FROM [56(1,3) AS t(a, b)] +---- +error (42703): column [3] does not exist + +build +SELECT * FROM [56(1,4) AS t(a, b)] +---- +error (42703): column [4] does not exist + +build +INSERT INTO [56(1,3) AS t(a,b)] VALUES (1,2) +---- +error (42703): column [3] does not exist + # Regression test for #28388. Ensure that selecting from a table with no # columns does not cause a panic. exec-ddl diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index ce52a4cf1ea7..152c428a737b 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -635,3 +635,38 @@ func (b *Builder) checkPrivilege(name opt.MDDepName, ds cat.DataSource, priv pri // cached and later checked for freshness. b.factory.Metadata().AddDependency(name, ds, priv) } + +// resolveNumericColumnRefs converts a list of tree.ColumnIDs from a +// tree.TableRef to a list of ordinal positions within the given table. Mutation +// columns are not visible. See tree.Table for more information on column +// ordinals. +func resolveNumericColumnRefs(tab cat.Table, columns []tree.ColumnID) (ordinals []int) { + ordinals = make([]int, len(columns)) + for i, c := range columns { + ord := 0 + cnt := tab.AllColumnCount() + for ord < cnt { + if tab.Column(ord).ColID() == cat.StableID(c) && !cat.IsMutationColumn(tab, ord) { + break + } + ord++ + } + if ord >= cnt { + panic(pgerror.Newf(pgcode.UndefinedColumn, "column [%d] does not exist", c)) + } + ordinals[i] = ord + } + return ordinals +} + +// findPublicTableColumnByName returns the ordinal of the non-mutation column +// having the given name, if one exists in the given table. Otherwise, it +// returns -1. +func findPublicTableColumnByName(tab cat.Table, name tree.Name) int { + for ord, n := 0, tab.AllColumnCount(); ord < n; ord++ { + if tab.Column(ord).ColName() == name && !cat.IsMutationColumn(tab, ord) { + return ord + } + } + return -1 +} diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index 52b613bffe75..013493d162c9 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -94,7 +94,7 @@ func (tc *Catalog) CreateTable(stmt *tree.CreateTable) *Table { if !hasPrimaryIndex { rowid := &Column{ - Ordinal: tab.ColumnCount(), + Ordinal: tab.AllColumnCount(), Name: "rowid", Type: types.Int, Hidden: true, @@ -254,7 +254,7 @@ func (tc *Catalog) CreateTableAs(name tree.TableName, columns []*Column) *Table tab := &Table{TabID: tc.nextStableID(), TabName: name, Catalog: tc, Columns: columns} rowid := &Column{ - Ordinal: tab.ColumnCount(), + Ordinal: tab.AllColumnCount(), Name: "rowid", Type: types.Int, Hidden: true, @@ -380,7 +380,7 @@ func (tc *Catalog) resolveFK(tab *Table, d *tree.ForeignKeyConstraintTableDef) { func (tt *Table) addColumn(def *tree.ColumnTableDef) { nullable := !def.PrimaryKey.IsPrimaryKey && def.Nullable.Nullability != tree.NotNull col := &Column{ - Ordinal: tt.ColumnCount(), + Ordinal: tt.AllColumnCount(), Name: string(def.Name), Type: tree.MustBeStaticallyKnownType(def.Type), Nullable: nullable, @@ -480,7 +480,7 @@ func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) *Index { pkOrdinals.Add(c.Ordinal) } // Add the rest of the columns in the table. - for i, n := 0, tt.DeletableColumnCount(); i < n; i++ { + for i, n := 0, tt.AllColumnCount(); i < n; i++ { if !pkOrdinals.Contains(i) { idx.addColumnByOrdinal(tt, i, tree.Ascending, nonKeyCol) } diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index 897c621fd433..ab17e75b9c62 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -596,18 +596,8 @@ func (tt *Table) IsVirtualTable() bool { return tt.IsVirtual } -// ColumnCount is part of the cat.Table interface. -func (tt *Table) ColumnCount() int { - return len(tt.Columns) - tt.writeOnlyColCount - tt.deleteOnlyColCount -} - -// WritableColumnCount is part of the cat.Table interface. -func (tt *Table) WritableColumnCount() int { - return len(tt.Columns) - tt.deleteOnlyColCount -} - -// DeletableColumnCount is part of the cat.Table interface. -func (tt *Table) DeletableColumnCount() int { +// AllColumnCount is part of the cat.Table interface. +func (tt *Table) AllColumnCount() int { return len(tt.Columns) } @@ -616,6 +606,20 @@ func (tt *Table) Column(i int) cat.Column { return tt.Columns[i] } +// ColumnKind is part of the cat.Table interface. +func (tt *Table) ColumnKind(i int) cat.ColumnKind { + writeOnlyEnd := len(tt.Columns) - tt.deleteOnlyColCount + standardEnd := writeOnlyEnd - tt.writeOnlyColCount + switch { + case i < standardEnd: + return cat.Ordinary + case i < writeOnlyEnd: + return cat.WriteOnly + default: + return cat.DeleteOnly + } +} + // IndexCount is part of the cat.Table interface. func (tt *Table) IndexCount() int { return len(tt.Indexes) - tt.writeOnlyIdxCount - tt.deleteOnlyIdxCount diff --git a/pkg/sql/opt/xform/memo_format.go b/pkg/sql/opt/xform/memo_format.go index 07cfdafa0d64..e808d59d2e8d 100644 --- a/pkg/sql/opt/xform/memo_format.go +++ b/pkg/sql/opt/xform/memo_format.go @@ -275,7 +275,7 @@ func (mf *memoFormatter) formatPrivate(e opt.Expr, physProps *physical.Required) switch t := e.(type) { case *memo.ScanExpr: tab := m.Metadata().Table(t.Table) - if tab.ColumnCount() != t.Cols.Len() { + if tab.AllColumnCount() != t.Cols.Len() { fmt.Fprintf(mf.buf, ",cols=%s", t.Cols) } if t.Constraint != nil { diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 25f0d71b2220..4b6f9ade8b27 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -565,8 +565,8 @@ func newOptTable( } // Create the table's column mapping from sqlbase.ColumnID to column ordinal. - ot.colMap = make(map[sqlbase.ColumnID]int, ot.DeletableColumnCount()) - for i, n := 0, ot.DeletableColumnCount(); i < n; i++ { + ot.colMap = make(map[sqlbase.ColumnID]int, ot.AllColumnCount()) + for i, n := 0, ot.AllColumnCount(); i < n; i++ { ot.colMap[sqlbase.ColumnID(ot.Column(i).ColID())] = i } @@ -634,15 +634,18 @@ func newOptTable( // Synthesize any check constraints for user defined types. var synthesizedChecks []cat.CheckConstraint - // TODO (rohany): We don't allow referencing columns in mutations in these - // expressions. However, it seems like we will need to have these checks - // operate on columns in mutations. Consider the following case: - // * a user adds a column with an enum type. - // * the column has a default expression of an enum that is not in the - // writeable state. - // * We will need a check constraint here to ensure that writes to the - // column are not successful, but we wouldn't be able to add that now. - for i := 0; i < ot.ColumnCount(); i++ { + for i := 0; i < ot.AllColumnCount(); i++ { + if cat.IsMutationColumn(ot, i) { + // TODO (rohany): We don't allow referencing columns in mutations in these + // expressions. However, it seems like we will need to have these checks + // operate on columns in mutations. Consider the following case: + // * a user adds a column with an enum type. + // * the column has a default expression of an enum that is not in the + // writeable state. + // * We will need a check constraint here to ensure that writes to the + // column are not successful, but we wouldn't be able to add that now. + continue + } col := ot.Column(i) colType := col.DatumType() if colType.UserDefined() { @@ -781,18 +784,8 @@ func (ot *optTable) IsVirtualTable() bool { return false } -// ColumnCount is part of the cat.Table interface. -func (ot *optTable) ColumnCount() int { - return len(ot.desc.Columns) -} - -// WritableColumnCount is part of the cat.Table interface. -func (ot *optTable) WritableColumnCount() int { - return len(ot.desc.WritableColumns()) -} - -// DeletableColumnCount is part of the cat.Table interface. -func (ot *optTable) DeletableColumnCount() int { +// AllColumnCount is part of the cat.Table interface. +func (ot *optTable) AllColumnCount() int { return len(ot.desc.DeletableColumns()) } @@ -801,6 +794,18 @@ func (ot *optTable) Column(i int) cat.Column { return &ot.desc.DeletableColumns()[i] } +// ColumnKind is part of the cat.Table interface. +func (ot *optTable) ColumnKind(i int) cat.ColumnKind { + switch { + case i < len(ot.desc.Columns): + return cat.Ordinary + case i < len(ot.desc.WritableColumns()): + return cat.WriteOnly + default: + return cat.DeleteOnly + } +} + // IndexCount is part of the cat.Table interface. func (ot *optTable) IndexCount() int { // Primary index is always present, so count is always >= 1. @@ -920,18 +925,18 @@ func (oi *optIndex) init( // Although the primary index contains all columns in the table, the index // descriptor does not contain columns that are not explicitly part of the // primary key. Retrieve those columns from the table descriptor. - oi.storedCols = make([]sqlbase.ColumnID, 0, tab.DeletableColumnCount()-len(desc.ColumnIDs)) + oi.storedCols = make([]sqlbase.ColumnID, 0, tab.AllColumnCount()-len(desc.ColumnIDs)) var pkCols util.FastIntSet for i := range desc.ColumnIDs { pkCols.Add(int(desc.ColumnIDs[i])) } - for i, n := 0, tab.DeletableColumnCount(); i < n; i++ { + for i, n := 0, tab.AllColumnCount(); i < n; i++ { id := tab.Column(i).ColID() if !pkCols.Contains(int(id)) { oi.storedCols = append(oi.storedCols, sqlbase.ColumnID(id)) } } - oi.numCols = tab.DeletableColumnCount() + oi.numCols = tab.AllColumnCount() } else { oi.storedCols = desc.StoreColumnIDs oi.numCols = len(desc.ColumnIDs) + len(desc.ExtraColumnIDs) + len(desc.StoreColumnIDs) @@ -1392,8 +1397,8 @@ func newOptVirtualTable( } // Create the table's column mapping from sqlbase.ColumnID to column ordinal. - ot.colMap = make(map[sqlbase.ColumnID]int, ot.DeletableColumnCount()) - for i, n := 0, ot.DeletableColumnCount(); i < n; i++ { + ot.colMap = make(map[sqlbase.ColumnID]int, ot.AllColumnCount()) + for i, n := 0, ot.AllColumnCount(); i < n; i++ { ot.colMap[sqlbase.ColumnID(ot.Column(i).ColID())] = i } @@ -1409,7 +1414,7 @@ func newOptVirtualTable( ot.indexes[0] = optVirtualIndex{ tab: ot, indexOrdinal: 0, - numCols: ot.ColumnCount(), + numCols: ot.AllColumnCount(), isPrimary: true, desc: &sqlbase.IndexDescriptor{ ID: 0, @@ -1429,7 +1434,7 @@ func newOptVirtualTable( desc: idxDesc, indexOrdinal: i + 1, // The virtual indexes don't return the bogus PK key? - numCols: ot.ColumnCount(), + numCols: ot.AllColumnCount(), } } @@ -1473,29 +1478,23 @@ func (ot *optVirtualTable) IsVirtualTable() bool { return true } -// ColumnCount is part of the cat.Table interface. -func (ot *optVirtualTable) ColumnCount() int { - // Virtual tables expose an extra (bogus) PK column. +// AllColumnCount is part of the cat.Table interface. +func (ot *optVirtualTable) AllColumnCount() int { return len(ot.desc.Columns) + 1 } -// WritableColumnCount is part of the cat.Table interface. -func (ot *optVirtualTable) WritableColumnCount() int { - return len(ot.desc.WritableColumns()) + 1 -} - -// DeletableColumnCount is part of the cat.Table interface. -func (ot *optVirtualTable) DeletableColumnCount() int { - return len(ot.desc.DeletableColumns()) + 1 -} - // Column is part of the cat.Table interface. func (ot *optVirtualTable) Column(i int) cat.Column { if i == 0 { // Column 0 is a dummy PK column. return optDummyVirtualPKColumn{} } - return &ot.desc.DeletableColumns()[i-1] + return &ot.desc.Columns[i-1] +} + +// ColumnKind is part of the cat.Table interface. +func (ot *optVirtualTable) ColumnKind(i int) cat.ColumnKind { + return cat.Ordinary } // IndexCount is part of the cat.Table interface. @@ -1803,7 +1802,7 @@ func (oi *optVirtualFamily) Name() tree.Name { // ColumnCount is part of the cat.Family interface. func (oi *optVirtualFamily) ColumnCount() int { - return oi.tab.ColumnCount() + return oi.tab.AllColumnCount() } // Column is part of the cat.Family interface. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 160f231a34cd..6e6395074e49 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1782,7 +1782,7 @@ func (rb *renderBuilder) setOutput(exprs tree.TypedExprs, columns sqlbase.Result // included if their ordinal position in the table schema is in the cols set. func makeColDescList(table cat.Table, cols exec.TableColumnOrdinalSet) []sqlbase.ColumnDescriptor { colDescs := make([]sqlbase.ColumnDescriptor, 0, cols.Len()) - for i, n := 0, table.DeletableColumnCount(); i < n; i++ { + for i, n := 0, table.AllColumnCount(); i < n; i++ { if !cols.Contains(i) { continue } diff --git a/pkg/sql/sqlbase/errors.go b/pkg/sql/sqlbase/errors.go index a3c61fe63b08..d4d7b4c69d36 100644 --- a/pkg/sql/sqlbase/errors.go +++ b/pkg/sql/sqlbase/errors.go @@ -133,7 +133,7 @@ func NewDatabaseAlreadyExistsError(name string) error { // when an error occurs while trying to get the colliding object for an // ObjectAlreadyExistsErr. func WrapErrorWhileConstructingObjectAlreadyExistsErr(err error) error { - return errors.Wrap(err, "object already exists") + return pgerror.WithCandidateCode(errors.Wrap(err, "object already exists"), pgcode.DuplicateObject) } // MakeObjectAlreadyExistsError creates an error for a namespace collision