From 1ee998720a66530a39c6a95e5186eba6c2bb6018 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Wed, 15 Apr 2020 22:02:43 -0400 Subject: [PATCH 1/3] sql: support virtual tables in LookupTableByID --- pkg/sql/planner.go | 3 +++ pkg/sql/virtual_schema.go | 16 +++++++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 86fbcbc4d306..c9e36ed1b50b 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -450,6 +450,9 @@ func (p *planner) ResolveTableName(ctx context.Context, tn *tree.TableName) (tre // CommonLookupFlags, it could use or skip the TableCollection cache. See // TableCollection.getTableVersionByID for how it's used. func (p *planner) LookupTableByID(ctx context.Context, tableID sqlbase.ID) (row.TableEntry, error) { + if entry, err := p.getVirtualTabler().getVirtualTableEntryByID(tableID); err == nil { + return row.TableEntry{Desc: sqlbase.NewImmutableTableDescriptor(*entry.desc)}, nil + } flags := tree.ObjectLookupFlags{CommonLookupFlags: tree.CommonLookupFlags{AvoidCached: p.avoidCachedDescriptors}} table, err := p.Tables().getTableVersionByID(ctx, p.txn, tableID, flags) if err != nil { diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index c7abc92bd69d..c410f1765a2c 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -12,6 +12,7 @@ package sql import ( "context" + "math" "sort" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -196,6 +197,7 @@ var virtualSchemas = map[sqlbase.ID]virtualSchema{ // on an Executor. type VirtualSchemaHolder struct { entries map[string]virtualSchemaEntry + defsByID map[sqlbase.ID]*virtualDefEntry orderedNames []string } @@ -311,6 +313,7 @@ func NewVirtualSchemaHolder( vs := &VirtualSchemaHolder{ entries: make(map[string]virtualSchemaEntry, len(virtualSchemas)), orderedNames: make([]string, len(virtualSchemas)), + defsByID: make(map[sqlbase.ID]*virtualDefEntry, math.MaxUint32-sqlbase.MinVirtualID), } order := 0 @@ -334,12 +337,14 @@ func NewVirtualSchemaHolder( } } - defs[tableDesc.Name] = virtualDefEntry{ + entry := virtualDefEntry{ virtualDef: def, desc: &tableDesc, validWithNoDatabaseContext: schema.validWithNoDatabaseContext, comment: def.getComment(), } + defs[tableDesc.Name] = entry + vs.defsByID[tableDesc.ID] = &entry orderedDefNames = append(orderedDefNames, tableDesc.Name) } @@ -410,11 +415,20 @@ func (vs *VirtualSchemaHolder) getVirtualTableEntry(tn *tree.TableName) (virtual return virtualDefEntry{}, nil } +func (vs *VirtualSchemaHolder) getVirtualTableEntryByID(id sqlbase.ID) (virtualDefEntry, error) { + entry, ok := vs.defsByID[id] + if !ok { + return virtualDefEntry{}, sqlbase.ErrDescriptorNotFound + } + return *entry, nil +} + // VirtualTabler is used to fetch descriptors for virtual tables and databases. type VirtualTabler interface { getVirtualTableDesc(tn *tree.TableName) (*sqlbase.TableDescriptor, error) getVirtualSchemaEntry(name string) (virtualSchemaEntry, bool) getVirtualTableEntry(tn *tree.TableName) (virtualDefEntry, error) + getVirtualTableEntryByID(id sqlbase.ID) (virtualDefEntry, error) getEntries() map[string]virtualSchemaEntry getSchemaNames() []string } From 2f7bc1c1015c869b5f7222c094a9f33e409e005b Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 10 Apr 2020 00:03:00 -0400 Subject: [PATCH 2/3] sql: add virtual table pushdown; some examples This commit adds virtual table "indexes", a concept that allows virtual tables to specify columns that can be accelerated given an equality constraint using custom code. For example, the information_schema.tables table contains one entry per table in the database. Before this commit, retrieving the row for a table 'foo' would require fetching the descriptors for all tables and filtering for the one that had name 'foo'. Now, if there's a constraint on the table name, an "index" is used that converts the name into a descriptor with a single lookup. This commit also adds virtual indexes to a few commonly-used virtual tables. There is much, much more that could be done - the ones that are added here are just examples. Release note (performance improvement): filtered scans over virtual tables have improved performance in common cases. --- pkg/sql/crdb_internal.go | 7 +- pkg/sql/delayed.go | 10 +- pkg/sql/information_schema.go | 87 +- .../testdata/logic_test/information_schema | 3 +- .../logictest/testdata/logic_test/pg_catalog | 16 + .../opt/exec/execbuilder/testdata/dist_union | 2 +- pkg/sql/opt/exec/execbuilder/testdata/explain | 91 +- .../execbuilder/testdata/information_schema | 19 +- pkg/sql/opt/exec/execbuilder/testdata/join | 268 +++--- pkg/sql/opt/exec/execbuilder/testdata/scalar | 11 +- pkg/sql/opt/exec/execbuilder/testdata/union | 2 +- pkg/sql/opt/exec/execbuilder/testdata/virtual | 57 ++ pkg/sql/opt/optbuilder/testdata/virtual-scan | 7 + pkg/sql/opt/xform/coster.go | 2 +- pkg/sql/opt/xform/custom_funcs.go | 3 + pkg/sql/opt_catalog.go | 140 ++- pkg/sql/opt_exec_factory.go | 22 +- pkg/sql/pg_catalog.go | 851 ++++++++++-------- pkg/sql/pg_oid_test.go | 2 +- pkg/sql/resolver.go | 13 +- pkg/sql/schema/schema.go | 16 +- pkg/sql/sessiondata/search_path.go | 6 + pkg/sql/show_create.go | 2 +- pkg/sql/show_create_clauses.go | 6 +- pkg/sql/sqlbase/errors.go | 5 + pkg/sql/sqlbase/structured.go | 9 +- pkg/sql/virtual_schema.go | 219 ++++- pkg/sql/virtual_table.go | 2 +- pkg/sql/vtable/information_schema.go | 3 +- pkg/sql/walk.go | 3 + 30 files changed, 1213 insertions(+), 671 deletions(-) create mode 100644 pkg/sql/opt/exec/execbuilder/testdata/virtual diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 2f3ff930bf1c..3c4e725d8be8 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -49,7 +50,7 @@ import ( "gopkg.in/yaml.v2" ) -const crdbInternalName = "crdb_internal" +const crdbInternalName = sessiondata.CRDBInternalSchemaName // Naming convention: // - if the response is served from memory, prefix with node_ @@ -1574,7 +1575,7 @@ func showAlterStatementWithInterleave( var err error var parentName tree.TableName if lCtx != nil { - parentName, err = lCtx.getParentAsTableName(parentTableID, contextName) + parentName, err = getParentAsTableName(lCtx, parentTableID, contextName) if err != nil { return err } @@ -1586,7 +1587,7 @@ func showAlterStatementWithInterleave( var tableName tree.TableName if lCtx != nil { - tableName, err = lCtx.getTableAsTableName(table, contextName) + tableName, err = getTableAsTableName(lCtx, table, contextName) if err != nil { return err } diff --git a/pkg/sql/delayed.go b/pkg/sql/delayed.go index 061666116608..f5ffb860b273 100644 --- a/pkg/sql/delayed.go +++ b/pkg/sql/delayed.go @@ -13,6 +13,7 @@ package sql import ( "context" + "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" ) @@ -21,10 +22,11 @@ import ( // constructor must be delayed during query execution (as opposed to // SQL prepare) for resource tracking purposes. type delayedNode struct { - name string - columns sqlbase.ResultColumns - constructor nodeConstructor - plan planNode + name string + columns sqlbase.ResultColumns + indexConstraint *constraint.Constraint + constructor nodeConstructor + plan planNode } type nodeConstructor func(context.Context, *planner) (planNode, error) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 19593201e15d..692ff5ede883 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/schema" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -29,8 +30,7 @@ import ( ) const ( - informationSchemaName = "information_schema" - pgCatalogName = sessiondata.PgCatalogName + pgCatalogName = sessiondata.PgCatalogName ) var pgCatalogNameDString = tree.NewDString(pgCatalogName) @@ -38,7 +38,7 @@ var pgCatalogNameDString = tree.NewDString(pgCatalogName) // informationSchema lists all the table definitions for // information_schema. var informationSchema = virtualSchema{ - name: informationSchemaName, + name: sessiondata.InformationSchemaName, allTableNames: buildStringSet( // Generated with: // select distinct '"'||table_name||'",' from information_schema.tables @@ -316,7 +316,7 @@ https://www.postgresql.org/docs/9.5/infoschema-check-constraints.html`, // uses the format ___not_null. // We might as well do the same. conNameStr := tree.NewDString(fmt.Sprintf( - "%s_%s_%d_not_null", h.NamespaceOid(db, scName), defaultOid(table.ID), colNum, + "%s_%s_%d_not_null", h.NamespaceOid(db, scName), tableOid(table.ID), colNum, )) chkExprStr := tree.NewDString(fmt.Sprintf( "%s IS NOT NULL", column.Name, @@ -1237,7 +1237,7 @@ CREATE TABLE information_schema.table_constraints ( colNum++ // NOT NULL column constraints are implemented as a CHECK in postgres. conNameStr := tree.NewDString(fmt.Sprintf( - "%s_%s_%d_not_null", h.NamespaceOid(db, scName), defaultOid(table.ID), colNum, + "%s_%s_%d_not_null", h.NamespaceOid(db, scName), tableOid(table.ID), colNum, )) if !col.Nullable { if err := addRow( @@ -1356,37 +1356,62 @@ var informationSchemaTablesTable = virtualSchemaTable{ https://www.postgresql.org/docs/9.5/infoschema-tables.html`, schema: vtable.InformationSchemaTables, populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { - return forEachTableDesc(ctx, p, dbContext, virtualMany, - func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error { - if table.IsSequence() { - return nil + return forEachTableDesc(ctx, p, dbContext, virtualMany, addTablesTableRow(addRow)) + }, + indexes: []virtualIndex{ + { + populate: func(ctx context.Context, constraint tree.Datum, p *planner, db *DatabaseDescriptor, + addRow func(...tree.Datum) error) (bool, error) { + // This index is on the TABLE_NAME column. + name := tree.MustBeDString(constraint) + desc, err := ResolveExistingObject(ctx, p, tree.NewUnqualifiedTableName(tree.Name(name)), + tree.ObjectLookupFlags{}, ResolveAnyDescType) + if err != nil || desc == nil { + return false, err } - tableType := tableTypeBaseTable - insertable := yesString - if table.IsVirtualTable() { - tableType = tableTypeSystemView - insertable = noString - } else if table.IsView() { - tableType = tableTypeView - insertable = noString - } else if table.Temporary { - tableType = tableTypeTemporary + schemaName, err := schema.ResolveNameByID(ctx, p.txn, db.ID, desc.GetParentSchemaID()) + if err != nil { + return false, err } - dbNameStr := tree.NewDString(db.Name) - scNameStr := tree.NewDString(scName) - tbNameStr := tree.NewDString(table.Name) - return addRow( - dbNameStr, // table_catalog - scNameStr, // table_schema - tbNameStr, // table_name - tableType, // table_type - insertable, // is_insertable_into - tree.NewDInt(tree.DInt(table.Version)), // version - ) - }) + return true, addTablesTableRow(addRow)(db, schemaName, desc.TableDesc()) + }, + }, }, } +func addTablesTableRow( + addRow func(...tree.Datum) error, +) func(db *sqlbase.DatabaseDescriptor, scName string, + table *sqlbase.TableDescriptor) error { + return func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error { + if table.IsSequence() { + return nil + } + tableType := tableTypeBaseTable + insertable := yesString + if table.IsVirtualTable() { + tableType = tableTypeSystemView + insertable = noString + } else if table.IsView() { + tableType = tableTypeView + insertable = noString + } else if table.Temporary { + tableType = tableTypeTemporary + } + dbNameStr := tree.NewDString(db.Name) + scNameStr := tree.NewDString(scName) + tbNameStr := tree.NewDString(table.Name) + return addRow( + dbNameStr, // table_catalog + scNameStr, // table_schema + tbNameStr, // table_name + tableType, // table_type + insertable, // is_insertable_into + tree.NewDInt(tree.DInt(table.Version)), // version + ) + } +} + // Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-views.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/views-table.html var informationSchemaViewsTable = virtualSchemaTable{ diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 097e0ab1c1e4..a6adaf3e0914 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -153,7 +153,8 @@ information_schema.tables CREATE TABLE tables ( table_name STRING NOT NULL, table_type STRING NOT NULL, is_insertable_into STRING NOT NULL, - version INT8 NULL + version INT8 NULL, + INDEX tables_table_name_idx (table_name ASC) STORING (table_catalog, table_schema, table_type, is_insertable_into, version) ) query TTBTTTB colnames diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 98e0a368d12e..cbe1ed38eb74 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -2193,3 +2193,19 @@ query TTTTTT select * from pg_indexes where indexname = 'regression_46450_idx' ---- 3047862654 public regression_46450 regression_46450_idx NULL CREATE INDEX regression_46450_idx ON test.public.regression_46450 USING gin (json ASC) + +# Make sure that selecting from vtables with indexes in other dbs properly +# hides descriptors that should be hidden. + +statement ok +CREATE TABLE hidden_in_vtable_index_test(a int) + +let $testid +SELECT oid FROM pg_class WHERE relname='hidden_in_vtable_index_test' + +statement ok +CREATE DATABASE other_db; SET DATABASE = other_db + +query O +SELECT oid FROM pg_class WHERE oid=$testid +---- diff --git a/pkg/sql/opt/exec/execbuilder/testdata/dist_union b/pkg/sql/opt/exec/execbuilder/testdata/dist_union index 0c28f0abff72..00779f4e3248 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/dist_union +++ b/pkg/sql/opt/exec/execbuilder/testdata/dist_union @@ -43,7 +43,7 @@ union · · │ size 1 column, 1 row └── render · · └── virtual table · · -· source · +· source node_build_info@primary statement ok CREATE TABLE abc (a INT, b INT, c INT) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index fbd06b27ddbe..2f23efc9315c 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -131,7 +131,7 @@ render · · │ filter ((job_type IS NULL) OR (job_type != 'AUTO CREATE STATS')) AND ((finished IS NULL) OR (finished > (now() - '12:00:00'))) └── render · · └── virtual table · · -· source · +· source jobs@primary statement ok CREATE INDEX a ON foo(x) @@ -178,7 +178,7 @@ sort · · │ distinct on database_name └── render · · └── virtual table · · -· source · +· source schemata@primary query TTT SELECT * FROM [EXPLAIN SHOW TABLES] WHERE field != 'size' @@ -196,12 +196,12 @@ render · · │ │ filter nspname NOT IN ('crdb_internal', 'information_schema', 'pg_catalog') │ └── render · · │ └── virtual table · · - │ source · + │ source pg_namespace@primary └── filter · · │ filter relkind IN ('S', 'r', 'v') └── render · · └── virtual table · · -· source · +· source pg_class@primary query TTT SELECT * FROM [EXPLAIN SHOW TABLES WITH COMMENT] WHERE field != 'size' @@ -222,17 +222,17 @@ render · · │ │ │ filter nspname NOT IN ('crdb_internal', 'information_schema', 'pg_catalog') │ │ └── render · · │ │ └── virtual table · · - │ │ source · + │ │ source pg_namespace@primary │ └── filter · · │ │ filter relkind IN ('S', 'r', 'v') │ └── render · · │ └── virtual table · · - │ source · + │ source pg_class@primary └── filter · · │ filter objsubid = 0 └── render · · └── virtual table · · -· source · +· source pg_description@primary query TTT SELECT * FROM [EXPLAIN SHOW DATABASE] WHERE field != 'size' @@ -244,7 +244,7 @@ render · · │ filter variable = 'database' └── render · · └── virtual table · · -· source · +· source session_variables@primary query TTT SELECT * FROM [EXPLAIN SHOW TIME ZONE] WHERE field != 'size' @@ -256,7 +256,7 @@ render · · │ filter variable = 'timezone' └── render · · └── virtual table · · -· source · +· source session_variables@primary query TTT SELECT * FROM [EXPLAIN SHOW DEFAULT_TRANSACTION_ISOLATION] WHERE field != 'size' @@ -268,7 +268,7 @@ render · · │ filter variable = 'default_transaction_isolation' └── render · · └── virtual table · · -· source · +· source session_variables@primary query TTT SELECT * FROM [EXPLAIN SHOW DEFAULT_TRANSACTION_PRIORITY] WHERE field != 'size' @@ -280,7 +280,7 @@ render · · │ filter variable = 'default_transaction_priority' └── render · · └── virtual table · · -· source · +· source session_variables@primary query TTT SELECT * FROM [EXPLAIN SHOW TRANSACTION ISOLATION LEVEL] WHERE field != 'size' @@ -292,7 +292,7 @@ render · · │ filter variable = 'transaction_isolation' └── render · · └── virtual table · · -· source · +· source session_variables@primary query TTT SELECT * FROM [EXPLAIN SHOW TRANSACTION PRIORITY] WHERE field != 'size' @@ -304,7 +304,7 @@ render · · │ filter variable = 'transaction_priority' └── render · · └── virtual table · · -· source · +· source session_variables@primary query TTT EXPLAIN SHOW COLUMNS FROM foo @@ -333,12 +333,12 @@ render · · │ │ filter ((table_catalog = 'test') AND (table_schema = 'public')) AND (table_name = 'foo') │ └── render · · │ └── virtual table · · - │ source · + │ source columns@primary └── filter · · │ filter ((table_catalog = 'test') AND (table_schema = 'public')) AND (table_name = 'foo') └── render · · └── virtual table · · -· source · +· source statistics@primary query TTT SELECT * FROM [EXPLAIN SHOW GRANTS ON foo] WHERE field != 'size' @@ -352,7 +352,7 @@ render · · │ filter (table_catalog, table_schema, table_name) IN (('test', 'public', 'foo'),) └── render · · └── virtual table · · -· source · +· source table_privileges@primary query TTT EXPLAIN SHOW INDEX FROM foo @@ -364,36 +364,43 @@ render · · │ filter ((table_catalog = 'test') AND (table_schema = 'public')) AND (table_name = 'foo') └── render · · └── virtual table · · -· source · +· source statistics@primary query TTT EXPLAIN SHOW CONSTRAINTS FROM foo ---- -· distributed false -· vectorized false -render · · - └── sort · · - │ order +conname - └── render · · - └── hash-join · · - │ type inner - │ equality (conrelid) = (oid) - ├── render · · - │ └── virtual table · · - │ source · - └── hash-join · · - │ type inner - │ equality (oid) = (relnamespace) - ├── filter · · - │ │ filter nspname = 'public' - │ └── render · · - │ └── virtual table · · - │ source · - └── filter · · - │ filter relname = 'foo' - └── render · · - └── virtual table · · -· source · +· distributed false +· vectorized false +render · · + └── sort · · + │ order +conname + └── render · · + └── hash-join · · + │ type inner + │ equality (oid) = (relnamespace) + ├── filter · · + │ │ filter nspname = 'public' + │ └── render · · + │ └── virtual table · · + │ source pg_namespace@primary + └── merge-join · · + │ type inner + │ equality (conrelid) = (oid) + │ left cols are key · + │ right cols are key · + │ mergeJoinOrder +"(conrelid=oid)" + ├── sort · · + │ │ order +conrelid + │ └── render · · + │ └── virtual table · · + │ source pg_constraint@pg_constraint_conrelid_idx + └── filter · · + │ filter relname = 'foo' + └── sort · · + │ order +oid + └── render · · + └── virtual table · · +· source pg_class@pg_class_oid_idx query TTT EXPLAIN SHOW USERS diff --git a/pkg/sql/opt/exec/execbuilder/testdata/information_schema b/pkg/sql/opt/exec/execbuilder/testdata/information_schema index 16d8a2cb458c..d730dd759bd9 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/information_schema +++ b/pkg/sql/opt/exec/execbuilder/testdata/information_schema @@ -3,20 +3,19 @@ query TTTTT EXPLAIN (VERBOSE) SELECT * FROM system.information_schema.schemata ---- -· distributed false · · -· vectorized false · · -virtual table · · (catalog_name, schema_name, default_character_set_name, sql_path) · -· source · · · +· distributed false · · +· vectorized false · · +virtual table · · (catalog_name, schema_name, default_character_set_name, sql_path) · +· source schemata@primary · · query TTT EXPLAIN SELECT * FROM system.information_schema.tables WHERE table_name='foo' ---- -· distributed false -· vectorized false -filter · · - │ filter table_name = 'foo' - └── virtual table · · -· source · +· distributed false +· vectorized false +virtual table · · +· source tables@tables_table_name_idx +· constraint /4: [/'foo' - /'foo'] statement error use of crdb_internal_vtable_pk column not allowed SELECT crdb_internal_vtable_pk FROM system.information_schema.schemata diff --git a/pkg/sql/opt/exec/execbuilder/testdata/join b/pkg/sql/opt/exec/execbuilder/testdata/join index 4856195a8031..07487d3fbe01 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/join +++ b/pkg/sql/opt/exec/execbuilder/testdata/join @@ -312,135 +312,145 @@ SELECT level, node_type, field, description FROM [EXPLAIN (VERBOSE) SELECT pos.n ] WHERE node_type <> 'values' AND field <> 'size' ---- -0 · distributed false -0 · vectorized false -0 render · · -0 · render 0 pktable_cat -0 · render 1 nspname -0 · render 2 relname -0 · render 3 attname -0 · render 4 pktable_cat -0 · render 5 nspname -0 · render 6 relname -0 · render 7 attname -0 · render 8 generate_series -0 · render 9 update_rule -0 · render 10 delete_rule -0 · render 11 conname -0 · render 12 relname -0 · render 13 deferrability -1 sort · · -1 · order +nspname,+relname,+conname,+generate_series -2 render · · -2 · render 0 CAST(NULL AS STRING) -2 · render 1 CASE confupdtype WHEN 'c' THEN 0 WHEN 'n' THEN 2 WHEN 'd' THEN 4 WHEN 'r' THEN 1 WHEN 'a' THEN 3 END -2 · render 2 CASE confdeltype WHEN 'c' THEN 0 WHEN 'n' THEN 2 WHEN 'd' THEN 4 WHEN 'r' THEN 1 WHEN 'a' THEN 3 END -2 · render 3 CASE WHEN condeferrable AND condeferred THEN 5 WHEN condeferrable THEN 6 ELSE 7 END -2 · render 4 nspname -2 · render 5 relname -2 · render 6 attname -2 · render 7 nspname -2 · render 8 relname -2 · render 9 attname -2 · render 10 conname -2 · render 11 generate_series -2 · render 12 relname -3 hash-join · · -3 · type inner -3 · equality (refobjid) = (oid) -4 hash-join · · -4 · type inner -4 · equality (oid) = (objid) -5 cross-join · · -5 · type inner -5 · pred (attnum = confkey[generate_series]) AND (attnum = conkey[generate_series]) -6 hash-join · · -6 · type inner -6 · equality (oid, oid) = (confrelid, conrelid) -7 hash-join · · -7 · type inner -7 · equality (oid) = (attrelid) -8 hash-join · · -8 · type inner -8 · equality (oid) = (relnamespace) -9 hash-join · · -9 · type inner -9 · equality (attrelid) = (oid) -10 cross-join · · -10 · type cross -11 render · · -11 · render 0 attrelid -11 · render 1 attname -11 · render 2 attnum -12 virtual table · · -12 · source · -11 filter · · -11 · filter nspname = 'public' -12 render · · -12 · render 0 oid -12 · render 1 nspname -13 virtual table · · -13 · source · -10 hash-join · · -10 · type inner -10 · equality (oid) = (relnamespace) -11 render · · -11 · render 0 oid -11 · render 1 nspname -12 virtual table · · -12 · source · -11 render · · -11 · render 0 oid -11 · render 1 relname -11 · render 2 relnamespace -12 virtual table · · -12 · source · -9 filter · · -9 · filter relname = 'orders' -10 render · · -10 · render 0 oid -10 · render 1 relname -10 · render 2 relnamespace -11 virtual table · · -11 · source · -8 render · · -8 · render 0 attrelid -8 · render 1 attname -8 · render 2 attnum -9 virtual table · · -9 · source · -7 filter · · -7 · filter contype = 'f' -8 render · · -8 · render 0 oid -8 · render 1 conname -8 · render 2 contype -8 · render 3 condeferrable -8 · render 4 condeferred -8 · render 5 conrelid -8 · render 6 confrelid -8 · render 7 confupdtype -8 · render 8 confdeltype -8 · render 9 conkey -8 · render 10 confkey -9 virtual table · · -9 · source · -6 project set · · -6 · render 0 generate_series(1, 32) -7 emptyrow · · -5 render · · -5 · render 0 objid -5 · render 1 refobjid -6 virtual table · · -6 · source · -4 filter · · -4 · filter relkind = 'i' -5 render · · -5 · render 0 oid -5 · render 1 relname -5 · render 2 relkind -6 virtual table · · -6 · source · +0 · distributed false +0 · vectorized false +0 render · · +0 · render 0 pktable_cat +0 · render 1 nspname +0 · render 2 relname +0 · render 3 attname +0 · render 4 pktable_cat +0 · render 5 nspname +0 · render 6 relname +0 · render 7 attname +0 · render 8 generate_series +0 · render 9 update_rule +0 · render 10 delete_rule +0 · render 11 conname +0 · render 12 relname +0 · render 13 deferrability +1 sort · · +1 · order +nspname,+relname,+conname,+generate_series +2 render · · +2 · render 0 CAST(NULL AS STRING) +2 · render 1 CASE confupdtype WHEN 'c' THEN 0 WHEN 'n' THEN 2 WHEN 'd' THEN 4 WHEN 'r' THEN 1 WHEN 'a' THEN 3 END +2 · render 2 CASE confdeltype WHEN 'c' THEN 0 WHEN 'n' THEN 2 WHEN 'd' THEN 4 WHEN 'r' THEN 1 WHEN 'a' THEN 3 END +2 · render 3 CASE WHEN condeferrable AND condeferred THEN 5 WHEN condeferrable THEN 6 ELSE 7 END +2 · render 4 nspname +2 · render 5 relname +2 · render 6 attname +2 · render 7 nspname +2 · render 8 relname +2 · render 9 attname +2 · render 10 conname +2 · render 11 generate_series +2 · render 12 relname +3 hash-join · · +3 · type inner +3 · equality (refobjid) = (oid) +3 · right cols are key · +4 hash-join · · +4 · type inner +4 · equality (objid) = (oid) +5 render · · +5 · render 0 objid +5 · render 1 refobjid +6 virtual table · · +6 · source pg_depend@primary +5 cross-join · · +5 · type inner +5 · pred (attnum = confkey[generate_series]) AND (attnum = conkey[generate_series]) +6 hash-join · · +6 · type inner +6 · equality (oid, oid) = (confrelid, conrelid) +6 · right cols are key · +7 hash-join · · +7 · type inner +7 · equality (oid) = (attrelid) +7 · right cols are key · +8 hash-join · · +8 · type inner +8 · equality (oid) = (relnamespace) +9 cross-join · · +9 · type cross +10 hash-join · · +10 · type inner +10 · equality (oid) = (relnamespace) +11 render · · +11 · render 0 oid +11 · render 1 nspname +12 virtual table · · +12 · source pg_namespace@primary +11 merge-join · · +11 · type inner +11 · equality (oid) = (attrelid) +11 · left cols are key · +11 · right cols are key · +11 · mergeJoinOrder +"(oid=attrelid)" +12 sort · · +12 · order +oid +13 render · · +13 · render 0 oid +13 · render 1 relname +13 · render 2 relnamespace +14 virtual table · · +14 · source pg_class@pg_class_oid_idx +12 sort · · +12 · order +attrelid +13 render · · +13 · render 0 attrelid +13 · render 1 attname +13 · render 2 attnum +14 virtual table · · +14 · source pg_attribute@pg_attribute_attrelid_idx +10 filter · · +10 · filter nspname = 'public' +11 render · · +11 · render 0 oid +11 · render 1 nspname +12 virtual table · · +12 · source pg_namespace@primary +9 filter · · +9 · filter relname = 'orders' +10 render · · +10 · render 0 oid +10 · render 1 relname +10 · render 2 relnamespace +11 virtual table · · +11 · source pg_class@primary +8 render · · +8 · render 0 attrelid +8 · render 1 attname +8 · render 2 attnum +9 virtual table · · +9 · source pg_attribute@primary +7 filter · · +7 · filter contype = 'f' +8 render · · +8 · render 0 oid +8 · render 1 conname +8 · render 2 contype +8 · render 3 condeferrable +8 · render 4 condeferred +8 · render 5 conrelid +8 · render 6 confrelid +8 · render 7 confupdtype +8 · render 8 confdeltype +8 · render 9 conkey +8 · render 10 confkey +9 virtual table · · +9 · source pg_constraint@primary +6 project set · · +6 · render 0 generate_series(1, 32) +7 emptyrow · · +4 filter · · +4 · filter relkind = 'i' +5 render · · +5 · render 0 oid +5 · render 1 relname +5 · render 2 relkind +6 virtual table · · +6 · source pg_class@primary # Ensure that left joins on non-null foreign keys turn into inner joins statement ok diff --git a/pkg/sql/opt/exec/execbuilder/testdata/scalar b/pkg/sql/opt/exec/execbuilder/testdata/scalar index 023c043353f0..1b2d340144a4 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/scalar +++ b/pkg/sql/opt/exec/execbuilder/testdata/scalar @@ -660,12 +660,11 @@ render · · ("least") · query TTTTT EXPLAIN (VERBOSE) SELECT * FROM pg_attribute WHERE attrelid='t'::regclass ---- -· distributed false · · -· vectorized false · · -filter · · (attrelid, attname, atttypid, attstattarget, attlen, attnum, attndims, attcacheoff, atttypmod, attbyval, attstorage, attalign, attnotnull, atthasdef, attisdropped, attislocal, attinhcount, attcollation, attacl, attoptions, attfdwoptions) · - │ filter attrelid = t · · - └── virtual table · · (attrelid, attname, atttypid, attstattarget, attlen, attnum, attndims, attcacheoff, atttypmod, attbyval, attstorage, attalign, attnotnull, atthasdef, attisdropped, attislocal, attinhcount, attcollation, attacl, attoptions, attfdwoptions) · -· source · · · +· distributed false · · +· vectorized false · · +virtual table · · (attrelid, attname, atttypid, attstattarget, attlen, attnum, attndims, attcacheoff, atttypmod, attbyval, attstorage, attalign, attnotnull, atthasdef, attisdropped, attislocal, attinhcount, attcollation, attacl, attoptions, attfdwoptions) · +· source pg_attribute@pg_attribute_attrelid_idx · · +· constraint /2: [/t - /t] · · query TTTTT EXPLAIN (VERBOSE) SELECT CASE WHEN current_database() = 'test' THEN 42 ELSE 1/3 END diff --git a/pkg/sql/opt/exec/execbuilder/testdata/union b/pkg/sql/opt/exec/execbuilder/testdata/union index e98232545e4f..6d426185e4e2 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/union +++ b/pkg/sql/opt/exec/execbuilder/testdata/union @@ -43,7 +43,7 @@ union · · │ size 1 column, 1 row └── render · · └── virtual table · · -· source · +· source node_build_info@primary statement ok CREATE TABLE abc (a INT, b INT, c INT) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/virtual b/pkg/sql/opt/exec/execbuilder/testdata/virtual new file mode 100644 index 000000000000..9b55d73bc95e --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/virtual @@ -0,0 +1,57 @@ +query TTT +EXPLAIN SELECT * FROM pg_catalog.pg_class WHERE oid = 50 +---- +· distributed false +· vectorized false +virtual table · · +· source pg_class@pg_class_oid_idx +· constraint /2: [/50 - /50] + +query TTT +EXPLAIN SELECT * FROM pg_catalog.pg_class WHERE relname = 'blah' +---- +· distributed false +· vectorized false +filter · · + │ filter relname = 'blah' + └── virtual table · · +· source pg_class@primary + + +# We can push the filter into information_schema.tables, which has an index +# on the table_name field. +query TTT +EXPLAIN SELECT * FROM information_schema.tables WHERE table_name = 'blah' +---- +· distributed false +· vectorized false +virtual table · · +· source tables@tables_table_name_idx +· constraint /4: [/'blah' - /'blah'] + +# Make sure that if we need an ordering on one of the virtual indexes we +# provide it using a sortNode even though the optimizer expects the virtual +# index to provide it "naturally". +query TTT +EXPLAIN SELECT * FROM information_schema.tables WHERE table_name > 'blah' ORDER BY table_name +---- +· distributed false +· vectorized false +sort · · + │ order +table_name + └── virtual table · · +· source tables@tables_table_name_idx +· constraint /4: [/e'blah\x00' - ] + +# Make sure that we properly push down just part of a filter on two columns +# where only one of them is satisfied by the virtual index. +query TTT +EXPLAIN SELECT * FROM information_schema.tables WHERE table_name = 'blah' AND table_type = 'foo' +---- +· distributed false +· vectorized false +filter · · + │ filter table_type = 'foo' + └── virtual table · · +· source tables@tables_table_name_idx +· constraint /4: [/'blah' - /'blah'] diff --git a/pkg/sql/opt/optbuilder/testdata/virtual-scan b/pkg/sql/opt/optbuilder/testdata/virtual-scan index 8bee525f6959..f7334cafb226 100644 --- a/pkg/sql/opt/optbuilder/testdata/virtual-scan +++ b/pkg/sql/opt/optbuilder/testdata/virtual-scan @@ -29,3 +29,10 @@ project ├── columns: table_catalog:2!null table_schema:3!null table_name:4!null column_name:5!null ordinal_position:6!null column_default:7 is_nullable:8!null data_type:9!null character_maximum_length:10 character_octet_length:11 numeric_precision:12 numeric_precision_radix:13 numeric_scale:14 datetime_precision:15 interval_type:16 interval_precision:17 character_set_catalog:18 character_set_schema:19 character_set_name:20 collation_catalog:21 collation_schema:22 collation_name:23 domain_catalog:24 domain_schema:25 domain_name:26 udt_catalog:27 udt_schema:28 udt_name:29 scope_catalog:30 scope_schema:31 scope_name:32 maximum_cardinality:33 dtd_identifier:34 is_self_referencing:35 is_identity:36 identity_generation:37 identity_start:38 identity_increment:39 identity_maximum:40 identity_minimum:41 identity_cycle:42 is_generated:43 generation_expression:44 is_updatable:45 is_hidden:46!null crdb_sql_type:47!null └── scan c └── columns: crdb_internal_vtable_pk:1!null table_catalog:2!null table_schema:3!null table_name:4!null column_name:5!null ordinal_position:6!null column_default:7 is_nullable:8!null data_type:9!null character_maximum_length:10 character_octet_length:11 numeric_precision:12 numeric_precision_radix:13 numeric_scale:14 datetime_precision:15 interval_type:16 interval_precision:17 character_set_catalog:18 character_set_schema:19 character_set_name:20 collation_catalog:21 collation_schema:22 collation_name:23 domain_catalog:24 domain_schema:25 domain_name:26 udt_catalog:27 udt_schema:28 udt_name:29 scope_catalog:30 scope_schema:31 scope_name:32 maximum_cardinality:33 dtd_identifier:34 is_self_referencing:35 is_identity:36 identity_generation:37 identity_start:38 identity_increment:39 identity_maximum:40 identity_minimum:41 identity_cycle:42 is_generated:43 generation_expression:44 is_updatable:45 is_hidden:46!null crdb_sql_type:47!null + +# Virtual tables can't have index hints. + +build +SELECT * FROM information_schema.tables@primary +---- +error (42601): index flags not allowed with virtual tables diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index 2d8b3ae5bb14..0ff0485e99e6 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -611,7 +611,7 @@ func (c *coster) rowScanCost(tabID opt.TableID, idxOrd int, numScannedCols int) // Adjust cost based on how well the current locality matches the index's // zone constraints. var costFactor memo.Cost = cpuCostFactor - if len(c.locality.Tiers) != 0 { + if !tab.IsVirtualTable() && len(c.locality.Tiers) != 0 { // If 0% of locality tiers have matching constraints, then add additional // cost. If 100% of locality tiers have matching constraints, then add no // additional cost. Anything in between is proportional to the number of diff --git a/pkg/sql/opt/xform/custom_funcs.go b/pkg/sql/opt/xform/custom_funcs.go index 7c7c5a70705a..9e5138481bdc 100644 --- a/pkg/sql/opt/xform/custom_funcs.go +++ b/pkg/sql/opt/xform/custom_funcs.go @@ -1317,6 +1317,9 @@ func (c *CustomFuncs) GenerateLookupJoins( return } md := c.e.mem.Metadata() + if md.Table(scanPrivate.Table).IsVirtualTable() { + return + } inputProps := input.Relational() leftEq, rightEq := memo.ExtractJoinEqualityColumns(inputProps.OutputCols, scanPrivate.Cols, on) diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index c5e20c2f1061..963124f87df3 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -1247,11 +1247,17 @@ type optVirtualTable struct { // virtual table. name cat.DataSourceName - // index is a synthesized primary index. - index optVirtualIndex + // indexes contains "virtual indexes", which are used to produce virtual table + // data given constraints using generator functions. The 0th index is a + // synthesized primary index. + indexes []optVirtualIndex // family is a synthesized primary family. family optVirtualFamily + + // colMap is a mapping from unique ColumnID to column ordinal within the + // table. This is a common lookup that needs to be fast. + colMap map[sqlbase.ColumnID]int } var _ cat.Table = &optVirtualTable{} @@ -1293,12 +1299,48 @@ func newOptVirtualTable( name: *name, } + // 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[sqlbase.ColumnID(ot.Column(i).ColID())] = i + } + ot.name.ExplicitSchema = true ot.name.ExplicitCatalog = true - ot.index.init(ot) ot.family.init(ot) + // Build the indexes (add 1 to account for lack of primary index in + // indexes slice). + ot.indexes = make([]optVirtualIndex, 1+len(ot.desc.Indexes)) + // Set up the primary index. + ot.indexes[0] = optVirtualIndex{ + tab: ot, + indexOrdinal: 0, + numCols: ot.ColumnCount(), + isPrimary: true, + desc: &sqlbase.IndexDescriptor{ + ID: 0, + Name: "primary", + }, + } + + for i := range ot.desc.Indexes { + idxDesc := &ot.desc.Indexes[i] + if len(idxDesc.ColumnIDs) > 1 { + panic("virtual indexes with more than 1 col not supported") + } + + // Add 1, since the 0th index will the the primary that we added above. + ot.indexes[i+1] = optVirtualIndex{ + tab: ot, + desc: idxDesc, + indexOrdinal: i + 1, + // The virtual indexes don't return the bogus PK key? + numCols: ot.ColumnCount(), + } + } + return ot, nil } @@ -1371,25 +1413,25 @@ func (ot *optVirtualTable) Column(i int) cat.Column { // IndexCount is part of the cat.Table interface. func (ot *optVirtualTable) IndexCount() int { - return 1 + // Primary index is always present, so count is always >= 1. + return 1 + len(ot.desc.Indexes) } // WritableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) WritableIndexCount() int { - return 1 + // Primary index is always present, so count is always >= 1. + return 1 + len(ot.desc.WritableIndexes()) } // DeletableIndexCount is part of the cat.Table interface. func (ot *optVirtualTable) DeletableIndexCount() int { - return 1 + // Primary index is always present, so count is always >= 1. + return 1 + len(ot.desc.DeletableIndexes()) } // Index is part of the cat.Table interface. -func (ot *optVirtualTable) Index(i int) cat.Index { - if i > 0 { - panic("invalid index") - } - return &ot.index +func (ot *optVirtualTable) Index(i cat.IndexOrdinal) cat.Index { + return &ot.indexes[i] } // StatisticCount is part of the cat.Table interface. @@ -1510,42 +1552,35 @@ func (optDummyVirtualPKColumn) ComputedExprStr() string { return "" } -// optVirtualIndex is a dummy implementation of cat.Index for the only index +// optVirtualIndex is a dummy implementation of cat.Index for the indexes // reported by a virtual table. The index assumes that table column 0 is a dummy // PK column. type optVirtualIndex struct { tab *optVirtualTable -} -var _ cat.Index = &optIndex{} + // isPrimary is set to true if this is the dummy PK index for virtual tables. + isPrimary bool -func (oi *optVirtualIndex) init(tab *optVirtualTable) { - oi.tab = tab + desc *sqlbase.IndexDescriptor + + numCols int + + indexOrdinal int } // ID is part of the cat.Index interface. func (oi *optVirtualIndex) ID() cat.StableID { - return 1 + return cat.StableID(oi.desc.ID) } // Name is part of the cat.Index interface. func (oi *optVirtualIndex) Name() tree.Name { - return "primary" -} - -// Table is part of the cat.Index interface. -func (oi *optVirtualIndex) Table() cat.Table { - return oi.tab -} - -// Ordinal is part of the cat.Index interface. -func (oi *optVirtualIndex) Ordinal() int { - return 0 + return tree.Name(oi.desc.Name) } // IsUnique is part of the cat.Index interface. func (oi *optVirtualIndex) IsUnique() bool { - return true + return oi.desc.Unique } // IsInverted is part of the cat.Index interface. @@ -1555,7 +1590,7 @@ func (oi *optVirtualIndex) IsInverted() bool { // ColumnCount is part of the cat.Index interface. func (oi *optVirtualIndex) ColumnCount() int { - return oi.tab.ColumnCount() + return oi.numCols } // KeyColumnCount is part of the cat.Index interface. @@ -1568,14 +1603,43 @@ func (oi *optVirtualIndex) LaxKeyColumnCount() int { return 1 } +// lookupColumnOrdinal returns the ordinal of the column with the given ID. A +// cache makes the lookup O(1). +func (ot *optVirtualTable) lookupColumnOrdinal(colID sqlbase.ColumnID) (int, error) { + col, ok := ot.colMap[colID] + if ok { + return col, nil + } + return col, pgerror.Newf(pgcode.UndefinedColumn, + "column [%d] does not exist", colID) +} + // Column is part of the cat.Index interface. -func (oi *optVirtualIndex) Column(ord int) cat.IndexColumn { +func (oi *optVirtualIndex) Column(i int) cat.IndexColumn { + if oi.isPrimary { + return cat.IndexColumn{Column: oi.tab.Column(i), Ordinal: i} + } + if i == oi.ColumnCount()-1 { + // The special bogus PK column goes at the end. It has ID 0. + return cat.IndexColumn{Column: oi.tab.Column(0), Ordinal: 0} + } + length := len(oi.desc.ColumnIDs) + if i < length { + ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.ColumnIDs[i]) + return cat.IndexColumn{ + Column: oi.tab.Column(ord), + Ordinal: ord, + } + } + + i -= length + ord, _ := oi.tab.lookupColumnOrdinal(oi.desc.StoreColumnIDs[i]) return cat.IndexColumn{Column: oi.tab.Column(ord), Ordinal: ord} } // Zone is part of the cat.Index interface. func (oi *optVirtualIndex) Zone() cat.Zone { - return &zonepb.ZoneConfig{} + panic("no zone") } // Span is part of the cat.Index interface. @@ -1583,9 +1647,19 @@ func (oi *optVirtualIndex) Span() roachpb.Span { panic("no span") } +// Table is part of the cat.Index interface. +func (oi *optVirtualIndex) Table() cat.Table { + return oi.tab +} + +// Ordinal is part of the cat.Index interface. +func (oi *optVirtualIndex) Ordinal() int { + return oi.indexOrdinal +} + // PartitionByListPrefixes is part of the cat.Index interface. func (oi *optVirtualIndex) PartitionByListPrefixes() []tree.Datums { - return nil + panic("no partition") } // optVirtualFamily is a dummy implementation of cat.Family for the only family diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 6f97af660ce1..c12258482728 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -151,17 +151,23 @@ func (ef *execFactory) constructVirtualScan( if err != nil { return nil, err } - columns, constructor := virtual.getPlanInfo() + indexDesc := index.(*optVirtualIndex).desc + columns, constructor := virtual.getPlanInfo( + table.(*optVirtualTable).desc.TableDesc(), + indexDesc, indexConstraint) var n exec.Node n = &delayedNode{ - columns: columns, + name: fmt.Sprintf("%s@%s", table.Name(), index.Name()), + columns: columns, + indexConstraint: indexConstraint, constructor: func(ctx context.Context, p *planner) (planNode, error) { return constructor(ctx, p, tn.Catalog()) }, } + // Check for explicit use of the dummy column. - if needed.Contains(0) || indexConstraint != nil || len(reqOrdering) > 0 || reverse { + if needed.Contains(0) { return nil, errors.Errorf("use of %s column not allowed.", table.Column(0).ColName()) } if locking != nil { @@ -187,6 +193,16 @@ func (ef *execFactory) constructVirtualScan( return nil, err } } + // reqOrdering will be set if the optimizer expects that the output of the + // exec.Node that we're returning will actually have a legitimate ordering. + // Virtual indexes never provide a legitimate ordering, so we have to make + // sure to sort if we have a required ordering. + if len(reqOrdering) != 0 { + n, err = ef.ConstructSort(n, sqlbase.ColumnOrdering(reqOrdering), 0) + if err != nil { + return nil, err + } + } return n, nil } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 5ad4f15fbad9..46b4c80fc0e4 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/schema" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -377,55 +378,55 @@ CREATE TABLE pg_catalog.pg_am ( }, } -var pgCatalogAttrDefTable = virtualSchemaTable{ - comment: `column default values +var pgCatalogAttrDefTable = makeAllRelationsVirtualTableWithDescriptorIDIndex( + `column default values https://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html`, - schema: ` + ` CREATE TABLE pg_catalog.pg_attrdef ( oid OID, - adrelid OID, + adrelid OID NOT NULL, adnum INT2, adbin STRING, - adsrc STRING + adsrc STRING, + INDEX(adrelid) )`, - populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { - h := makeOidHasher() - return forEachTableDesc(ctx, p, dbContext, virtualMany, - func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error { - colNum := 0 - return forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error { - colNum++ - if column.DefaultExpr == nil { - // pg_attrdef only expects rows for columns with default values. - return nil - } - var defSrc *tree.DString - expr, err := parser.ParseExpr(*column.DefaultExpr) - if err != nil { - defSrc = tree.NewDString(*column.DefaultExpr) - } else { - ctx := tree.NewFmtCtx(tree.FmtPGAttrdefAdbin) - ctx.FormatNode(expr) - defSrc = tree.NewDString(ctx.String()) - } - return addRow( - h.ColumnOid(table.ID, column.ID), // oid - defaultOid(table.ID), // adrelid - tree.NewDInt(tree.DInt(colNum)), // adnum - defSrc, // adbin - defSrc, // adsrc - ) - }) - }) - }, -} + virtualMany, false, /* includesIndexEntries */ + func(ctx context.Context, p *planner, h oidHasher, db *DatabaseDescriptor, scName string, + table *sqlbase.TableDescriptor, + lookup simpleSchemaResolver, + addRow func(...tree.Datum) error) error { + colNum := 0 + return forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error { + colNum++ + if column.DefaultExpr == nil { + // pg_attrdef only expects rows for columns with default values. + return nil + } + var defSrc *tree.DString + expr, err := parser.ParseExpr(*column.DefaultExpr) + if err != nil { + defSrc = tree.NewDString(*column.DefaultExpr) + } else { + ctx := tree.NewFmtCtx(tree.FmtPGAttrdefAdbin) + ctx.FormatNode(expr) + defSrc = tree.NewDString(ctx.String()) + } + return addRow( + h.ColumnOid(table.ID, column.ID), // oid + tableOid(table.ID), // adrelid + tree.NewDInt(tree.DInt(colNum)), // adnum + defSrc, // adbin + defSrc, // adsrc + ) + }) + }) -var pgCatalogAttributeTable = virtualSchemaTable{ - comment: `table columns (incomplete - see also information_schema.columns) +var pgCatalogAttributeTable = makeAllRelationsVirtualTableWithDescriptorIDIndex( + `table columns (incomplete - see also information_schema.columns) https://www.postgresql.org/docs/9.5/catalog-pg-attribute.html`, - schema: ` + ` CREATE TABLE pg_catalog.pg_attribute ( - attrelid OID, + attrelid OID NOT NULL, attname NAME, atttypid OID, attstattarget INT4, @@ -445,76 +446,77 @@ CREATE TABLE pg_catalog.pg_attribute ( attcollation OID, attacl STRING[], attoptions STRING[], - attfdwoptions STRING[] + attfdwoptions STRING[], + INDEX(attrelid) )`, - populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { - h := makeOidHasher() - return forEachTableDesc(ctx, p, dbContext, virtualMany, func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error { - // addColumn adds adds either a table or a index column to the pg_attribute table. - addColumn := func(column *sqlbase.ColumnDescriptor, attRelID tree.Datum, colID sqlbase.ColumnID) error { - colTyp := &column.Type - attTypMod := int32(-1) - if width := colTyp.Width(); width != 0 { - switch colTyp.Family() { - case types.StringFamily: - // Postgres adds 4 to the attypmod for bounded string types, the - // var header size. - attTypMod = width + 4 - case types.BitFamily: - attTypMod = width - case types.DecimalFamily: - // attTypMod is calculated by putting the precision in the upper - // bits and the scale in the lower bits of a 32-bit int, and adding - // 4 (the var header size). We mock this for clients' sake. See - // numeric.c. - attTypMod = ((colTyp.Precision() << 16) | width) + 4 - } + virtualMany, true, /* includesIndexEntries */ + func(ctx context.Context, p *planner, h oidHasher, db *DatabaseDescriptor, scName string, + table *sqlbase.TableDescriptor, + lookup simpleSchemaResolver, + addRow func(...tree.Datum) error) error { + // addColumn adds adds either a table or a index column to the pg_attribute table. + addColumn := func(column *sqlbase.ColumnDescriptor, attRelID tree.Datum, colID sqlbase.ColumnID) error { + colTyp := &column.Type + attTypMod := int32(-1) + if width := colTyp.Width(); width != 0 { + switch colTyp.Family() { + case types.StringFamily: + // Postgres adds 4 to the attypmod for bounded string types, the + // var header size. + attTypMod = width + 4 + case types.BitFamily: + attTypMod = width + case types.DecimalFamily: + // attTypMod is calculated by putting the precision in the upper + // bits and the scale in the lower bits of a 32-bit int, and adding + // 4 (the var header size). We mock this for clients' sake. See + // numeric.c. + attTypMod = ((colTyp.Precision() << 16) | width) + 4 } - return addRow( - attRelID, // attrelid - tree.NewDName(column.Name), // attname - typOid(colTyp), // atttypid - zeroVal, // attstattarget - typLen(colTyp), // attlen - tree.NewDInt(tree.DInt(colID)), // attnum - zeroVal, // attndims - negOneVal, // attcacheoff - tree.NewDInt(tree.DInt(attTypMod)), // atttypmod - tree.DNull, // attbyval (see pg_type.typbyval) - tree.DNull, // attstorage - tree.DNull, // attalign - tree.MakeDBool(tree.DBool(!column.Nullable)), // attnotnull - tree.MakeDBool(tree.DBool(column.DefaultExpr != nil)), // atthasdef - tree.DBoolFalse, // attisdropped - tree.DBoolTrue, // attislocal - zeroVal, // attinhcount - typColl(colTyp, h), // attcollation - tree.DNull, // attacl - tree.DNull, // attoptions - tree.DNull, // attfdwoptions - ) } + return addRow( + attRelID, // attrelid + tree.NewDName(column.Name), // attname + typOid(colTyp), // atttypid + zeroVal, // attstattarget + typLen(colTyp), // attlen + tree.NewDInt(tree.DInt(colID)), // attnum + zeroVal, // attndims + negOneVal, // attcacheoff + tree.NewDInt(tree.DInt(attTypMod)), // atttypmod + tree.DNull, // attbyval (see pg_type.typbyval) + tree.DNull, // attstorage + tree.DNull, // attalign + tree.MakeDBool(tree.DBool(!column.Nullable)), // attnotnull + tree.MakeDBool(tree.DBool(column.DefaultExpr != nil)), // atthasdef + tree.DBoolFalse, // attisdropped + tree.DBoolTrue, // attislocal + zeroVal, // attinhcount + typColl(colTyp, h), // attcollation + tree.DNull, // attacl + tree.DNull, // attoptions + tree.DNull, // attfdwoptions + ) + } - // Columns for table. - if err := forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error { - tableID := defaultOid(table.ID) - return addColumn(column, tableID, column.ID) - }); err != nil { - return err - } + // Columns for table. + if err := forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error { + tableID := tableOid(table.ID) + return addColumn(column, tableID, column.ID) + }); err != nil { + return err + } - // Columns for each index. - return forEachIndexInTable(table, func(index *sqlbase.IndexDescriptor) error { - return forEachColumnInIndex(table, index, - func(column *sqlbase.ColumnDescriptor) error { - idxID := h.IndexOid(table.ID, index.ID) - return addColumn(column, idxID, column.ID) - }, - ) - }) + // Columns for each index. + return forEachIndexInTable(table, func(index *sqlbase.IndexDescriptor) error { + return forEachColumnInIndex(table, index, + func(column *sqlbase.ColumnDescriptor) error { + idxID := h.IndexOid(table.ID, index.ID) + return addColumn(column, idxID, column.ID) + }, + ) }) - }, -} + }) var pgCatalogCastTable = virtualSchemaTable{ comment: `casts (empty - needs filling out) @@ -628,12 +630,12 @@ var ( relPersistencePermanent = tree.NewDString("p") ) -var pgCatalogClassTable = virtualSchemaTable{ - comment: `tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +var pgCatalogClassTable = makeAllRelationsVirtualTableWithDescriptorIDIndex( + `tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, - schema: ` + ` CREATE TABLE pg_catalog.pg_class ( - oid OID, + oid OID NOT NULL, relname NAME NOT NULL, relnamespace OID, reltype OID, @@ -660,105 +662,101 @@ CREATE TABLE pg_catalog.pg_class ( relhassubclass BOOL, relfrozenxid INT, relacl STRING[], - reloptions STRING[] + reloptions STRING[], + INDEX (oid) )`, - populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { - h := makeOidHasher() - return forEachTableDesc(ctx, p, dbContext, virtualMany, - func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error { - // The only difference between tables, views and sequences are the relkind and relam columns. - relKind := relKindTable - relAm := forwardIndexOid - if table.IsView() { - relKind = relKindView - relAm = oidZero - } else if table.IsSequence() { - relKind = relKindSequence - relAm = oidZero - } - - namespaceOid := h.NamespaceOid(db, scName) - if err := addRow( - defaultOid(table.ID), // oid - tree.NewDName(table.Name), // relname - namespaceOid, // relnamespace - oidZero, // reltype (PG creates a composite type in pg_type for each table) - oidZero, // reloftype (PG creates a composite type in pg_type for each table) - tree.DNull, // relowner - relAm, // relam - oidZero, // relfilenode - oidZero, // reltablespace - tree.DNull, // relpages - tree.DNull, // reltuples - zeroVal, // relallvisible - oidZero, // reltoastrelid - tree.MakeDBool(tree.DBool(table.IsPhysicalTable())), // relhasindex - tree.DBoolFalse, // relisshared - relPersistencePermanent, // relPersistence - tree.DBoolFalse, // relistemp - relKind, // relkind - tree.NewDInt(tree.DInt(len(table.Columns))), // relnatts - tree.NewDInt(tree.DInt(len(table.Checks))), // relchecks - tree.DBoolFalse, // relhasoids - tree.MakeDBool(tree.DBool(table.IsPhysicalTable())), // relhaspkey - tree.DBoolFalse, // relhasrules - tree.DBoolFalse, // relhastriggers - tree.DBoolFalse, // relhassubclass - zeroVal, // relfrozenxid - tree.DNull, // relacl - tree.DNull, // reloptions - ); err != nil { - return err - } - - // Skip adding indexes for sequences (their table descriptors hav a primary - // index to make them comprehensible to backup/restore, but PG doesn't include - // an index in pg_class). - if table.IsSequence() { - return nil - } + virtualMany, true, /* includesIndexEntries */ + func(ctx context.Context, p *planner, h oidHasher, db *DatabaseDescriptor, scName string, + table *sqlbase.TableDescriptor, _ simpleSchemaResolver, addRow func(...tree.Datum) error) error { + // The only difference between tables, views and sequences are the relkind and relam columns. + relKind := relKindTable + relAm := forwardIndexOid + if table.IsView() { + relKind = relKindView + relAm = oidZero + } else if table.IsSequence() { + relKind = relKindSequence + relAm = oidZero + } + namespaceOid := h.NamespaceOid(db, scName) + if err := addRow( + tableOid(table.ID), // oid + tree.NewDName(table.Name), // relname + namespaceOid, // relnamespace + oidZero, // reltype (PG creates a composite type in pg_type for each table) + oidZero, // reloftype (PG creates a composite type in pg_type for each table) + tree.DNull, // relowner + relAm, // relam + oidZero, // relfilenode + oidZero, // reltablespace + tree.DNull, // relpages + tree.DNull, // reltuples + zeroVal, // relallvisible + oidZero, // reltoastrelid + tree.MakeDBool(tree.DBool(table.IsPhysicalTable())), // relhasindex + tree.DBoolFalse, // relisshared + relPersistencePermanent, // relPersistence + tree.DBoolFalse, // relistemp + relKind, // relkind + tree.NewDInt(tree.DInt(len(table.Columns))), // relnatts + tree.NewDInt(tree.DInt(len(table.Checks))), // relchecks + tree.DBoolFalse, // relhasoids + tree.MakeDBool(tree.DBool(table.IsPhysicalTable())), // relhaspkey + tree.DBoolFalse, // relhasrules + tree.DBoolFalse, // relhastriggers + tree.DBoolFalse, // relhassubclass + zeroVal, // relfrozenxid + tree.DNull, // relacl + tree.DNull, // reloptions + ); err != nil { + return err + } - // Indexes. - return forEachIndexInTable(table, func(index *sqlbase.IndexDescriptor) error { - indexType := forwardIndexOid - if index.Type == sqlbase.IndexDescriptor_INVERTED { - indexType = invertedIndexOid - } + // Skip adding indexes for sequences (their table descriptors hav a primary + // index to make them comprehensible to backup/restore, but PG doesn't include + // an index in pg_class). + if table.IsSequence() { + return nil + } - return addRow( - h.IndexOid(table.ID, index.ID), // oid - tree.NewDName(index.Name), // relname - namespaceOid, // relnamespace - oidZero, // reltype - oidZero, // reloftype - tree.DNull, // relowner - indexType, // relam - oidZero, // relfilenode - oidZero, // reltablespace - tree.DNull, // relpages - tree.DNull, // reltuples - zeroVal, // relallvisible - oidZero, // reltoastrelid - tree.DBoolFalse, // relhasindex - tree.DBoolFalse, // relisshared - relPersistencePermanent, // relPersistence - tree.DBoolFalse, // relistemp - relKindIndex, // relkind - tree.NewDInt(tree.DInt(len(index.ColumnNames))), // relnatts - zeroVal, // relchecks - tree.DBoolFalse, // relhasoids - tree.DBoolFalse, // relhaspkey - tree.DBoolFalse, // relhasrules - tree.DBoolFalse, // relhastriggers - tree.DBoolFalse, // relhassubclass - zeroVal, // relfrozenxid - tree.DNull, // relacl - tree.DNull, // reloptions - ) - }) - }) - }, -} + // Indexes. + return forEachIndexInTable(table, func(index *sqlbase.IndexDescriptor) error { + indexType := forwardIndexOid + if index.Type == sqlbase.IndexDescriptor_INVERTED { + indexType = invertedIndexOid + } + return addRow( + h.IndexOid(table.ID, index.ID), // oid + tree.NewDName(index.Name), // relname + namespaceOid, // relnamespace + oidZero, // reltype + oidZero, // reloftype + tree.DNull, // relowner + indexType, // relam + oidZero, // relfilenode + oidZero, // reltablespace + tree.DNull, // relpages + tree.DNull, // reltuples + zeroVal, // relallvisible + oidZero, // reltoastrelid + tree.DBoolFalse, // relhasindex + tree.DBoolFalse, // relisshared + relPersistencePermanent, // relPersistence + tree.DBoolFalse, // relistemp + relKindIndex, // relkind + tree.NewDInt(tree.DInt(len(index.ColumnNames))), // relnatts + zeroVal, // relchecks + tree.DBoolFalse, // relhasoids + tree.DBoolFalse, // relhaspkey + tree.DBoolFalse, // relhasrules + tree.DBoolFalse, // relhastriggers + tree.DBoolFalse, // relhassubclass + zeroVal, // relfrozenxid + tree.DNull, // relacl + tree.DNull, // reloptions + ) + }) + }) var pgCatalogCollationTable = virtualSchemaTable{ comment: `available collations (incomplete) @@ -835,10 +833,245 @@ var ( } ) -var pgCatalogConstraintTable = virtualSchemaTable{ - comment: `table constraints (incomplete - see also information_schema.table_constraints) +func populateTableConstraints( + ctx context.Context, + p *planner, + h oidHasher, + db *sqlbase.DatabaseDescriptor, + scName string, + table *sqlbase.TableDescriptor, + tableLookup simpleSchemaResolver, + addRow func(...tree.Datum) error, +) error { + conInfo, err := table.GetConstraintInfoWithLookup(tableLookup.getTableByID) + if err != nil { + return err + } + namespaceOid := h.NamespaceOid(db, scName) + tblOid := tableOid(table.ID) + for conName, con := range conInfo { + oid := tree.DNull + contype := tree.DNull + conindid := oidZero + confrelid := oidZero + confupdtype := tree.DNull + confdeltype := tree.DNull + confmatchtype := tree.DNull + conkey := tree.DNull + confkey := tree.DNull + consrc := tree.DNull + conbin := tree.DNull + condef := tree.DNull + + // Determine constraint kind-specific fields. + var err error + switch con.Kind { + case sqlbase.ConstraintTypePK: + oid = h.PrimaryKeyConstraintOid(db, scName, table, con.Index) + contype = conTypePKey + conindid = h.IndexOid(table.ID, con.Index.ID) + + var err error + if conkey, err = colIDArrayToDatum(con.Index.ColumnIDs); err != nil { + return err + } + condef = tree.NewDString(table.PrimaryKeyString()) + + case sqlbase.ConstraintTypeFK: + oid = h.ForeignKeyConstraintOid(db, scName, table, con.FK) + contype = conTypeFK + // Foreign keys don't have a single linked index. Pick the first one + // that matches on the referenced table. + referencedTable, err := tableLookup.getTableByID(con.FK.ReferencedTableID) + if err != nil { + return err + } + if idx, err := sqlbase.FindFKReferencedIndex(referencedTable, con.FK.ReferencedColumnIDs); err != nil { + // We couldn't find an index that matched. This shouldn't happen. + log.Warningf(ctx, "broken fk reference: %v", err) + } else { + conindid = h.IndexOid(con.ReferencedTable.ID, idx.ID) + } + confrelid = tableOid(con.ReferencedTable.ID) + if r, ok := fkActionMap[con.FK.OnUpdate]; ok { + confupdtype = r + } + if r, ok := fkActionMap[con.FK.OnDelete]; ok { + confdeltype = r + } + if r, ok := fkMatchMap[con.FK.Match]; ok { + confmatchtype = r + } + if conkey, err = colIDArrayToDatum(con.FK.OriginColumnIDs); err != nil { + return err + } + if confkey, err = colIDArrayToDatum(con.FK.ReferencedColumnIDs); err != nil { + return err + } + var buf bytes.Buffer + if err := showForeignKeyConstraint(&buf, db.Name, table, con.FK, tableLookup); err != nil { + return err + } + condef = tree.NewDString(buf.String()) + + case sqlbase.ConstraintTypeUnique: + oid = h.UniqueConstraintOid(db, scName, table, con.Index) + contype = conTypeUnique + conindid = h.IndexOid(table.ID, con.Index.ID) + var err error + if conkey, err = colIDArrayToDatum(con.Index.ColumnIDs); err != nil { + return err + } + f := tree.NewFmtCtx(tree.FmtSimple) + f.WriteString("UNIQUE (") + con.Index.ColNamesFormat(f) + f.WriteByte(')') + condef = tree.NewDString(f.CloseAndGetString()) + + case sqlbase.ConstraintTypeCheck: + oid = h.CheckConstraintOid(db, scName, table, con.CheckConstraint) + contype = conTypeCheck + if conkey, err = colIDArrayToDatum(con.CheckConstraint.ColumnIDs); err != nil { + return err + } + consrc = tree.NewDString(fmt.Sprintf("(%s)", con.Details)) + conbin = consrc + condef = tree.NewDString(fmt.Sprintf("CHECK ((%s))", con.Details)) + } + + if err := addRow( + oid, // oid + dNameOrNull(conName), // conname + namespaceOid, // connamespace + contype, // contype + tree.DBoolFalse, // condeferrable + tree.DBoolFalse, // condeferred + tree.MakeDBool(tree.DBool(!con.Unvalidated)), // convalidated + tblOid, // conrelid + oidZero, // contypid + conindid, // conindid + confrelid, // confrelid + confupdtype, // confupdtype + confdeltype, // confdeltype + confmatchtype, // confmatchtype + tree.DBoolTrue, // conislocal + zeroVal, // coninhcount + tree.DBoolTrue, // connoinherit + conkey, // conkey + confkey, // confkey + tree.DNull, // conpfeqop + tree.DNull, // conppeqop + tree.DNull, // conffeqop + tree.DNull, // conexclop + conbin, // conbin + consrc, // consrc + condef, // condef + ); err != nil { + return err + } + } + return nil +} + +type oneAtATimeSchemaResolver struct { + ctx context.Context + p *planner +} + +func (r oneAtATimeSchemaResolver) getDatabaseByID(id sqlbase.ID) (*DatabaseDescriptor, error) { + return r.p.Tables().databaseCache.getDatabaseDescByID(r.ctx, r.p.txn, id) +} + +func (r oneAtATimeSchemaResolver) getTableByID(id sqlbase.ID) (*TableDescriptor, error) { + table, err := r.p.LookupTableByID(r.ctx, id) + if err != nil { + return nil, err + } + return table.Desc.TableDesc(), nil +} + +// makeAllRelationsVirtualTableWithDescriptorIDIndex creates a virtual table that searches through +// all table descriptors in the system. It automatically adds a virtual index implementation to the +// table id column as well. The input schema must have a single INDEX definition +// with a single column, which must be the column that contains the table id. +// includesIndexEntries should be set to true if the indexed column produces +// index ids as well as just ordinary table descriptor ids. In this case, the +// caller must pass true for this variable to prevent failed lookups. +func makeAllRelationsVirtualTableWithDescriptorIDIndex( + comment string, + schemaDef string, + virtualOpts virtualOpts, + includesIndexEntries bool, + populateFromTable func(ctx context.Context, p *planner, h oidHasher, db *sqlbase.DatabaseDescriptor, + scName string, table *sqlbase.TableDescriptor, lookup simpleSchemaResolver, + addRow func(...tree.Datum) error, + ) error, +) virtualSchemaTable { + populateAll := func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { + h := makeOidHasher() + return forEachTableDescWithTableLookup(ctx, p, dbContext, virtualOpts, + func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor, lookup tableLookupFn) error { + return populateFromTable(ctx, p, h, db, scName, table, lookup, addRow) + }) + } + return virtualSchemaTable{ + comment: comment, + schema: schemaDef, + indexes: []virtualIndex{ + { + partial: includesIndexEntries, + populate: func(ctx context.Context, constraint tree.Datum, p *planner, db *DatabaseDescriptor, + addRow func(...tree.Datum) error) (bool, error) { + var id sqlbase.ID + switch t := constraint.(type) { + case *tree.DOid: + id = sqlbase.ID(t.DInt) + case *tree.DInt: + id = sqlbase.ID(*t) + default: + return false, errors.AssertionFailedf("unexpected type %T for table id column in virtual table %s", + constraint, schemaDef) + } + table, err := p.LookupTableByID(ctx, id) + if err != nil { + if sqlbase.IsUndefinedRelationError(err) { + // No table found, so no rows. In this case, we'll fall back to the + // full table scan if the index isn't complete - see the + // indexContainsNonTableDescriptorIDs parameter. + //nolint:returnerrcheck + return false, nil + } + return false, err + } + // Don't include tables that aren't in the current database unless + // they're virtual, dropped tables, or ones that the user can't see. + if (!table.Desc.IsVirtualTable() && table.Desc.ParentID != db.ID) || + table.Desc.Dropped() || + !userCanSeeTable(ctx, p, table.Desc.TableDesc(), true /*allowAdding*/) { + return false, nil + } + h := makeOidHasher() + resolver := oneAtATimeSchemaResolver{p: p, ctx: ctx} + scName, err := schema.ResolveNameByID(ctx, p.txn, db.ID, table.Desc.GetParentSchemaID()) + if err != nil { + return false, err + } + if err := populateFromTable(ctx, p, h, db, scName, table.Desc.TableDesc(), resolver, + addRow); err != nil { + return false, err + } + return true, nil + }, + }, + }, + populate: populateAll, + } +} + +var pgCatalogConstraintTable = makeAllRelationsVirtualTableWithDescriptorIDIndex( + `table constraints (incomplete - see also information_schema.table_constraints) https://www.postgresql.org/docs/9.5/catalog-pg-constraint.html`, - schema: ` + ` CREATE TABLE pg_catalog.pg_constraint ( oid OID, conname NAME, @@ -847,7 +1080,7 @@ CREATE TABLE pg_catalog.pg_constraint ( condeferrable BOOL, condeferred BOOL, convalidated BOOL, - conrelid OID, + conrelid OID NOT NULL, contypid OID, conindid OID, confrelid OID, @@ -867,147 +1100,12 @@ CREATE TABLE pg_catalog.pg_constraint ( consrc STRING, -- condef is a CockroachDB extension that provides a SHOW CREATE CONSTRAINT -- style string, for use by pg_get_constraintdef(). - condef STRING + condef STRING, + INDEX (conrelid) )`, - populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { - h := makeOidHasher() - return forEachTableDescWithTableLookup(ctx, p, dbContext, hideVirtual /*virtual tables have no constraints*/, func( - db *sqlbase.DatabaseDescriptor, - scName string, - table *sqlbase.TableDescriptor, - tableLookup tableLookupFn, - ) error { - conInfo, err := table.GetConstraintInfoWithLookup(tableLookup.getTableByID) - if err != nil { - return err - } - namespaceOid := h.NamespaceOid(db, scName) - tblOid := defaultOid(table.ID) - for conName, con := range conInfo { - oid := tree.DNull - contype := tree.DNull - conindid := oidZero - confrelid := oidZero - confupdtype := tree.DNull - confdeltype := tree.DNull - confmatchtype := tree.DNull - conkey := tree.DNull - confkey := tree.DNull - consrc := tree.DNull - conbin := tree.DNull - condef := tree.DNull - - // Determine constraint kind-specific fields. - var err error - switch con.Kind { - case sqlbase.ConstraintTypePK: - oid = h.PrimaryKeyConstraintOid(db, scName, table, con.Index) - contype = conTypePKey - conindid = h.IndexOid(table.ID, con.Index.ID) - - var err error - if conkey, err = colIDArrayToDatum(con.Index.ColumnIDs); err != nil { - return err - } - condef = tree.NewDString(table.PrimaryKeyString()) - - case sqlbase.ConstraintTypeFK: - oid = h.ForeignKeyConstraintOid(db, scName, table, con.FK) - contype = conTypeFK - // Foreign keys don't have a single linked index. Pick the first one - // that matches on the referenced table. - referencedTable, err := tableLookup.getTableByID(con.FK.ReferencedTableID) - if err != nil { - return err - } - if idx, err := sqlbase.FindFKReferencedIndex(referencedTable, con.FK.ReferencedColumnIDs); err != nil { - // We couldn't find an index that matched. This shouldn't happen. - log.Warningf(ctx, "broken fk reference: %v", err) - } else { - conindid = h.IndexOid(con.ReferencedTable.ID, idx.ID) - } - confrelid = defaultOid(con.ReferencedTable.ID) - if r, ok := fkActionMap[con.FK.OnUpdate]; ok { - confupdtype = r - } - if r, ok := fkActionMap[con.FK.OnDelete]; ok { - confdeltype = r - } - if r, ok := fkMatchMap[con.FK.Match]; ok { - confmatchtype = r - } - if conkey, err = colIDArrayToDatum(con.FK.OriginColumnIDs); err != nil { - return err - } - if confkey, err = colIDArrayToDatum(con.FK.ReferencedColumnIDs); err != nil { - return err - } - var buf bytes.Buffer - if err := showForeignKeyConstraint(&buf, db.Name, table, con.FK, tableLookup); err != nil { - return err - } - condef = tree.NewDString(buf.String()) - - case sqlbase.ConstraintTypeUnique: - oid = h.UniqueConstraintOid(db, scName, table, con.Index) - contype = conTypeUnique - conindid = h.IndexOid(table.ID, con.Index.ID) - var err error - if conkey, err = colIDArrayToDatum(con.Index.ColumnIDs); err != nil { - return err - } - f := tree.NewFmtCtx(tree.FmtSimple) - f.WriteString("UNIQUE (") - con.Index.ColNamesFormat(f) - f.WriteByte(')') - condef = tree.NewDString(f.CloseAndGetString()) - - case sqlbase.ConstraintTypeCheck: - oid = h.CheckConstraintOid(db, scName, table, con.CheckConstraint) - contype = conTypeCheck - if conkey, err = colIDArrayToDatum(con.CheckConstraint.ColumnIDs); err != nil { - return err - } - consrc = tree.NewDString(fmt.Sprintf("(%s)", con.Details)) - conbin = consrc - condef = tree.NewDString(fmt.Sprintf("CHECK ((%s))", con.Details)) - } - - if err := addRow( - oid, // oid - dNameOrNull(conName), // conname - namespaceOid, // connamespace - contype, // contype - tree.DBoolFalse, // condeferrable - tree.DBoolFalse, // condeferred - tree.MakeDBool(tree.DBool(!con.Unvalidated)), // convalidated - tblOid, // conrelid - oidZero, // contypid - conindid, // conindid - confrelid, // confrelid - confupdtype, // confupdtype - confdeltype, // confdeltype - confmatchtype, // confmatchtype - tree.DBoolTrue, // conislocal - zeroVal, // coninhcount - tree.DBoolTrue, // connoinherit - conkey, // conkey - confkey, // confkey - tree.DNull, // conpfeqop - tree.DNull, // conppeqop - tree.DNull, // conffeqop - tree.DNull, // conexclop - conbin, // conbin - consrc, // consrc - condef, // condef - ); err != nil { - return err - } - } - return nil - }) - }, -} + hideVirtual, /* Virtual tables have no constraints */ + false, /* includesIndexEntries */ + populateTableConstraints) // colIDArrayToDatum returns an int[] containing the ColumnIDs, or NULL if there // are no ColumnIDs. @@ -1079,7 +1177,7 @@ CREATE TABLE pg_catalog.pg_database ( populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { return forEachDatabaseDesc(ctx, p, nil /*all databases*/, func(db *sqlbase.DatabaseDescriptor) error { return addRow( - defaultOid(db.ID), // oid + dbOid(db.ID), // oid tree.NewDName(db.Name), // datname tree.DNull, // datdba // If there is a change in encoding value for the database we must update @@ -1173,13 +1271,13 @@ CREATE TABLE pg_catalog.pg_depend ( table *sqlbase.TableDescriptor, tableLookup tableLookupFn, ) error { - pgConstraintTableOid := defaultOid(pgConstraintsDesc.ID) - pgClassTableOid := defaultOid(pgClassDesc.ID) + pgConstraintTableOid := tableOid(pgConstraintsDesc.ID) + pgClassTableOid := tableOid(pgClassDesc.ID) if table.IsSequence() && !table.SequenceOpts.SequenceOwner.Equal(sqlbase.TableDescriptor_SequenceOpts_SequenceOwner{}) { - refObjID := defaultOid(table.SequenceOpts.SequenceOwner.OwnerTableID) + refObjID := tableOid(table.SequenceOpts.SequenceOwner.OwnerTableID) refObjSubID := tree.NewDInt(tree.DInt(table.SequenceOpts.SequenceOwner.OwnerColumnID)) - objID := defaultOid(table.GetID()) + objID := tableOid(table.GetID()) return addRow( pgConstraintTableOid, // classid objID, // objid @@ -1329,7 +1427,7 @@ CREATE TABLE pg_catalog.pg_shdescription ( classOid := tree.NewDOid(sqlbase.PgCatalogDatabaseTableID) objID := sqlbase.ID(tree.MustBeDInt(comment[0])) if err := addRow( - defaultOid(objID), + tableOid(objID), classOid, comment[2]); err != nil { return err @@ -1486,7 +1584,7 @@ CREATE TABLE pg_catalog.pg_index ( h := makeOidHasher() return forEachTableDesc(ctx, p, dbContext, hideVirtual, /* virtual tables do not have indexes */ func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error { - tableOid := defaultOid(table.ID) + tableOid := tableOid(table.ID) return forEachIndexInTable(table, func(index *sqlbase.IndexDescriptor) error { isMutation, isWriteOnly := table.GetIndexMutationCapabilities(index.ID) @@ -1526,29 +1624,30 @@ CREATE TABLE pg_catalog.pg_index ( // TODO(bram): #27763 indclass still needs to be populated but it // requires pg_catalog.pg_opclass first. indclass, err := makeZeroedOidVector(len(index.ColumnIDs)) + isPrimary := table.PrimaryIndex.ID == index.ID && table.IsPhysicalTable() if err != nil { return err } return addRow( h.IndexOid(table.ID, index.ID), // indexrelid tableOid, // indrelid - tree.NewDInt(tree.DInt(len(index.ColumnNames))), // indnatts - tree.MakeDBool(tree.DBool(index.Unique)), // indisunique - tree.MakeDBool(tree.DBool(table.IsPhysicalTable() && index.ID == table.PrimaryIndex.ID)), // indisprimary - tree.DBoolFalse, // indisexclusion - tree.MakeDBool(tree.DBool(index.Unique)), // indimmediate - tree.DBoolFalse, // indisclustered - tree.MakeDBool(tree.DBool(!isMutation)), // indisvalid - tree.DBoolFalse, // indcheckxmin - tree.MakeDBool(tree.DBool(isReady)), // indisready - tree.DBoolTrue, // indislive - tree.DBoolFalse, // indisreplident - indkey, // indkey - collationOidVector, // indcollation - indclass, // indclass - indoptionIntVector, // indoption - tree.DNull, // indexprs - tree.DNull, // indpred + tree.NewDInt(tree.DInt(len(index.ColumnNames))), // indnatts + tree.MakeDBool(tree.DBool(index.Unique)), // indisunique + tree.MakeDBool(tree.DBool(isPrimary)), // indisprimary + tree.DBoolFalse, // indisexclusion + tree.MakeDBool(tree.DBool(index.Unique)), // indimmediate + tree.DBoolFalse, // indisclustered + tree.MakeDBool(tree.DBool(!isMutation)), // indisvalid + tree.DBoolFalse, // indcheckxmin + tree.MakeDBool(tree.DBool(isReady)), // indisready + tree.DBoolTrue, // indislive + tree.DBoolFalse, // indisreplident + indkey, // indkey + collationOidVector, // indcollation + indclass, // indclass + indoptionIntVector, // indoption + tree.DNull, // indexprs + tree.DNull, // indpred ) }) }) @@ -2249,7 +2348,7 @@ CREATE TABLE pg_catalog.pg_sequence ( } opts := table.SequenceOpts return addRow( - defaultOid(table.ID), // seqrelid + tableOid(table.ID), // seqrelid tree.NewDOid(tree.DInt(oid.T_int8)), // seqtypid tree.NewDInt(tree.DInt(opts.Start)), // seqstart tree.NewDInt(tree.DInt(opts.Increment)), // seqincrement @@ -3061,7 +3160,11 @@ func (h oidHasher) OperatorOid(name string, leftType, rightType, returnType *tre return h.getOid() } -func defaultOid(id sqlbase.ID) *tree.DOid { +func tableOid(id sqlbase.ID) *tree.DOid { + return tree.NewDOid(tree.DInt(id)) +} + +func dbOid(id sqlbase.ID) *tree.DOid { return tree.NewDOid(tree.DInt(id)) } diff --git a/pkg/sql/pg_oid_test.go b/pkg/sql/pg_oid_test.go index 903414380f49..b4e8395ae956 100644 --- a/pkg/sql/pg_oid_test.go +++ b/pkg/sql/pg_oid_test.go @@ -36,7 +36,7 @@ func TestDefaultOid(t *testing.T) { } for _, tc := range testCases { - oid := defaultOid(tc.id) + oid := tableOid(tc.id) if tc.oid.DInt != oid.DInt { t.Fatalf("expected oid %d(%32b), got %d(%32b)", tc.oid.DInt, tc.oid.DInt, oid.DInt, oid.DInt) } diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index e730a48966f7..a66a97f7fab4 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -688,8 +688,8 @@ func (l *internalLookupCtx) getParentName(table *TableDescriptor) string { // getParentAsTableName returns a TreeTable object of the parent table for a // given table ID. Used to get the parent table of a table with interleaved // indexes. -func (l *internalLookupCtx) getParentAsTableName( - parentTableID sqlbase.ID, dbPrefix string, +func getParentAsTableName( + l simpleSchemaResolver, parentTableID sqlbase.ID, dbPrefix string, ) (tree.TableName, error) { var parentName tree.TableName parentTable, err := l.getTableByID(parentTableID) @@ -706,8 +706,8 @@ func (l *internalLookupCtx) getParentAsTableName( } // getTableAsTableName returns a TableName object for a given TableDescriptor. -func (l *internalLookupCtx) getTableAsTableName( - table *sqlbase.TableDescriptor, dbPrefix string, +func getTableAsTableName( + l simpleSchemaResolver, table *sqlbase.TableDescriptor, dbPrefix string, ) (tree.TableName, error) { var tableName tree.TableName tableDbDesc, err := l.getDatabaseByID(table.ParentID) @@ -795,3 +795,8 @@ func (p *planner) ResolveExistingObjectEx( func (p *planner) ResolvedName(u *tree.UnresolvedObjectName) *tree.TableName { return u.Resolved(&p.semaCtx.Annotations) } + +type simpleSchemaResolver interface { + getDatabaseByID(id sqlbase.ID) (*DatabaseDescriptor, error) + getTableByID(id sqlbase.ID) (*TableDescriptor, error) +} diff --git a/pkg/sql/schema/schema.go b/pkg/sql/schema/schema.go index cc1045464e53..924460f29f5f 100644 --- a/pkg/sql/schema/schema.go +++ b/pkg/sql/schema/schema.go @@ -18,11 +18,19 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) +var staticSchemaIDMap = map[sqlbase.ID]string{ + keys.PublicSchemaID: tree.PublicSchema, + sqlbase.PgCatalogID: sessiondata.PgCatalogName, + sqlbase.InformationSchemaID: sessiondata.InformationSchemaName, + sqlbase.CrdbInternalID: sessiondata.CRDBInternalSchemaName, +} + // ResolveNameByID resolves a schema's name based on db and schema id. // TODO(sqlexec): this should return the descriptor instead if given an ID. // Instead, we have to rely on a scan of the kv table. @@ -30,9 +38,11 @@ import ( func ResolveNameByID( ctx context.Context, txn *kv.Txn, dbID sqlbase.ID, schemaID sqlbase.ID, ) (string, error) { - // Fast-path for public schema, to avoid hot lookups. - if schemaID == keys.PublicSchemaID { - return string(tree.PublicSchemaName), nil + // Fast-path for public schema and virtual schemas, to avoid hot lookups. + for id, schemaName := range staticSchemaIDMap { + if id == schemaID { + return schemaName, nil + } } schemas, err := GetForDatabase(ctx, txn, dbID) if err != nil { diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index 85ae2fe99d78..0dfbf64cd3cb 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -27,6 +27,12 @@ const DefaultDatabaseName = "defaultdb" // PgCatalogName is the name of the pg_catalog system schema. const PgCatalogName = "pg_catalog" +// InformationSchemaName is the name of the information_schema system schema. +const InformationSchemaName = "information_schema" + +// CRDBInternalSchemaName is the name of the crdb_internal system schema. +const CRDBInternalSchemaName = "crdb_internal" + // PgTempSchemaName is the alias for temporary schemas across sessions. const PgTempSchemaName = "pg_temp" diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index caa4a17ffe9c..53bc3ba4e565 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -51,7 +51,7 @@ func ShowCreateTable( tn *tree.Name, dbPrefix string, desc *sqlbase.TableDescriptor, - lCtx *internalLookupCtx, + lCtx simpleSchemaResolver, fkDisplayMode shouldOmitFKClausesFromCreate, ) (string, error) { a := &sqlbase.DatumAlloc{} diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index fac7e0c568de..70e2bd0e7b81 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -138,7 +138,7 @@ func showForeignKeyConstraint( dbPrefix string, originTable *sqlbase.TableDescriptor, fk *sqlbase.ForeignKeyConstraint, - lCtx *internalLookupCtx, + lCtx simpleSchemaResolver, ) error { var refNames []string var originNames []string @@ -245,7 +245,7 @@ func showFamilyClause(desc *sqlbase.TableDescriptor, f *tree.FmtCtx) { // it is equal to the given dbPrefix. This allows us to elide the prefix // when the given index is interleaved in a table of the current database. func showCreateInterleave( - idx *sqlbase.IndexDescriptor, buf *bytes.Buffer, dbPrefix string, lCtx *internalLookupCtx, + idx *sqlbase.IndexDescriptor, buf *bytes.Buffer, dbPrefix string, lCtx simpleSchemaResolver, ) error { if len(idx.Interleave.Ancestors) == 0 { return nil @@ -255,7 +255,7 @@ func showCreateInterleave( var err error var parentName tree.TableName if lCtx != nil { - parentName, err = lCtx.getParentAsTableName(parentTableID, dbPrefix) + parentName, err = getParentAsTableName(lCtx, parentTableID, dbPrefix) if err != nil { return err } diff --git a/pkg/sql/sqlbase/errors.go b/pkg/sql/sqlbase/errors.go index cd769a044a78..7322880d31a7 100644 --- a/pkg/sql/sqlbase/errors.go +++ b/pkg/sql/sqlbase/errors.go @@ -182,6 +182,11 @@ func IsUndefinedColumnError(err error) bool { return errHasCode(err, pgcode.UndefinedColumn) } +// IsUndefinedRelationError checks whether this is an undefined relation error. +func IsUndefinedRelationError(err error) bool { + return errHasCode(err, pgcode.UndefinedTable) +} + func errHasCode(err error, code ...string) bool { pgCode := pgerror.GetPGCode(err) for _, c := range code { diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 2b7ba6143c9b..e792bd65e7bf 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -1221,12 +1221,15 @@ func (desc *MutableTableDescriptor) AllocateIDs() error { } } - // Only physical tables can have / need indexes and column families. - if desc.IsPhysicalTable() { + // Only tables can have / need indexes and column families. + if desc.IsTable() { if err := desc.allocateIndexIDs(columnNames); err != nil { return err } - desc.allocateColumnFamilyIDs(columnNames) + // Virtual tables don't have column families. + if desc.IsPhysicalTable() { + desc.allocateColumnFamilyIDs(columnNames) + } } // This is sort of ugly. If the descriptor does not have an ID, we hack one in diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index c410f1765a2c..30f2a860fa7a 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -16,6 +16,7 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -53,11 +54,27 @@ type virtualSchema struct { type virtualSchemaDef interface { getSchema() string initVirtualTableDesc( - ctx context.Context, st *cluster.Settings, id sqlbase.ID, + ctx context.Context, st *cluster.Settings, parentSchemaID, id sqlbase.ID, ) (sqlbase.TableDescriptor, error) getComment() string } +type virtualIndex struct { + // populate populates the table given the constraint. matched is true if any + // rows were generated. + populate func(ctx context.Context, constraint tree.Datum, p *planner, db *DatabaseDescriptor, + addRow func(...tree.Datum) error, + ) (matched bool, err error) + + // partial is true if the virtual index isn't able to satisfy all constraints. + // For example, the pg_class table contains both indexes and tables. Tables + // can be looked up via a virtual index, since we can look up their descriptor + // by their ID directly. But indexes can't - they're hashed identifiers with + // no actual index. So we mark this index as partial, and if we get no match + // during populate, we'll fall back on populating the entire table. + partial bool +} + // virtualSchemaTable represents a table within a virtualSchema. type virtualSchemaTable struct { // Exactly one of the populate and generator fields should be defined for @@ -72,6 +89,12 @@ type virtualSchemaTable struct { // during initialization of the valuesNode. populate func(ctx context.Context, p *planner, db *DatabaseDescriptor, addRow func(...tree.Datum) error) error + // indexes, if non empty, is a slice of populate methods that also take a + // constraint, only generating rows that match the constraint. The order of + // indexes must match the order of the index definitions in the virtual table's + // schema. + indexes []virtualIndex + // generator, if non-nil, is a function that is used when creating a // virtualTableNode. This function returns a virtualTableGenerator function // which generates the next row of the virtual table when called. @@ -91,7 +114,7 @@ func (t virtualSchemaTable) getSchema() string { // initVirtualTableDesc is part of the virtualSchemaDef interface. func (t virtualSchemaTable) initVirtualTableDesc( - ctx context.Context, st *cluster.Settings, id sqlbase.ID, + ctx context.Context, st *cluster.Settings, parentSchemaID, id sqlbase.ID, ) (sqlbase.TableDescriptor, error) { stmt, err := parser.ParseOne(t.schema) if err != nil { @@ -99,13 +122,27 @@ func (t virtualSchemaTable) initVirtualTableDesc( } create := stmt.AST.(*tree.CreateTable) + var firstColDef *tree.ColumnTableDef for _, def := range create.Defs { - if d, ok := def.(*tree.ColumnTableDef); ok && d.HasDefaultExpr() { + if d, ok := def.(*tree.ColumnTableDef); ok { + if d.HasDefaultExpr() { + return sqlbase.TableDescriptor{}, + errors.Errorf("virtual tables are not allowed to use default exprs "+ + "because bootstrapping: %s:%s", &create.Table, d.Name) + } + if firstColDef == nil { + firstColDef = d + } + } + if _, ok := def.(*tree.UniqueConstraintTableDef); ok { return sqlbase.TableDescriptor{}, - errors.Errorf("virtual tables are not allowed to use default exprs "+ - "because bootstrapping: %s:%s", &create.Table, d.Name) + errors.Errorf("virtual tables are not allowed to have unique constraints") } } + if firstColDef == nil { + return sqlbase.TableDescriptor{}, + errors.Errorf("can't have empty virtual tables") + } // Virtual tables never use SERIAL so we need not process SERIAL // types here. @@ -116,7 +153,7 @@ func (t virtualSchemaTable) initVirtualTableDesc( st, create, 0, /* parentID */ - 0, /* parentSchemaID */ + parentSchemaID, id, hlc.Timestamp{}, /* creationTime */ publicSelectPrivileges, @@ -126,7 +163,34 @@ func (t virtualSchemaTable) initVirtualTableDesc( &sessiondata.SessionData{}, /* sessionData */ false, /* temporary */ ) - return mutDesc.TableDescriptor, err + if err != nil { + return mutDesc.TableDescriptor, err + } + for i := range mutDesc.Indexes { + idx := &mutDesc.Indexes[i] + if len(idx.ColumnIDs) > 1 { + panic("we don't know how to deal with virtual composite indexes yet") + } + // All indexes of virtual tables automatically STORE all other columns in + // the table. + idx.StoreColumnIDs = make([]sqlbase.ColumnID, len(mutDesc.Columns)-len(idx.ColumnIDs)) + idx.StoreColumnNames = make([]string, len(mutDesc.Columns)-len(idx.ColumnIDs)) + // Store all columns but the ones in the index. + outputIdx := 0 + EACHCOLUMN: + for j := range mutDesc.Columns { + for _, id := range idx.ColumnIDs { + if mutDesc.Columns[j].ID == id { + // Skip columns in the index. + continue EACHCOLUMN + } + } + idx.StoreColumnIDs[outputIdx] = mutDesc.Columns[j].ID + idx.StoreColumnNames[outputIdx] = mutDesc.Columns[j].Name + outputIdx++ + } + } + return mutDesc.TableDescriptor, nil } // getComment is part of the virtualSchemaDef interface. @@ -141,7 +205,7 @@ func (v virtualSchemaView) getSchema() string { // initVirtualTableDesc is part of the virtualSchemaDef interface. func (v virtualSchemaView) initVirtualTableDesc( - ctx context.Context, st *cluster.Settings, id sqlbase.ID, + ctx context.Context, st *cluster.Settings, parentSchemaID sqlbase.ID, id sqlbase.ID, ) (sqlbase.TableDescriptor, error) { stmt, err := parser.ParseOne(v.schema) if err != nil { @@ -158,7 +222,7 @@ func (v virtualSchemaView) initVirtualTableDesc( create.Name.Table(), tree.AsStringWithFlags(create.AsSource, tree.FmtParsable), 0, /* parentID */ - 0, /* parentSchemaID */ + parentSchemaID, id, columns, hlc.Timestamp{}, /* creationTime */ @@ -234,7 +298,11 @@ func newInvalidVirtualDefEntryError() error { // valuesNode for the virtual table. We use deferred construction here // so as to avoid populating a RowContainer during query preparation, // where we can't guarantee it will be Close()d in case of error. -func (e virtualDefEntry) getPlanInfo() (sqlbase.ResultColumns, virtualTableConstructor) { +func (e virtualDefEntry) getPlanInfo( + table *sqlbase.TableDescriptor, + index *sqlbase.IndexDescriptor, + idxConstraint *constraint.Constraint, +) (sqlbase.ResultColumns, virtualTableConstructor) { var columns sqlbase.ResultColumns for i := range e.desc.Columns { col := &e.desc.Columns[i] @@ -270,11 +338,14 @@ func (e virtualDefEntry) getPlanInfo() (sqlbase.ResultColumns, virtualTableConst if err != nil { return nil, err } - return p.newContainerVirtualTableNode(columns, 0, next, cleanup), nil + return p.newVirtualTableNode(columns, 0, next, cleanup), nil } + v := p.newContainerValuesNode(columns, 0) - if err := def.populate(ctx, p, dbDesc, func(datums ...tree.Datum) error { + constrainedScan := idxConstraint != nil && !idxConstraint.IsUnconstrained() + + validateRow := func(datums ...tree.Datum) { if r, c := len(datums), len(v.columns); r != c { log.Fatalf(ctx, "datum row count and column count differ: %d vs %d", r, c) } @@ -290,9 +361,127 @@ func (e virtualDefEntry) getPlanInfo() (sqlbase.ResultColumns, virtualTableConst col.Name, col.Typ, datum.ResolvedType()) } } - _, err := v.rows.AddRow(ctx, datums) + } + if !constrainedScan { + // If we don't have a constrained scan, just populate all the rows and + // we're done. + addRow := func(datums ...tree.Datum) error { + validateRow(datums...) + _, err := v.rows.AddRow(ctx, datums) + return err + } + if err := def.populate(ctx, p, dbDesc, addRow); err != nil { + v.Close(ctx) + return nil, err + } + return v, nil + } + + // We are now dealing with a constrained virtual index scan. + + if index.ID == 1 { + return nil, errors.AssertionFailedf( + "programming error: can't constrain scan on primary virtual index of table %s", e.desc.Name) + } + + // Figure out the ordinal position of the column that we're filtering on. + // TODO(jordan): there is probably a better way to do this. + constraintColumnOrd := table.ColumnIdxMap()[index.ColumnIDs[0]] + addRowIfPassesFilter := func(datums ...tree.Datum) error { + validateRow(datums...) + matchedFilter := false + // If we have a constraint, apply it as a filter. If the filter passes, + // or we had no filter at all, add the datums to the row container. + datumToFilter := datums[constraintColumnOrd] + for i := 0; i < idxConstraint.Spans.Count(); i++ { + span := idxConstraint.Spans.Get(i) + if span.HasSingleKey(p.EvalContext()) { + matchedFilter = datumToFilter.Compare(p.EvalContext(), span.StartKey().Value(0)) == 0 + if matchedFilter { + break + } + } else { + // Check both boundaries - the datum must be greater than the + // startKey, and less than the endKey. + var matchedStart, matchedEnd bool + if span.StartKey().IsEmpty() { + matchedStart = true + } else { + startCmp := datumToFilter.Compare(p.EvalContext(), span.StartKey().Value(0)) + if span.StartBoundary() == constraint.IncludeBoundary { + matchedStart = startCmp >= 0 + } else { + matchedStart = startCmp > 0 + } + } + if span.EndKey().IsEmpty() { + matchedEnd = true + } else { + endCmp := datumToFilter.Compare(p.EvalContext(), span.EndKey().Value(0)) + if span.EndBoundary() == constraint.IncludeBoundary { + matchedEnd = endCmp <= 0 + } else { + matchedEnd = endCmp < 0 + } + } + matchedFilter = matchedStart && matchedEnd + if matchedFilter { + break + } + } + } + + var err error + if matchedFilter { + _, err = v.rows.AddRow(ctx, datums) + } return err - }); err != nil { + } + + giveUp := false + + // We have a virtual index with a constraint. Run the constrained + // populate routine for every span. If for some reason we can't use the + // index for a span, we set the giveUp flag to true, which will populate + // the whole table, filtering the output. + for i := 0; i < idxConstraint.Spans.Count(); i++ { + span := idxConstraint.Spans.Get(i) + if span.StartKey().Length() > 1 { + return nil, errors.AssertionFailedf( + "programming error: can't push down composite constraints into vtables") + } + if !span.HasSingleKey(p.EvalContext()) { + // No hope - we can't deal with range scans on virtual indexes. + giveUp = true + break + } + constraintDatum := span.StartKey().Value(0) + // Subtract 2 from the index id to get the ordinal in def.indexes, since + // the index with ID 1 is the "primary" index defined by def.populate. + virtualIndex := def.indexes[index.ID-2] + + // For each span, run populate. + found, err := virtualIndex.populate(ctx, constraintDatum, p, dbDesc, + addRowIfPassesFilter) + if err != nil { + v.Close(ctx) + return nil, err + } + if !found && virtualIndex.partial { + // If we found nothing, and the index was partial, we have no choice + // but to populate the entire table and search through it. + giveUp = true + break + } + } + if !giveUp { + return v, nil + } + + // Fall back to populating the whole table - clear what we have so far + // and start over with a full populate and filter. + v.rows.Clear(ctx) + if err := def.populate(ctx, p, dbDesc, addRowIfPassesFilter); err != nil { v.Close(ctx) return nil, err } @@ -324,7 +513,7 @@ func NewVirtualSchemaHolder( orderedDefNames := make([]string, 0, len(schema.tableDefs)) for id, def := range schema.tableDefs { - tableDesc, err := def.initVirtualTableDesc(ctx, st, id) + tableDesc, err := def.initVirtualTableDesc(ctx, st, schemaID, id) if err != nil { return nil, errors.NewAssertionErrorWithWrappedErrf(err, diff --git a/pkg/sql/virtual_table.go b/pkg/sql/virtual_table.go index af5f5f7afb33..89255111b9bc 100644 --- a/pkg/sql/virtual_table.go +++ b/pkg/sql/virtual_table.go @@ -147,7 +147,7 @@ type virtualTableNode struct { currentRow tree.Datums } -func (p *planner) newContainerVirtualTableNode( +func (p *planner) newVirtualTableNode( columns sqlbase.ResultColumns, capacity int, next virtualTableGenerator, cleanup func(), ) *virtualTableNode { return &virtualTableNode{ diff --git a/pkg/sql/vtable/information_schema.go b/pkg/sql/vtable/information_schema.go index 92555939cf55..96ec93728345 100644 --- a/pkg/sql/vtable/information_schema.go +++ b/pkg/sql/vtable/information_schema.go @@ -135,5 +135,6 @@ CREATE TABLE information_schema.tables ( TABLE_NAME STRING NOT NULL, TABLE_TYPE STRING NOT NULL, IS_INSERTABLE_INTO STRING NOT NULL, - VERSION INT + VERSION INT, + INDEX(TABLE_NAME) )` diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index a7584a2301cd..848322d77ed2 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -633,6 +633,9 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { case *delayedNode: if v.observer.attr != nil { v.observer.attr(name, "source", n.name) + if n.indexConstraint != nil { + v.observer.attr(name, "constraint", n.indexConstraint.String()) + } } if n.plan != nil { n.plan = v.visit(n.plan) From c44619c38ba6f22d8d6590a34600ba0c2fd7fdd9 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 10 Apr 2020 00:12:03 -0400 Subject: [PATCH 3/3] sql: SHOW CREATE takes advantage of virtual index This commit updates the implementation of SHOW CREATE so that it doesn't have to filter every single table in the database. It does 2 things: 1. Stop outputting zone configs in crdb_internal.create_statements 2. Start outputting partitioning status in the above instead 3. Get zone configs from the SQL query in SHOW CREATE with a subquery from the zone configs table and join together in SQL This strategy makes it possible to give crdb_internal.create_statements a virtual index. Now SHOW CREATE is really fast! Release note (performance improvement): SHOW CREATE is much more efficient --- pkg/sql/crdb_internal.go | 209 +++++++----------- pkg/sql/delegate/show_table.go | 32 ++- .../testdata/logic_test/crdb_internal | 2 +- .../logictest/testdata/logic_test/sequences | 6 +- pkg/sql/partition_test.go | 5 + 5 files changed, 103 insertions(+), 151 deletions(-) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 3c4e725d8be8..19cb048c5d9e 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -47,7 +47,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" - "gopkg.in/yaml.v2" ) const crdbInternalName = sessiondata.CRDBInternalSchemaName @@ -1372,13 +1371,18 @@ CREATE TABLE crdb_internal.builtin_functions ( }, } +// Prepare the row populate function. +var typeView = tree.NewDString("view") +var typeTable = tree.NewDString("table") +var typeSequence = tree.NewDString("sequence") + // crdbInternalCreateStmtsTable exposes the CREATE TABLE/CREATE VIEW // statements. // // TODO(tbg): prefix with kv_. -var crdbInternalCreateStmtsTable = virtualSchemaTable{ - comment: `CREATE and ALTER statements for all tables accessible by current user in current database (KV scan)`, - schema: ` +var crdbInternalCreateStmtsTable = makeAllRelationsVirtualTableWithDescriptorIDIndex( + `CREATE and ALTER statements for all tables accessible by current user in current database (KV scan)`, + ` CREATE TABLE crdb_internal.create_statements ( database_id INT, database_name STRING, @@ -1391,148 +1395,83 @@ CREATE TABLE crdb_internal.create_statements ( create_nofks STRING NOT NULL, alter_statements STRING[] NOT NULL, validate_statements STRING[] NOT NULL, - zone_configuration_statements STRING[] NOT NULL + has_partitions BOOL NOT NULL, + INDEX(descriptor_id) ) -`, - populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { +`, virtualOnce, false, /* includesIndexEntries */ + func(ctx context.Context, p *planner, h oidHasher, db *sqlbase.DatabaseDescriptor, scName string, + table *sqlbase.TableDescriptor, lookup simpleSchemaResolver, addRow func(...tree.Datum) error) error { contextName := "" - if dbContext != nil { - contextName = dbContext.Name - } - - // Prepare the row populate function. - typeView := tree.NewDString("view") - typeTable := tree.NewDString("table") - typeSequence := tree.NewDString("sequence") - - // Hold the configuration statements for each table - zoneConfigStmts := make(map[string][]string) - // Prepare a query used to see zones configuations on this table. - configStmtsQuery := ` - SELECT - table_name, raw_config_yaml, raw_config_sql - FROM - crdb_internal.zones - WHERE - database_name = '%[1]s' - AND table_name IS NOT NULL - AND raw_config_yaml IS NOT NULL - AND raw_config_sql IS NOT NULL - ORDER BY - database_name, table_name, index_name, partition_name - ` - // The create_statements table is used at times where other internal - // tables have not been created, or are unaccessible (perhaps during - // certain tests (TestDumpAsOf in pkg/cli/dump_test.go)). So if something - // goes wrong querying this table, proceed without any constraint data. - zoneConstraintRows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query( - ctx, "zone-constraints-for-show-create-table", p.txn, - fmt.Sprintf(configStmtsQuery, contextName)) - if err != nil { - log.VEventf(ctx, 1, "%q", err) + parentNameStr := tree.DNull + if db != nil { + contextName = db.Name + parentNameStr = tree.NewDString(db.Name) + } + scNameStr := tree.NewDString(scName) + + var descType tree.Datum + var stmt, createNofk string + alterStmts := tree.NewDArray(types.String) + validateStmts := tree.NewDArray(types.String) + var err error + if table.IsView() { + descType = typeView + stmt, err = ShowCreateView(ctx, (*tree.Name)(&table.Name), table) + } else if table.IsSequence() { + descType = typeSequence + stmt, err = ShowCreateSequence(ctx, (*tree.Name)(&table.Name), table) } else { - for _, row := range zoneConstraintRows { - tableName := string(tree.MustBeDString(row[0])) - var zoneConfig zonepb.ZoneConfig - yamlString := string(tree.MustBeDString(row[1])) - err := yaml.UnmarshalStrict([]byte(yamlString), &zoneConfig) - if err != nil { - return err - } - // If all constraints are default, then don't show anything. - if !zoneConfig.Equal(zonepb.ZoneConfig{}) { - sqlString := string(tree.MustBeDString(row[2])) - zoneConfigStmts[tableName] = append(zoneConfigStmts[tableName], sqlString) - } + descType = typeTable + tn := (*tree.Name)(&table.Name) + createNofk, err = ShowCreateTable(ctx, p, tn, contextName, table, lookup, OmitFKClausesFromCreate) + if err != nil { + return err + } + allIdx := append(table.Indexes, table.PrimaryIndex) + if err := showAlterStatementWithInterleave(ctx, tn, contextName, lookup, allIdx, table, alterStmts, + validateStmts); err != nil { + return err } + stmt, err = ShowCreateTable(ctx, p, tn, contextName, table, lookup, IncludeFkClausesInCreate) + } + if err != nil { + return err } - return forEachTableDescWithTableLookupInternal(ctx, p, dbContext, virtualOnce, true, /*allowAdding*/ - func(db *DatabaseDescriptor, scName string, table *TableDescriptor, lCtx tableLookupFn) error { - parentNameStr := tree.DNull - if db != nil { - parentNameStr = tree.NewDString(db.Name) - } - scNameStr := tree.NewDString(scName) - - var descType tree.Datum - var stmt, createNofk string - alterStmts := tree.NewDArray(types.String) - validateStmts := tree.NewDArray(types.String) - var err error - if table.IsView() { - descType = typeView - stmt, err = ShowCreateView(ctx, (*tree.Name)(&table.Name), table) - } else if table.IsSequence() { - descType = typeSequence - stmt, err = ShowCreateSequence(ctx, (*tree.Name)(&table.Name), table) - } else { - descType = typeTable - tn := (*tree.Name)(&table.Name) - createNofk, err = ShowCreateTable(ctx, p, tn, contextName, table, lCtx, OmitFKClausesFromCreate) - if err != nil { - return err - } - allIdx := append(table.Indexes, table.PrimaryIndex) - if err := showAlterStatementWithInterleave(ctx, tn, contextName, lCtx, allIdx, table, alterStmts, validateStmts); err != nil { - return err - } - stmt, err = ShowCreateTable(ctx, p, tn, contextName, table, lCtx, IncludeFkClausesInCreate) - } - if err != nil { - return err - } - - zoneRows := tree.NewDArray(types.String) - if val, ok := zoneConfigStmts[table.Name]; ok { - for _, s := range val { - if err := zoneRows.Append(tree.NewDString(s)); err != nil { - return err - } - } - } else { - // If there are partitions applied to this table and no zone configurations, display a warning. - hasPartitions := false - for i := range table.Indexes { - if table.Indexes[i].Partitioning.NumColumns != 0 { - hasPartitions = true - break - } - } - hasPartitions = hasPartitions || table.PrimaryIndex.Partitioning.NumColumns != 0 - if hasPartitions { - stmt += "\n-- Warning: Partitioned table with no zone configurations." - } - } - - descID := tree.NewDInt(tree.DInt(table.ID)) - dbDescID := tree.NewDInt(tree.DInt(table.GetParentID())) - if createNofk == "" { - createNofk = stmt - } - return addRow( - dbDescID, - parentNameStr, - scNameStr, - descID, - descType, - tree.NewDString(table.Name), - tree.NewDString(stmt), - tree.NewDString(table.State.String()), - tree.NewDString(createNofk), - alterStmts, - validateStmts, - zoneRows, - ) - }) - }, -} + descID := tree.NewDInt(tree.DInt(table.ID)) + dbDescID := tree.NewDInt(tree.DInt(table.GetParentID())) + if createNofk == "" { + createNofk = stmt + } + hasPartitions := false + for i := range table.Indexes { + if table.Indexes[i].Partitioning.NumColumns != 0 { + hasPartitions = true + break + } + } + hasPartitions = hasPartitions || table.PrimaryIndex.Partitioning.NumColumns != 0 + return addRow( + dbDescID, + parentNameStr, + scNameStr, + descID, + descType, + tree.NewDString(table.Name), + tree.NewDString(stmt), + tree.NewDString(table.State.String()), + tree.NewDString(createNofk), + alterStmts, + validateStmts, + tree.MakeDBool(tree.DBool(hasPartitions)), + ) + }) func showAlterStatementWithInterleave( ctx context.Context, tn *tree.Name, contextName string, - lCtx tableLookupFn, + lCtx simpleSchemaResolver, allIdx []sqlbase.IndexDescriptor, table *sqlbase.TableDescriptor, alterStmts *tree.DArray, diff --git a/pkg/sql/delegate/show_table.go b/pkg/sql/delegate/show_table.go index 5d8904bcbd95..c672c4203d0c 100644 --- a/pkg/sql/delegate/show_table.go +++ b/pkg/sql/delegate/show_table.go @@ -21,23 +21,30 @@ import ( func (d *delegator) delegateShowCreate(n *tree.ShowCreate) (tree.Statement, error) { sqltelemetry.IncrementShowCounter(sqltelemetry.Create) + const showCreateQuery = ` + WITH zone_configs AS ( + SELECT string_agg(raw_config_sql, e';\n') FROM crdb_internal.zones + WHERE database_name = %[1]s + AND table_name = %[2]s + AND raw_config_yaml IS NOT NULL + AND raw_config_sql IS NOT NULL + ) SELECT %[3]s AS table_name, - array_to_string( - array_cat( - ARRAY[create_statement], - zone_configuration_statements - ), - e';\n' - ) - AS create_statement + concat(create_statement, + CASE + WHEN NOT has_partitions + THEN NULL + WHEN (SELECT * FROM zone_configs) IS NULL + THEN e'\n-- Warning: Partitioned table with no zone configurations.' + ELSE concat(e';\n', (SELECT * FROM zone_configs)) + END + ) AS create_statement FROM %[4]s.crdb_internal.create_statements WHERE - (database_name IS NULL OR database_name = %[1]s) - AND schema_name = %[5]s - AND descriptor_name = %[2]s + descriptor_id = %[6]d ` return d.showTableDetails(n.Name, showCreateQuery) @@ -164,6 +171,7 @@ func (d *delegator) delegateShowConstraints(n *tree.ShowConstraints) (tree.State // %[3]s the given table name as SQL string literal. // %[4]s the database name as SQL identifier. // %[5]s the schema name as SQL string literal. +// %[6]s the table ID. func (d *delegator) showTableDetails( name *tree.UnresolvedObjectName, query string, ) (tree.Statement, error) { @@ -185,7 +193,7 @@ func (d *delegator) showTableDetails( lex.EscapeSQLString(resName.String()), resName.CatalogName.String(), // note: CatalogName.String() != Catalog() lex.EscapeSQLString(resName.Schema()), - dataSource.ID(), + dataSource.PostgresDescriptorID(), ) return parse(fullQuery) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index d287bebf384f..14943c69bb35 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -187,7 +187,7 @@ function signature category details query ITTITTTTTTTT colnames SELECT * FROM crdb_internal.create_statements WHERE database_name = '' ---- -database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements zone_configuration_statements +database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions query ITITTBTB colnames SELECT * FROM crdb_internal.table_columns WHERE descriptor_name = '' diff --git a/pkg/sql/logictest/testdata/logic_test/sequences b/pkg/sql/logictest/testdata/logic_test/sequences index b8b55d407804..f7e2e7681ede 100644 --- a/pkg/sql/logictest/testdata/logic_test/sequences +++ b/pkg/sql/logictest/testdata/logic_test/sequences @@ -113,11 +113,11 @@ CREATE SEQUENCE ignored_options_test CACHE 1 NO CYCLE statement ok CREATE SEQUENCE show_create_test -query ITTITTTTTTTT colnames +query ITTITTTTTTTB colnames SELECT * FROM crdb_internal.create_statements WHERE descriptor_name = 'show_create_test' ---- -database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements zone_configuration_statements -52 test public 66 sequence show_create_test CREATE SEQUENCE show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 PUBLIC CREATE SEQUENCE show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 {} {} {} +database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions +52 test public 66 sequence show_create_test CREATE SEQUENCE show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 PUBLIC CREATE SEQUENCE show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 {} {} false query TT colnames SHOW CREATE SEQUENCE show_create_test diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go index 27767caed98d..a373f38cb02e 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/partition_test.go @@ -60,9 +60,14 @@ func TestRemovePartitioningOSS(t *testing.T) { ToExclusive: encoding.EncodeIntValue(nil /* appendTo */, encoding.NoColumnID, 2), }}, } + // Note that this is really a gross hack - it breaks planner caches, which + // assume that nothing is going to change out from under them like this. We + // "fix" the issue by altering the table's name to refresh the cache, below. if err := kvDB.Put(ctx, tableKey, sqlbase.WrapDescriptor(tableDesc)); err != nil { t.Fatal(err) } + sqlDB.Exec(t, "ALTER TABLE t.kv RENAME to t.kv2") + sqlDB.Exec(t, "ALTER TABLE t.kv2 RENAME to t.kv") exp := `CREATE TABLE kv ( k INT8 NOT NULL, v INT8 NULL,