diff --git a/pkg/sql/opt/catalog.go b/pkg/sql/opt/catalog.go index 7bb71ee96be1..50948b64e492 100644 --- a/pkg/sql/opt/catalog.go +++ b/pkg/sql/opt/catalog.go @@ -195,6 +195,10 @@ type Table interface { // position within the table, where i < ColumnCount. Column(i int) Column + // LookupColumnOrdinal returns the ordinal of the column with the given ID. A + // cache makes the lookup O(1). + LookupColumnOrdinal(colID uint32) (int, error) + // IndexCount returns the number of indexes defined on this table. This // includes the primary index, so the count is always >= 1. IndexCount() int @@ -219,6 +223,7 @@ type Catalog interface { // FindTable returns a Table interface for the database table matching the // given table name. Returns an error if the table does not exist. FindTable(ctx context.Context, name *tree.TableName) (Table, error) + FindTableByTableRef(ctx context.Context, tableID int64) (Table, error) } // FormatCatalogTable nicely formats a catalog table using a treeprinter for diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select b/pkg/sql/opt/exec/execbuilder/testdata/select index 9d1a97280b96..58629996db49 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select +++ b/pkg/sql/opt/exec/execbuilder/testdata/select @@ -1,5 +1,207 @@ # LogicTest: local-opt + +# ------------------------------------------------------------------------------ +# Numeric References Tests. +# These are put at the beginning of the file to ensure the numeric table +# reference is 53 (the numeric reference of the first table). +# If the numbering scheme in cockroach changes, this test will break. +# TODO(madhavsuresh): get the numeric reference ID in a less brittle fashion +# ------------------------------------------------------------------------------ +statement ok +CREATE TABLE num_ref (a INT PRIMARY KEY, xx INT, b INT, c INT, INDEX bc (b,c)) + +statement ok +ALTER TABLE num_ref RENAME COLUMN b TO d + +statement ok +ALTER TABLE num_ref RENAME COLUMN a TO p + +statement ok +ALTER TABLE num_ref DROP COLUMN xx + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53 AS num_ref_alias] +---- +scan · · (p, d, c) · +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4) AS num_ref_alias] +---- +scan · · (c) · +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(1,4) AS num_ref_alias] +---- +scan · · (p, c) · +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(1,3,4) AS num_ref_alias] +---- +scan · · (p, d, c) · +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4,3,1) AS num_ref_alias] +---- +render · · (c, d, p) · + │ render 0 c · · + │ render 1 d · · + │ render 2 p · · + └── scan · · (p, d, c) · +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4,3,1) AS num_ref_alias(col1,col2,col3)] +---- +render · · (col1, col2, col3) · + │ render 0 c · · + │ render 1 d · · + │ render 2 p · · + └── scan · · (p, d, c) · +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4,3,1) AS num_ref_alias]@bc +---- +scan · · (c, d, p) p!=NULL; weak-key(c,d,p) +· table num_ref@bc · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4) AS num_ref_alias]@bc +---- +render · · (c) · + │ render 0 num_ref_alias.c · · + └── scan · · (c, p[hidden,omitted], d[hidden,omitted]) p!=NULL; weak-key(c,p,d) +· table num_ref@bc · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(3) AS num_ref_alias]@bc +---- +render · · (d) · + │ render 0 num_ref_alias.d · · + └── scan · · (d, p[hidden,omitted], c[hidden,omitted]) p!=NULL; weak-key(d,p,c) +· table num_ref@bc · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(1) AS num_ref_alias]@bc +---- +render · · (p) p!=NULL + │ render 0 num_ref_alias.p · · + └── scan · · (p, d[hidden,omitted], c[hidden,omitted]) p!=NULL; weak-key(p,d,c) +· table num_ref@bc · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(1) AS num_ref_alias]@[1] +---- +render · · (p) p!=NULL; key(p) + │ render 0 num_ref_alias.p · · + └── scan · · (p, d[hidden,omitted], c[hidden,omitted]) p!=NULL; key(p) +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(1) AS num_ref_alias]@[2] +---- +render · · (p) p!=NULL + │ render 0 num_ref_alias.p · · + └── scan · · (p, d[hidden,omitted], c[hidden,omitted]) p!=NULL; weak-key(p,d,c) +· table num_ref@bc · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(3) AS num_ref_alias]@[1] +---- +render · · (d) · + │ render 0 num_ref_alias.d · · + └── scan · · (d, p[hidden,omitted], c[hidden,omitted]) p!=NULL; key(p) +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(3) AS num_ref_alias]@[2] +---- +render · · (d) · + │ render 0 num_ref_alias.d · · + └── scan · · (d, p[hidden,omitted], c[hidden,omitted]) p!=NULL; weak-key(d,p,c) +· table num_ref@bc · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4) AS num_ref_alias]@[1] +---- +render · · (c) · + │ render 0 num_ref_alias.c · · + └── scan · · (c, p[hidden,omitted], d[hidden,omitted]) p!=NULL; key(p) +· table num_ref@primary · · +· spans ALL · · + +query TTTTT +EXPLAIN (VERBOSE) SELECT * FROM [53(4) AS num_ref_alias]@[2] +---- +render · · (c) · + │ render 0 num_ref_alias.c · · + └── scan · · (c, p[hidden,omitted], d[hidden,omitted]) p!=NULL; weak-key(c,p,d) +· table num_ref@bc · · +· spans ALL · · + +query error pq: descriptor not found +EXPLAIN (VERBOSE) SELECT * FROM [666(1) AS num_ref_alias] + +query error pq: column \[666\] does not exist +EXPLAIN (VERBOSE) SELECT * FROM [53(666) AS num_ref_alias] + +query error pq: column \[2\] does not exist +EXPLAIN (VERBOSE) SELECT * FROM [53(2) AS num_ref_alias] + +statement ok +INSERT INTO num_ref VALUES (1, 10, 100), (2, 20, 200), (3, 30, 300) + +query error pq: cannot use "\*" without a FROM clause +SELECT * FROM [53() AS num_ref_alias] + +query III +SELECT * FROM [53(4,3,1) AS num_ref_alias] +---- +100 10 1 +200 20 2 +300 30 3 + +query I +SELECT * FROM [53(4) AS num_ref_alias]@[2] +---- +100 +200 +300 + +query I +SELECT * FROM [53(1) AS num_ref_alias]@[1] +---- +1 +2 +3 + +query III +SELECT * FROM [53(1,3,4) AS num_ref_alias(col1,col2,col3)] +---- +1 10 100 +2 20 200 +3 30 300 + + # ------------------------------------------------------------------------------ # Basic filter combinations. # ------------------------------------------------------------------------------ diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 3776b595caaf..eec4107030be 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -88,6 +88,12 @@ func (b *Builder) buildTable(texpr tree.TableExpr, inScope *scope) (outScope *sc outScope = b.buildStmt(source.Statement, inScope) return outScope + case *tree.TableRef: + tab := b.resolveTableRef(source) + outScope = b.buildScanWithTableRef(tab, tab.TabName(), inScope, source) + b.renameSource(source.As, outScope) + return outScope + default: panic(unimplementedf("not yet implemented: table expr: %T", texpr)) } @@ -139,6 +145,56 @@ func (b *Builder) renameSource(as tree.AliasClause, scope *scope) { } } +func (b *Builder) buildScanWithTableRef(tab opt.Table, tn *tree.TableName, inScope *scope, + ref *tree.TableRef) (outScope *scope) { + + tabName := tree.AsStringWithFlags(tn, b.FmtFlags) + tabID := b.factory.Metadata().AddTableWithName(tab, tabName) + + var colsToAdd []int + // See tree.TableRef: "Note that a nil [Columns] array means 'unspecified' (all columns)." + // whereas an array of length 0 means 'zero columns'. + if ref.Columns == nil { + for i := 0; i < tab.ColumnCount(); i++ { + colsToAdd = append(colsToAdd, i) + } + } else { + for _, c := range ref.Columns { + ordinalCol, error := tab.LookupColumnOrdinal(uint32(c)) + if error != nil { + panic(builderError{error}) + } + colsToAdd = append(colsToAdd, ordinalCol) + } + } + var tabCols opt.ColSet + outScope = inScope.push() + for _, i := range colsToAdd { + col := tab.Column(i) + colID := b.factory.Metadata().TableColumn(tabID, i) + name := tree.Name(col.ColName()) + colProps := scopeColumn{ + id: colID, + origName: name, + name: name, + table: *tn, + typ: col.DatumType(), + hidden: col.IsHidden(), + } + tabCols.Add(int(colID)) + b.colMap = append(b.colMap, colProps) + outScope.cols = append(outScope.cols, colProps) + } + if tab.IsVirtualTable() { + def := memo.VirtualScanOpDef{Table: tabID, Cols: tabCols} + outScope.group = b.factory.ConstructVirtualScan(b.factory.InternVirtualScanOpDef(&def)) + } else { + def := memo.ScanOpDef{Table: tabID, Cols: tabCols} + outScope.group = b.factory.ConstructScan(b.factory.InternScanOpDef(&def)) + } + return outScope +} + // buildScan builds a memo group for a ScanOp or VirtualScanOp expression on the // given table with the given table name. // @@ -162,7 +218,6 @@ func (b *Builder) buildScan(tab opt.Table, tn *tree.TableName, inScope *scope) ( typ: col.DatumType(), hidden: col.IsHidden(), } - tabCols.Add(int(colID)) b.colMap = append(b.colMap, colProps) outScope.cols = append(outScope.cols, colProps) diff --git a/pkg/sql/opt/optbuilder/testdata/select b/pkg/sql/opt/optbuilder/testdata/select index fdeffa510c4f..80686874c695 100644 --- a/pkg/sql/opt/optbuilder/testdata/select +++ b/pkg/sql/opt/optbuilder/testdata/select @@ -1094,3 +1094,70 @@ project ├── columns: x:1(int!null) └── scan a └── columns: x:1(int!null) y:2(float) + +exec-ddl +CREATE TABLE t (a INT PRIMARY KEY, xx INT, b INT, c INT, INDEX bc (b,c)) +---- +TABLE t + ├── a int not null + ├── xx int + ├── b int + ├── c int + ├── INDEX primary + │ └── a int not null + └── INDEX bc + ├── b int + ├── c int + └── a int not null + +# Numeric Reference Test (1/7) +build +SELECT * FROM [1 AS t] +---- +scan t + └── columns: a:1(int!null) xx:2(int) b:3(int) c:4(int) + +# Numeric Reference Test (2/7) +# These columns are off by one from what +# the logictests will run. This is due +# to the implementation of the test_catalog +build +SELECT * FROM [1(0) AS t] +---- +scan t + └── columns: a:1(int!null) + +# Numeric Reference Test (3/7) +build +SELECT * FROM [1(0,1) AS t] +---- +scan t + └── columns: a:1(int!null) xx:2(int) + +# Numeric Reference Test (4/7) +build +SELECT * FROM [1(3) AS t] +---- +scan t + └── columns: c:4(int) + +# Numeric Reference Test (5/7) +build +SELECT * FROM [1(4) AS t] +---- +error: column [4] does not exist + +# Numeric Reference Test (6/7) +build +SELECT * FROM [1(1,3) AS t] +---- +scan t + └── columns: xx:2(int) c:4(int) + +# Numeric Reference Test (7/7) +build +SELECT * FROM [1(1,2) AS t(col1,col2)] +---- +scan t + └── columns: col1:2(int) col2:3(int) + diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index ea41caa494dc..3e7cc8034288 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -342,6 +342,14 @@ func (b *Builder) assertNoAggregationOrWindowing(expr tree.Expr, op string) { } } +func (b *Builder) resolveTableRef(ref *tree.TableRef) opt.Table { + tab, err := b.catalog.FindTableByTableRef(b.ctx, ref.TableID) + if err != nil { + panic(builderError{err}) + } + return tab +} + // resolveTable returns the table in the catalog with the given name. func (b *Builder) resolveTable(tn *tree.TableName) opt.Table { tab, err := b.catalog.FindTable(b.ctx, tn) diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index 5d11c9d5c741..ccbf65e4381d 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util" ) @@ -104,12 +105,15 @@ func (tc *Catalog) CreateTable(stmt *tree.CreateTable) *Table { case *tree.IndexTableDef: tab.addIndex(def, nonUniqueIndex) - } + // TODO(rytaft): In the future we will likely want to check for unique // constraints, indexes, and foreign key constraints to determine // nullability, uniqueness, etc. } + // We need to keep track of the tableID from + // numeric references + tab.tableID = sqlbase.ID(len(tc.tables)) // Add the new table to the catalog. tc.AddTable(tab) diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index 9bba4e0910ef..536cc724b78f 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/util/treeprinter" ) @@ -44,6 +45,15 @@ const ( testDB = "t" ) +func (tc *Catalog) FindTableByTableRef(ctx context.Context, tableID int64) (opt.Table, error) { + for _, table := range tc.tables { + if int64(table.tableID) == tableID { + return table, nil + } + } + return nil, fmt.Errorf("table %d not found", tableID) +} + // FindTable is part of the opt.Catalog interface. func (tc *Catalog) FindTable(ctx context.Context, name *tree.TableName) (opt.Table, error) { // This is a simplified version of tree.TableName.ResolveExisting() from @@ -170,6 +180,7 @@ func (tc *Catalog) qualifyTableName(name *tree.TableName) { // Table implements the opt.Table interface for testing purposes. type Table struct { Name tree.TableName + tableID sqlbase.ID Columns []*Column Indexes []*Index Stats TableStats @@ -204,6 +215,14 @@ func (tt *Table) Column(i int) opt.Column { return tt.Columns[i] } +// LookupColumnOrdinal is part of the opt.Table interface. +func (tt *Table) LookupColumnOrdinal(colID uint32) (int, error) { + if int(colID) >= len(tt.Columns) { + return int(colID), error(fmt.Errorf("column [%d] does not exist", colID)) + } + return int(colID), nil +} + // IndexCount is part of the opt.Table interface. func (tt *Table) IndexCount() int { return len(tt.Indexes) diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 855c1dc62f80..021d90d0f5d3 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -18,6 +18,8 @@ import ( "context" "time" + "fmt" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -49,6 +51,29 @@ func (oc *optCatalog) init(statsCache *stats.TableStatisticsCache, resolver Logi oc.statsCache = statsCache } +func (oc *optCatalog) FindTableByTableRef(ctx context.Context, tableID int64) (opt.Table, error) { + // TODO(madhavsuresh): this does not completely mirror planner.getTableDescByID. + // It does not use the descriptorCache + desc, err := sqlbase.GetTableDescFromID(ctx, oc.resolver.Txn(), sqlbase.ID(tableID)) + if err != nil { + return nil, err + } + + dbDesc, err := sqlbase.GetDatabaseDescFromID(ctx, oc.resolver.Txn(), desc.ParentID) + + if oc.wrappers == nil { + oc.wrappers = make(map[*sqlbase.TableDescriptor]*optTable) + } + wrapper, ok := oc.wrappers[desc] + if !ok { + // Is this a normalized table name? + tbName := tree.MakeTableName(tree.Name(dbDesc.Name), tree.Name(desc.Name)) + wrapper = newOptTable(&tbName, oc.statsCache, desc) + oc.wrappers[desc] = wrapper + } + return wrapper, nil +} + // FindTable is part of the opt.Catalog interface. func (oc *optCatalog) FindTable(ctx context.Context, name *tree.TableName) (opt.Table, error) { desc, err := ResolveExistingObject(ctx, oc.resolver, name, true /*required*/, requireTableDesc) @@ -212,11 +237,25 @@ func (ot *optTable) ensureColMap() { } } +// LookupColumnOrdinal exposes optTable.lookupColumnOrdinal. +// In order to preserve the argument type information +// on lookupColumnOrdinal, this wrapper function exists. +// colID as the sqlbase.ColumnID type would result in a +// circular dependency for catalog.Table - the interface this +// implements. +func (ot *optTable) LookupColumnOrdinal(colID uint32) (int, error) { + return ot.lookupColumnOrdinal(sqlbase.ColumnID(colID)) +} + // lookupColumnOrdinal returns the ordinal of the column with the given ID. A // cache makes the lookup O(1). -func (ot *optTable) lookupColumnOrdinal(colID sqlbase.ColumnID) int { +func (ot *optTable) lookupColumnOrdinal(colID sqlbase.ColumnID) (int, error) { ot.ensureColMap() - return ot.colMap[colID] + col, ok := ot.colMap[colID] + if ok { + return col, nil + } + return col, error(fmt.Errorf("column [%d] does not exist", colID)) } // optIndex is a wrapper around sqlbase.IndexDescriptor that caches some @@ -266,7 +305,7 @@ func (oi *optIndex) init(tab *optTable, desc *sqlbase.IndexDescriptor) { if desc.Unique { notNull := true for _, id := range desc.ColumnIDs { - ord := tab.lookupColumnOrdinal(id) + ord, _ := tab.lookupColumnOrdinal(id) if tab.desc.Columns[ord].Nullable { notNull = false break @@ -323,7 +362,7 @@ func (oi *optIndex) LaxKeyColumnCount() int { func (oi *optIndex) Column(i int) opt.IndexColumn { length := len(oi.desc.ColumnIDs) if i < length { - ord := oi.tab.lookupColumnOrdinal(oi.desc.ColumnIDs[i]) + ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.ColumnIDs[i]) return opt.IndexColumn{ Column: oi.tab.Column(ord), Ordinal: ord, @@ -334,12 +373,12 @@ func (oi *optIndex) Column(i int) opt.IndexColumn { i -= length length = len(oi.desc.ExtraColumnIDs) if i < length { - ord := oi.tab.lookupColumnOrdinal(oi.desc.ExtraColumnIDs[i]) + ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.ExtraColumnIDs[i]) return opt.IndexColumn{Column: oi.tab.Column(ord), Ordinal: ord} } i -= length - ord := oi.tab.lookupColumnOrdinal(oi.storedCols[i]) + ord, _ := oi.tab.lookupColumnOrdinal(oi.storedCols[i]) return opt.IndexColumn{Column: oi.tab.Column(ord), Ordinal: ord} } diff --git a/pkg/sql/table_ref_test.go b/pkg/sql/table_ref_test.go index 71c9e69b15d0..f020fa5c173f 100644 --- a/pkg/sql/table_ref_test.go +++ b/pkg/sql/table_ref_test.go @@ -33,7 +33,10 @@ func TestTableRefs(t *testing.T) { defer s.Stopper().Stop(context.TODO()) // Populate the test database. + // This test path should not run with the optimizer. + // The error paths are different. stmt := ` +SET experimental_opt='off'; CREATE DATABASE test; CREATE TABLE test.t(a INT PRIMARY KEY, xx INT, b INT, c INT); CREATE INDEX bc ON test.t(b, c);