Skip to content

Commit

Permalink
Merge #51400 #51406
Browse files Browse the repository at this point in the history
51400: opt, cat: cleaner API for mutation columns r=RaduBerinde a=RaduBerinde

**NOTE** I used `AllColumnCount` so that the compiler forces me to look at all callsites, and left it like this so they are also apparent in the diff. I intend to rename it to `ColumnCount` before merging.

The catalog currently relies on a particular ordering to separate
different kinds of columns (ordinary, write-only, delete-only). This
is unfortunate because it restricts the underlying implementation; in
particular, it makes it tricky to add a new kind of column (like the
`mvcc_timestamp` system column) without potentially breaking existing
code in subtle ways.

This change reworks the cat API to have a single `ColumnCount` method
and a separate `ColumnKind` method used to figure out the column kind.
This forces all code that cares about the kind to check it explicitly
rather than relying on a potentially faulty column count.

Fixes #51323.

Release note: None

51406: sql: ensure SQLSTATE 42710 on secondary errors r=rohany a=knz

Amends #51260

When creating an object that collides with another, we have to perform
another lookup to get the kind of object that it collided with.

If an error occurs during this second stage, we want the error to
propose SQLSTATE "Duplicate object" (42710) to the client if there is
no pgcode in the error object already.

Release note: None

Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Jul 15, 2020
3 parents 3428a4a + a6ad229 + b5db9ca commit 8c01a02
Show file tree
Hide file tree
Showing 24 changed files with 266 additions and 204 deletions.
18 changes: 17 additions & 1 deletion pkg/sql/opt/cat/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
}
24 changes: 9 additions & 15 deletions pkg/sql/opt/cat/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down
36 changes: 1 addition & 35 deletions pkg/sql/opt/cat/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand All @@ -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())
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
17 changes: 10 additions & 7 deletions pkg/sql/opt/memo/check_expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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"))
}
}
Expand All @@ -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:
Expand Down Expand Up @@ -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"))
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/opt/memo/logical_props_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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
Expand Down Expand Up @@ -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))
}

Expand Down Expand Up @@ -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))
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/multiplicity_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/multiplicity_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/statistics_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
Loading

0 comments on commit 8c01a02

Please sign in to comment.