From cd1a7bb06352121268f563a6ae46f0b55eeba7d3 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Thu, 4 Mar 2021 14:59:45 -0500 Subject: [PATCH] sql: internal tables for cross db refs and interleaved indexes/tables Fixes: #58867 Previously, users had no way of determining which objects in their database utilized either deprecated features like interleaved indexes/tables or cross DB references. This was inadequate since users need to know which object utilize this functionality. To address this, this patch introduces the crdb_internal tables: cross_db_references, interleaved, Release justification: Low risk internal tables for detecting deprecated features. Release note (sql change): Added crdb_internal tables cross_db_references and interleaved for detecting the deprecated features cross db references and interleaved tables / indexes. --- pkg/cli/zip_test.go | 2 + pkg/sql/catalog/catconstants/constants.go | 2 + pkg/sql/crdb_internal.go | 233 +++++++ .../testdata/logic_test/crdb_internal | 2 + .../testdata/logic_test/crdb_internal_tenant | 2 + .../testdata/logic_test/create_statements | 30 + pkg/sql/logictest/testdata/logic_test/fk | 9 + .../logictest/testdata/logic_test/grant_table | 2 + .../testdata/logic_test/information_schema | 10 + .../logictest/testdata/logic_test/interleaved | 43 ++ .../logictest/testdata/logic_test/pg_catalog | 570 +++++++++--------- .../logictest/testdata/logic_test/sequences | 17 + pkg/sql/logictest/testdata/logic_test/table | 2 + pkg/sql/logictest/testdata/logic_test/views | 34 ++ 14 files changed, 674 insertions(+), 284 deletions(-) diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index f9a6f4efcbe4..c4438e699bc5 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -68,9 +68,11 @@ table_name NOT IN ( 'builtin_functions', 'create_statements', 'create_type_statements', + 'cross_db_references', 'databases', 'forward_dependencies', 'index_columns', + 'interleaved_indexes', 'table_columns', 'table_indexes', 'table_row_statistics', diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index 42888507b855..89336f65cce5 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -79,6 +79,8 @@ const ( CrdbInternalZonesTableID CrdbInternalInvalidDescriptorsTableID CrdbInternalClusterDatabasePrivilegesTableID + CrdbInternalInterleaved + CrdbInternalCrossDbRefrences InformationSchemaID InformationSchemaAdministrableRoleAuthorizationsID InformationSchemaApplicableRolesID diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 1015e2e4759a..9a89c730c67f 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -132,6 +132,8 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalZonesTableID: crdbInternalZonesTable, catconstants.CrdbInternalInvalidDescriptorsTableID: crdbInternalInvalidDescriptorsTable, catconstants.CrdbInternalClusterDatabasePrivilegesTableID: crdbInternalClusterDatabasePrivilegesTable, + catconstants.CrdbInternalInterleaved: crdbInternalInterleaved, + catconstants.CrdbInternalCrossDbRefrences: crdbInternalCrossDbReferences, }, validWithNoDatabaseContext: true, } @@ -4024,3 +4026,234 @@ CREATE TABLE crdb_internal.cluster_database_privileges ( }) }, } + +var crdbInternalInterleaved = virtualSchemaTable{ + comment: `virtual table with interleaved table information`, + schema: ` +CREATE TABLE crdb_internal.interleaved ( + database_name + STRING NOT NULL, + schema_name + STRING NOT NULL, + table_name + STRING NOT NULL, + index_name + STRING NOT NULL, + parent_index + STRING NOT NULL +);`, + populate: func(ctx context.Context, p *planner, dbContext *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return forEachTableDescAllWithTableLookup(ctx, p, dbContext, hideVirtual, + func(db *dbdesc.Immutable, schemaName string, table catalog.TableDescriptor, lookupFn tableLookupFn) error { + if !table.IsInterleaved() { + return nil + } + indexes := table.NonDropIndexes() + for _, index := range indexes { + if index.NumInterleaveAncestors() == 0 { + continue + } + + ancestor := index.GetInterleaveAncestor(index.NumInterleaveAncestors() - 1) + parentTable, err := lookupFn.getTableByID(ancestor.TableID) + if err != nil { + return err + } + parentIndex, err := parentTable.FindIndexWithID(ancestor.IndexID) + if err != nil { + return err + } + parentSchemaName := "" + if parentTable.GetParentSchemaID() == keys.PublicSchemaID { + parentSchemaName = tree.PublicSchema + } else { + schema, err := lookupFn.getSchemaByID(parentTable.GetParentSchemaID()) + if err != nil { + return err + } + parentSchemaName = schema.GetName() + } + database, err := lookupFn.getDatabaseByID(parentTable.GetParentID()) + if err != nil { + return err + } + + if err := addRow(tree.NewDString(database.GetName()), + tree.NewDString(parentSchemaName), + tree.NewDString(table.GetName()), + tree.NewDString(index.GetName()), + tree.NewDString(parentIndex.GetName())); err != nil { + return err + } + } + return nil + }) + }, +} + +var crdbInternalCrossDbReferences = virtualSchemaTable{ + comment: `virtual table with cross db references`, + schema: ` +CREATE TABLE crdb_internal.cross_db_references ( + object_database + STRING NOT NULL, + object_schema + STRING NOT NULL, + object_name + STRING NOT NULL, + referenced_object_database + STRING NOT NULL, + referenced_object_schema + STRING NOT NULL, + referenced_object_name + STRING NOT NULL, + cross_database_reference_description + STRING NOT NULL +);`, + populate: func(ctx context.Context, p *planner, dbContext *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return forEachTableDescAllWithTableLookup(ctx, p, dbContext, hideVirtual, + func(db *dbdesc.Immutable, schemaName string, table catalog.TableDescriptor, lookupFn tableLookupFn) error { + // For tables detect if foreign key references point at a different + // database. Additionally, check if any of the columns have sequence + // references to a different database. + if table.IsTable() { + objectDatabaseName := lookupFn.getParentName(table) + err := table.ForeachOutboundFK( + func(fk *descpb.ForeignKeyConstraint) error { + referencedTable, err := lookupFn.getTableByID(fk.ReferencedTableID) + if err != nil { + return err + } + if referencedTable.GetParentID() != table.GetParentID() { + refSchemaName := "" + if referencedTable.GetParentSchemaID() == keys.PublicSchemaID { + refSchemaName = tree.PublicSchema + } else { + schema, err := lookupFn.getSchemaByID(referencedTable.GetParentSchemaID()) + if err != nil { + return err + } + refSchemaName = schema.GetName() + } + refDatabaseName := lookupFn.getParentName(referencedTable) + + if err := addRow(tree.NewDString(objectDatabaseName), + tree.NewDString(schemaName), + tree.NewDString(table.GetName()), + tree.NewDString(refDatabaseName), + tree.NewDString(refSchemaName), + tree.NewDString(referencedTable.GetName()), + tree.NewDString("table foreign key reference")); err != nil { + return err + } + } + return nil + }) + if err != nil { + return err + } + + // Check for sequence dependencies + for _, col := range table.PublicColumns() { + for i := 0; i < col.NumUsesSequences(); i++ { + sequenceID := col.GetUsesSequenceID(i) + seqDesc, err := lookupFn.getTableByID(sequenceID) + if err != nil { + return err + } + if seqDesc.GetParentID() != table.GetParentID() { + seqSchemaName := "" + if seqDesc.GetParentSchemaID() == keys.PublicSchemaID { + seqSchemaName = tree.PublicSchema + } else { + schema, err := lookupFn.getSchemaByID(seqDesc.GetParentSchemaID()) + if err != nil { + return err + } + seqSchemaName = schema.GetName() + } + refDatabaseName := lookupFn.getParentName(seqDesc) + if err := addRow(tree.NewDString(objectDatabaseName), + tree.NewDString(schemaName), + tree.NewDString(table.GetName()), + tree.NewDString(refDatabaseName), + tree.NewDString(seqSchemaName), + tree.NewDString(seqDesc.GetName()), + tree.NewDString("table column refers to sequence")); err != nil { + return err + } + } + } + } + } else if table.IsView() { + // For views check if we depend on tables in a different database. + dependsOn := table.GetDependsOn() + for _, dependency := range dependsOn { + dependentTable, err := lookupFn.getTableByID(dependency) + if err != nil { + return err + } + if dependentTable.GetParentID() != table.GetParentID() { + objectDatabaseName := lookupFn.getParentName(table) + refSchemaName := "" + if dependentTable.GetParentSchemaID() == keys.PublicSchemaID { + refSchemaName = tree.PublicSchema + } else { + schema, err := lookupFn.getSchemaByID(dependentTable.GetParentSchemaID()) + if err != nil { + return err + } + refSchemaName = schema.GetName() + } + refDatabaseName := lookupFn.getParentName(dependentTable) + + if err := addRow(tree.NewDString(objectDatabaseName), + tree.NewDString(schemaName), + tree.NewDString(table.GetName()), + tree.NewDString(refDatabaseName), + tree.NewDString(refSchemaName), + tree.NewDString(dependentTable.GetName()), + tree.NewDString("view references table")); err != nil { + return err + } + } + } + } else if table.IsSequence() { + // For sequences check if the sequence is owned by + // a different database. + sequenceOpts := table.GetSequenceOpts() + if sequenceOpts.SequenceOwner.OwnerTableID != descpb.InvalidID { + ownerTable, err := lookupFn.getTableByID(sequenceOpts.SequenceOwner.OwnerTableID) + if err != nil { + return err + } + if ownerTable.GetParentID() != table.GetParentID() { + objectDatabaseName := lookupFn.getParentName(table) + refSchemaName := "" + if ownerTable.GetParentSchemaID() == keys.PublicSchemaID { + refSchemaName = tree.PublicSchema + } else { + schema, err := lookupFn.getSchemaByID(ownerTable.GetParentSchemaID()) + if err != nil { + return err + } + refSchemaName = schema.GetName() + } + refDatabaseName := lookupFn.getParentName(ownerTable) + + if err := addRow(tree.NewDString(objectDatabaseName), + tree.NewDString(schemaName), + tree.NewDString(table.GetName()), + tree.NewDString(refDatabaseName), + tree.NewDString(refSchemaName), + tree.NewDString(ownerTable.GetName()), + tree.NewDString("sequences owning table")); err != nil { + return err + } + } + } + } + return nil + }) + }, +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index cbcf98546aa6..b7d58ac3633d 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -24,6 +24,7 @@ crdb_internal cluster_settings table NULL NULL NULL crdb_internal cluster_transactions table NULL NULL NULL crdb_internal create_statements table NULL NULL NULL crdb_internal create_type_statements table NULL NULL NULL +crdb_internal cross_db_references table NULL NULL NULL crdb_internal databases table NULL NULL NULL crdb_internal feature_usage table NULL NULL NULL crdb_internal forward_dependencies table NULL NULL NULL @@ -32,6 +33,7 @@ crdb_internal gossip_liveness table NULL NULL NULL crdb_internal gossip_network table NULL NULL NULL crdb_internal gossip_nodes table NULL NULL NULL crdb_internal index_columns table NULL NULL NULL +crdb_internal interleaved_indexes table NULL NULL NULL crdb_internal invalid_objects table NULL NULL NULL crdb_internal jobs table NULL NULL NULL crdb_internal kv_node_status table NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index 3989a10eea66..b8f5210ae6bf 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -36,6 +36,7 @@ crdb_internal cluster_settings table NULL NULL NULL crdb_internal cluster_transactions table NULL NULL NULL crdb_internal create_statements table NULL NULL NULL crdb_internal create_type_statements table NULL NULL NULL +crdb_internal cross_db_references table NULL NULL NULL crdb_internal databases table NULL NULL NULL crdb_internal feature_usage table NULL NULL NULL crdb_internal forward_dependencies table NULL NULL NULL @@ -44,6 +45,7 @@ crdb_internal gossip_liveness table NULL NULL NULL crdb_internal gossip_network table NULL NULL NULL crdb_internal gossip_nodes table NULL NULL NULL crdb_internal index_columns table NULL NULL NULL +crdb_internal interleaved_indexes table NULL NULL NULL crdb_internal invalid_objects table NULL NULL NULL crdb_internal jobs table NULL NULL NULL crdb_internal kv_node_status table NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index ffba1ef833c5..544fcd52ddc8 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -230,6 +230,23 @@ CREATE TABLE crdb_internal.create_type_statements ( enum_members STRING[] NULL, INDEX create_type_statements_descriptor_id_idx (descriptor_id ASC) STORING (database_id, database_name, schema_name, descriptor_name, create_statement, enum_members) ) {} {} +CREATE TABLE crdb_internal.cross_db_references ( + object_database STRING NOT NULL, + object_schema STRING NOT NULL, + object_name STRING NOT NULL, + referenced_object_database STRING NOT NULL, + referenced_object_schema STRING NOT NULL, + referenced_object_name STRING NOT NULL, + cross_database_reference_description STRING NOT NULL +) CREATE TABLE crdb_internal.cross_db_references ( + object_database STRING NOT NULL, + object_schema STRING NOT NULL, + object_name STRING NOT NULL, + referenced_object_database STRING NOT NULL, + referenced_object_schema STRING NOT NULL, + referenced_object_name STRING NOT NULL, + cross_database_reference_description STRING NOT NULL +) {} {} CREATE TABLE crdb_internal.databases ( id INT8 NOT NULL, name STRING NOT NULL, @@ -364,6 +381,19 @@ CREATE TABLE crdb_internal.index_columns ( column_direction STRING NULL, implicit BOOL NULL ) {} {} +CREATE TABLE crdb_internal.interleaved ( +database_name STRING NOT NULL, +schema_name STRING NOT NULL, +table_name STRING NOT NULL, +index_name STRING NOT NULL, +parent_index STRING NOT NULL +) CREATE TABLE crdb_internal.interleaved ( +database_name STRING NOT NULL, +schema_name STRING NOT NULL, +table_name STRING NOT NULL, +index_name STRING NOT NULL, +parent_index STRING NOT NULL +) {} {} CREATE TABLE crdb_internal.invalid_objects ( id INT8 NULL, database_name STRING NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/fk b/pkg/sql/logictest/testdata/logic_test/fk index c4b1797e5653..3e692b61097d 100644 --- a/pkg/sql/logictest/testdata/logic_test/fk +++ b/pkg/sql/logictest/testdata/logic_test/fk @@ -3633,6 +3633,15 @@ USE db2 statement ok CREATE TABLE child2 (c INT PRIMARY KEY, p INT REFERENCES db1.public.parent(p)) + +# Validate that cross DB foreign keys are detected by internal tables +query TTTTTTT +select * from "".crdb_internal.cross_db_references; +---- +db2 public child db1 public parent table foreign key reference +db2 public child2 db1 public parent table foreign key reference + + # Test that foreign keys cannot reference columns that are indexed by a partial # unique index or a partial unique constraint. Partial unique indexes and # constraints do not guarantee uniqueness in the entire table. diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 0455ae381bed..b542fa1dae6a 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -36,6 +36,7 @@ test crdb_internal cluster_settings pu test crdb_internal cluster_transactions public SELECT test crdb_internal create_statements public SELECT test crdb_internal create_type_statements public SELECT +test crdb_internal cross_db_references public SELECT test crdb_internal databases public SELECT test crdb_internal feature_usage public SELECT test crdb_internal forward_dependencies public SELECT @@ -44,6 +45,7 @@ test crdb_internal gossip_liveness pu test crdb_internal gossip_network public SELECT test crdb_internal gossip_nodes public SELECT test crdb_internal index_columns public SELECT +test crdb_internal interleaved_indexes public SELECT test crdb_internal invalid_objects public SELECT test crdb_internal jobs public SELECT test crdb_internal kv_node_status public SELECT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 4a99d13b92e5..fcd3eac36fbe 100755 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -288,6 +288,7 @@ crdb_internal cluster_settings crdb_internal cluster_transactions crdb_internal create_statements crdb_internal create_type_statements +crdb_internal cross_db_references crdb_internal databases crdb_internal feature_usage crdb_internal forward_dependencies @@ -296,6 +297,7 @@ crdb_internal gossip_liveness crdb_internal gossip_network crdb_internal gossip_nodes crdb_internal index_columns +crdb_internal interleaved_indexes crdb_internal invalid_objects crdb_internal jobs crdb_internal kv_node_status @@ -597,6 +599,7 @@ cluster_settings cluster_transactions create_statements create_type_statements +cross_db_references databases feature_usage forward_dependencies @@ -605,6 +608,7 @@ gossip_liveness gossip_network gossip_nodes index_columns +interleaved_indexes invalid_objects jobs kv_node_status @@ -925,6 +929,7 @@ system crdb_internal cluster_settings SY system crdb_internal cluster_transactions SYSTEM VIEW NO 1 system crdb_internal create_statements SYSTEM VIEW NO 1 system crdb_internal create_type_statements SYSTEM VIEW NO 1 +system crdb_internal cross_db_references SYSTEM VIEW NO 1 system crdb_internal databases SYSTEM VIEW NO 1 system crdb_internal feature_usage SYSTEM VIEW NO 1 system crdb_internal forward_dependencies SYSTEM VIEW NO 1 @@ -933,6 +938,7 @@ system crdb_internal gossip_liveness SY system crdb_internal gossip_network SYSTEM VIEW NO 1 system crdb_internal gossip_nodes SYSTEM VIEW NO 1 system crdb_internal index_columns SYSTEM VIEW NO 1 +system crdb_internal interleaved_indexes SYSTEM VIEW NO 1 system crdb_internal invalid_objects SYSTEM VIEW NO 1 system crdb_internal jobs SYSTEM VIEW NO 1 system crdb_internal kv_node_status SYSTEM VIEW NO 1 @@ -2335,6 +2341,7 @@ NULL public system crdb_internal cluster_settings NULL public system crdb_internal cluster_transactions SELECT NULL YES NULL public system crdb_internal create_statements SELECT NULL YES NULL public system crdb_internal create_type_statements SELECT NULL YES +NULL public system crdb_internal cross_db_references SELECT NULL YES NULL public system crdb_internal databases SELECT NULL YES NULL public system crdb_internal feature_usage SELECT NULL YES NULL public system crdb_internal forward_dependencies SELECT NULL YES @@ -2343,6 +2350,7 @@ NULL public system crdb_internal gossip_liveness NULL public system crdb_internal gossip_network SELECT NULL YES NULL public system crdb_internal gossip_nodes SELECT NULL YES NULL public system crdb_internal index_columns SELECT NULL YES +NULL public system crdb_internal interleaved_indexes SELECT NULL YES NULL public system crdb_internal invalid_objects SELECT NULL YES NULL public system crdb_internal jobs SELECT NULL YES NULL public system crdb_internal kv_node_status SELECT NULL YES @@ -2879,6 +2887,7 @@ NULL public system crdb_internal cluster_settings NULL public system crdb_internal cluster_transactions SELECT NULL YES NULL public system crdb_internal create_statements SELECT NULL YES NULL public system crdb_internal create_type_statements SELECT NULL YES +NULL public system crdb_internal cross_db_references SELECT NULL YES NULL public system crdb_internal databases SELECT NULL YES NULL public system crdb_internal feature_usage SELECT NULL YES NULL public system crdb_internal forward_dependencies SELECT NULL YES @@ -2887,6 +2896,7 @@ NULL public system crdb_internal gossip_liveness NULL public system crdb_internal gossip_network SELECT NULL YES NULL public system crdb_internal gossip_nodes SELECT NULL YES NULL public system crdb_internal index_columns SELECT NULL YES +NULL public system crdb_internal interleaved_indexes SELECT NULL YES NULL public system crdb_internal invalid_objects SELECT NULL YES NULL public system crdb_internal jobs SELECT NULL YES NULL public system crdb_internal kv_node_status SELECT NULL YES diff --git a/pkg/sql/logictest/testdata/logic_test/interleaved b/pkg/sql/logictest/testdata/logic_test/interleaved index 0e0466f0c4b6..fc8ae12508f8 100644 --- a/pkg/sql/logictest/testdata/logic_test/interleaved +++ b/pkg/sql/logictest/testdata/logic_test/interleaved @@ -453,6 +453,7 @@ CREATE TABLE interleave_parent (x INT PRIMARY KEY); statement notice NOTICE: interleaved tables and interleaved indexes are deprecated in 20.2 and will be removed in 21.2\nHINT: .*52009.* CREATE TABLE interleave_create_notice (x INT, y INT, s STRING, PRIMARY KEY (x, y)) INTERLEAVE IN PARENT interleave_parent(x); + statement notice NOTICE: interleaved tables and interleaved indexes are deprecated in 20.2 and will be removed in 21.2\nHINT: .*52009.* CREATE INDEX interleave_index ON interleave_create_notice (x, s) INTERLEAVE IN PARENT interleave_parent(x); @@ -473,3 +474,45 @@ CREATE INDEX willfail ON interleave_create_notice (x, s) INTERLEAVE IN PARENT in statement error interleaved tables and interleaved indexes are disabled ALTER TABLE interleave_pk_notice ALTER PRIMARY KEY USING COLUMNS (x) INTERLEAVE IN PARENT interleave_parent(x); + +statement ok +SET CLUSTER SETTING sql.defaults.interleaved_tables.enabled = true + +# Interleaved index as non-primary one check +statement ok +CREATE TABLE parent(x int, y int); + +statement ok +CREATE TABLE child(x int, y int, z int ); + +statement ok +CREATE INDEX NIINDX ON PARENT(x,y); + +statement ok +CREATE INDEX NIINDX2 ON CHILD(z); + +statement ok +CREATE INDEX NIINDX3 ON CHILD(z); + +statement ok +CREATE INDEX IINDX ON CHILD(rowid,x,y,z) INTERLEAVE IN PARENT PARENT(rowid); + + + +query TTTTT +select * from "".crdb_internal.interleaved; +---- +test public p1_1 p1_id primary +test public all_interleaves primary primary +test public all_interleaves all_interleaves_c_d_idx primary +test public all_interleaves all_interleaves_d_c_key primary +test public orders primary primary +other public interdb primary primary +test public c20067 primary primary +test public documents primary primary +test public big_interleave_parent primary primary +test public big_interleave_child primary primary +test public interleave_create_notice primary primary +test public interleave_create_notice interleave_index primary +test public interleave_pk_notice primary primary +test public child iindx primary diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 353355fe0dad..756fa6d67487 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1354,14 +1354,14 @@ FROM pg_catalog.pg_depend ORDER BY objid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967192 58 0 4294967192 55 1 n -4294967192 58 0 4294967192 55 2 n -4294967192 58 0 4294967192 55 3 n -4294967192 58 0 4294967192 55 4 n -4294967184 2143281868 0 4294967192 450499961 0 n -4294967184 2355671820 0 4294967192 0 0 n -4294967184 3911002394 0 4294967192 0 0 n -4294967184 4089604113 0 4294967192 450499960 0 n +4294967190 58 0 4294967190 55 1 n +4294967190 58 0 4294967190 55 2 n +4294967190 58 0 4294967190 55 3 n +4294967190 58 0 4294967190 55 4 n +4294967182 2143281868 0 4294967190 450499961 0 n +4294967182 2355671820 0 4294967190 0 0 n +4294967182 3911002394 0 4294967190 0 0 n +4294967182 4089604113 0 4294967190 450499960 0 n # Some entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. Other entries are links to pg_class when it is @@ -1374,8 +1374,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967192 4294967192 pg_class pg_class -4294967184 4294967192 pg_constraint pg_class +4294967190 4294967190 pg_class pg_class +4294967182 4294967190 pg_constraint pg_class # Some entries in pg_depend are foreign key constraints that reference an index # in pg_class. Other entries are table-view dependencies @@ -2083,279 +2083,281 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967294 4294967192 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967192 0 built-in functions (RAM/static) -4294967291 4294967192 0 contention information (cluster RPC; expensive!) -4294967249 4294967192 0 virtual table with database privileges -4294967290 4294967192 0 running queries visible by current user (cluster RPC; expensive!) -4294967288 4294967192 0 running sessions visible to current user (cluster RPC; expensive!) -4294967287 4294967192 0 cluster settings (RAM) -4294967289 4294967192 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967286 4294967192 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967285 4294967192 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967284 4294967192 0 databases accessible by the current user (KV scan) -4294967283 4294967192 0 telemetry counters (RAM; local node only) -4294967282 4294967192 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967280 4294967192 0 locally known gossiped health alerts (RAM; local node only) -4294967279 4294967192 0 locally known gossiped node liveness (RAM; local node only) -4294967278 4294967192 0 locally known edges in the gossip network (RAM; local node only) -4294967281 4294967192 0 locally known gossiped node details (RAM; local node only) -4294967277 4294967192 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967250 4294967192 0 virtual table to validate descriptors -4294967275 4294967192 0 decoded job metadata from system.jobs (KV scan) -4294967274 4294967192 0 node details across the entire cluster (cluster RPC; expensive!) -4294967273 4294967192 0 store details and status (cluster RPC; expensive!) -4294967272 4294967192 0 acquired table leases (RAM; local node only) -4294967293 4294967192 0 detailed identification strings (RAM, local node only) -4294967271 4294967192 0 contention information (RAM; local node only) -4294967276 4294967192 0 in-flight spans (RAM; local node only) -4294967267 4294967192 0 current values for metrics (RAM; local node only) -4294967270 4294967192 0 running queries visible by current user (RAM; local node only) -4294967262 4294967192 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967268 4294967192 0 running sessions visible by current user (RAM; local node only) -4294967258 4294967192 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967253 4294967192 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967269 4294967192 0 running user transactions visible by the current user (RAM; local node only) -4294967252 4294967192 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967266 4294967192 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967265 4294967192 0 comments for predefined virtual tables (RAM/static) -4294967264 4294967192 0 range metadata without leaseholder details (KV join; expensive!) -4294967261 4294967192 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967260 4294967192 0 session trace accumulated so far (RAM) -4294967259 4294967192 0 session variables (RAM) -4294967257 4294967192 0 details for all columns accessible by current user in current database (KV scan) -4294967256 4294967192 0 indexes accessible by current user in current database (KV scan) -4294967254 4294967192 0 stats for all tables accessible by current user in current database as of 10s ago -4294967255 4294967192 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967251 4294967192 0 decoded zone configurations from system.zones (KV scan) -4294967247 4294967192 0 roles for which the current user has admin option -4294967246 4294967192 0 roles available to the current user -4294967245 4294967192 0 character sets available in the current database -4294967244 4294967192 0 check constraints -4294967243 4294967192 0 identifies which character set the available collations are -4294967242 4294967192 0 shows the collations available in the current database -4294967241 4294967192 0 column privilege grants (incomplete) -4294967239 4294967192 0 columns with user defined types -4294967240 4294967192 0 table and view columns (incomplete) -4294967238 4294967192 0 columns usage by constraints -4294967237 4294967192 0 roles for the current user -4294967236 4294967192 0 column usage by indexes and key constraints -4294967235 4294967192 0 built-in function parameters (empty - introspection not yet supported) -4294967234 4294967192 0 foreign key constraints -4294967233 4294967192 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967232 4294967192 0 built-in functions (empty - introspection not yet supported) -4294967230 4294967192 0 schema privileges (incomplete; may contain excess users or roles) -4294967231 4294967192 0 database schemas (may contain schemata without permission) -4294967228 4294967192 0 sequences -4294967229 4294967192 0 exposes the session variables. -4294967227 4294967192 0 index metadata and statistics (incomplete) -4294967226 4294967192 0 table constraints -4294967225 4294967192 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967224 4294967192 0 tables and views -4294967223 4294967192 0 type privileges (incomplete; may contain excess users or roles) -4294967221 4294967192 0 grantable privileges (incomplete) -4294967222 4294967192 0 views (incomplete) -4294967218 4294967192 0 aggregated built-in functions (incomplete) -4294967219 4294967192 0 pg_aggregate_fnoid_index was created for compatibility and is currently unimplemented -4294967216 4294967192 0 index access methods (incomplete) -4294967217 4294967192 0 pg_am_oid_index was created for compatibility and is currently unimplemented -4294967212 4294967192 0 pg_amop was created for compatibility and is currently unimplemented -4294967215 4294967192 0 pg_amop_fam_strat_index was created for compatibility and is currently unimplemented -4294967214 4294967192 0 pg_amop_oid_index was created for compatibility and is currently unimplemented -4294967213 4294967192 0 pg_amop_opr_fam_index was created for compatibility and is currently unimplemented -4294967209 4294967192 0 pg_amproc was created for compatibility and is currently unimplemented -4294967211 4294967192 0 pg_amproc_fam_proc_index was created for compatibility and is currently unimplemented -4294967210 4294967192 0 pg_amproc_oid_index was created for compatibility and is currently unimplemented -4294967208 4294967192 0 column default values -4294967207 4294967192 0 pg_attrdef_adrelid_adnum_index was created for compatibility and is currently unimplemented -4294967206 4294967192 0 pg_attrdef_oid_index was created for compatibility and is currently unimplemented -4294967204 4294967192 0 table columns (incomplete - see also information_schema.columns) -4294967205 4294967192 0 pg_attribute_relid_attnum_index was created for compatibility and is currently unimplemented -4294967200 4294967192 0 role membership -4294967202 4294967192 0 pg_auth_members_member_role_index was created for compatibility and is currently unimplemented -4294967201 4294967192 0 pg_auth_members_role_member_index was created for compatibility and is currently unimplemented -4294967203 4294967192 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967199 4294967192 0 pg_authid_oid_index was created for compatibility and is currently unimplemented -4294967198 4294967192 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967197 4294967192 0 available extensions -4294967194 4294967192 0 casts (empty - needs filling out) -4294967196 4294967192 0 pg_cast_oid_index was created for compatibility and is currently unimplemented -4294967195 4294967192 0 pg_cast_source_target_index was created for compatibility and is currently unimplemented -4294967192 4294967192 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967193 4294967192 0 pg_class_oid_index was created for compatibility and is currently unimplemented -4294967191 4294967192 0 pg_class_tblspc_relfilenode_index was created for compatibility and is currently unimplemented -4294967189 4294967192 0 available collations (incomplete) -4294967190 4294967192 0 pg_collation_oid_index was created for compatibility and is currently unimplemented -4294967188 4294967192 0 pg_config was created for compatibility and is currently unimplemented -4294967184 4294967192 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967187 4294967192 0 pg_constraint_conparentid_index was created for compatibility and is currently unimplemented -4294967186 4294967192 0 pg_constraint_contypid_index was created for compatibility and is currently unimplemented -4294967185 4294967192 0 pg_constraint_oid_index was created for compatibility and is currently unimplemented -4294967181 4294967192 0 encoding conversions (empty - unimplemented) -4294967183 4294967192 0 pg_conversion_default_index was created for compatibility and is currently unimplemented -4294967182 4294967192 0 pg_conversion_oid_index was created for compatibility and is currently unimplemented -4294967180 4294967192 0 pg_cursors was created for compatibility and is currently unimplemented -4294967178 4294967192 0 available databases (incomplete) -4294967179 4294967192 0 pg_database_oid_index was created for compatibility and is currently unimplemented -4294967176 4294967192 0 pg_db_role_setting was created for compatibility and is currently unimplemented -4294967177 4294967192 0 pg_db_role_setting_databaseid_rol_index was created for compatibility and is currently unimplemented -4294967173 4294967192 0 default ACLs (empty - unimplemented) -4294967175 4294967192 0 pg_default_acl_oid_index was created for compatibility and is currently unimplemented -4294967174 4294967192 0 pg_default_acl_role_nsp_obj_index was created for compatibility and is currently unimplemented -4294967170 4294967192 0 dependency relationships (incomplete) -4294967172 4294967192 0 pg_depend_depender_index was created for compatibility and is currently unimplemented -4294967171 4294967192 0 pg_depend_reference_index was created for compatibility and is currently unimplemented -4294967168 4294967192 0 object comments -4294967169 4294967192 0 pg_description_o_c_o_index was created for compatibility and is currently unimplemented -4294967166 4294967192 0 enum types and labels (empty - feature does not exist) -4294967167 4294967192 0 pg_enum_oid_index was created for compatibility and is currently unimplemented -4294967165 4294967192 0 pg_enum_typid_sortorder_index was created for compatibility and is currently unimplemented -4294967163 4294967192 0 event triggers (empty - feature does not exist) -4294967164 4294967192 0 pg_event_trigger_oid_index was created for compatibility and is currently unimplemented -4294967161 4294967192 0 installed extensions (empty - feature does not exist) -4294967162 4294967192 0 pg_extension_oid_index was created for compatibility and is currently unimplemented -4294967160 4294967192 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967158 4294967192 0 foreign data wrappers (empty - feature does not exist) -4294967159 4294967192 0 pg_foreign_data_wrapper_oid_index was created for compatibility and is currently unimplemented -4294967156 4294967192 0 foreign servers (empty - feature does not exist) -4294967157 4294967192 0 pg_foreign_server_oid_index was created for compatibility and is currently unimplemented -4294967154 4294967192 0 foreign tables (empty - feature does not exist) -4294967155 4294967192 0 pg_foreign_table_relid_index was created for compatibility and is currently unimplemented -4294967153 4294967192 0 pg_group was created for compatibility and is currently unimplemented -4294967152 4294967192 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967149 4294967192 0 indexes (incomplete) -4294967151 4294967192 0 pg_index_indexrelid_index was created for compatibility and is currently unimplemented -4294967150 4294967192 0 pg_index_indrelid_index was created for compatibility and is currently unimplemented -4294967148 4294967192 0 index creation statements -4294967145 4294967192 0 table inheritance hierarchy (empty - feature does not exist) -4294967147 4294967192 0 pg_inherits_parent_index was created for compatibility and is currently unimplemented -4294967146 4294967192 0 pg_inherits_relid_seqno_index was created for compatibility and is currently unimplemented -4294967144 4294967192 0 pg_init_privs_o_c_o_index was created for compatibility and is currently unimplemented -4294967142 4294967192 0 available languages (empty - feature does not exist) -4294967143 4294967192 0 pg_language_oid_index was created for compatibility and is currently unimplemented -4294967139 4294967192 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967141 4294967192 0 pg_largeobject_loid_pn_index was created for compatibility and is currently unimplemented -4294967140 4294967192 0 pg_largeobject_metadata_oid_index was created for compatibility and is currently unimplemented -4294967138 4294967192 0 locks held by active processes (empty - feature does not exist) -4294967137 4294967192 0 available materialized views (empty - feature does not exist) -4294967135 4294967192 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967136 4294967192 0 pg_namespace_oid_index was created for compatibility and is currently unimplemented -4294967133 4294967192 0 opclass (empty - Operator classes not supported yet) -4294967134 4294967192 0 pg_opclass_oid_index was created for compatibility and is currently unimplemented -4294967131 4294967192 0 operators (incomplete) -4294967132 4294967192 0 pg_operator_oid_index was created for compatibility and is currently unimplemented -4294967129 4294967192 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967130 4294967192 0 pg_opfamily_oid_index was created for compatibility and is currently unimplemented -4294967128 4294967192 0 pg_partitioned_table_partrelid_index was created for compatibility and is currently unimplemented -4294967127 4294967192 0 pg_policies was created for compatibility and is currently unimplemented -4294967126 4294967192 0 pg_policy_oid_index was created for compatibility and is currently unimplemented -4294967125 4294967192 0 prepared statements -4294967124 4294967192 0 prepared transactions (empty - feature does not exist) -4294967122 4294967192 0 built-in functions (incomplete) -4294967123 4294967192 0 pg_proc_oid_index was created for compatibility and is currently unimplemented -4294967117 4294967192 0 pg_publication was created for compatibility and is currently unimplemented -4294967121 4294967192 0 pg_publication_oid_index was created for compatibility and is currently unimplemented -4294967118 4294967192 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967120 4294967192 0 pg_publication_rel_oid_index was created for compatibility and is currently unimplemented -4294967119 4294967192 0 pg_publication_rel_prrelid_prpubid_index was created for compatibility and is currently unimplemented -4294967116 4294967192 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967114 4294967192 0 range types (empty - feature does not exist) -4294967115 4294967192 0 pg_range_rngtypid_index was created for compatibility and is currently unimplemented -4294967111 4294967192 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967113 4294967192 0 pg_replication_origin_roiident_index was created for compatibility and is currently unimplemented -4294967112 4294967192 0 pg_replication_origin_roname_index was created for compatibility and is currently unimplemented -4294967109 4294967192 0 rewrite rules (empty - feature does not exist) -4294967110 4294967192 0 pg_rewrite_oid_index was created for compatibility and is currently unimplemented -4294967108 4294967192 0 database roles -4294967107 4294967192 0 pg_rules was created for compatibility and is currently unimplemented -4294967104 4294967192 0 security labels (empty - feature does not exist) -4294967105 4294967192 0 pg_seclabel_object_index was created for compatibility and is currently unimplemented -4294967106 4294967192 0 security labels (empty) -4294967102 4294967192 0 sequences (see also information_schema.sequences) -4294967103 4294967192 0 pg_sequence_seqrelid_index was created for compatibility and is currently unimplemented -4294967101 4294967192 0 session variables (incomplete) -4294967100 4294967192 0 pg_shadow was created for compatibility and is currently unimplemented -4294967095 4294967192 0 shared dependencies (empty - not implemented) -4294967097 4294967192 0 pg_shdepend_depender_index was created for compatibility and is currently unimplemented -4294967096 4294967192 0 pg_shdepend_reference_index was created for compatibility and is currently unimplemented -4294967099 4294967192 0 shared object comments -4294967094 4294967192 0 pg_shdescription_o_c_index was created for compatibility and is currently unimplemented -4294967093 4294967192 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967098 4294967192 0 shared security labels (empty - feature not supported) -4294967092 4294967192 0 pg_shseclabel_object_index was created for compatibility and is currently unimplemented -4294967091 4294967192 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967090 4294967192 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967089 4294967192 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967088 4294967192 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967087 4294967192 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967085 4294967192 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967086 4294967192 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967084 4294967192 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967083 4294967192 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967082 4294967192 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967081 4294967192 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967080 4294967192 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967079 4294967192 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967078 4294967192 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967077 4294967192 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967076 4294967192 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967075 4294967192 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967074 4294967192 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967073 4294967192 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967072 4294967192 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967071 4294967192 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967070 4294967192 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967069 4294967192 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967068 4294967192 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967067 4294967192 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967066 4294967192 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967065 4294967192 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967064 4294967192 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967063 4294967192 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967062 4294967192 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967061 4294967192 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967060 4294967192 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967059 4294967192 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967055 4294967192 0 pg_statistic_ext was created for compatibility and is currently unimplemented -4294967058 4294967192 0 pg_statistic_ext_data_stxoid_index was created for compatibility and is currently unimplemented -4294967057 4294967192 0 pg_statistic_ext_oid_index was created for compatibility and is currently unimplemented -4294967056 4294967192 0 pg_statistic_ext_relid_index was created for compatibility and is currently unimplemented -4294967054 4294967192 0 pg_statistic_relid_att_inh_index was created for compatibility and is currently unimplemented -4294967051 4294967192 0 pg_subscription was created for compatibility and is currently unimplemented -4294967053 4294967192 0 pg_subscription_oid_index was created for compatibility and is currently unimplemented -4294967052 4294967192 0 pg_subscription_rel_srrelid_srsubid_index was created for compatibility and is currently unimplemented -4294967050 4294967192 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967048 4294967192 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967049 4294967192 0 pg_tablespace_oid_index was created for compatibility and is currently unimplemented -4294967047 4294967192 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967046 4294967192 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967044 4294967192 0 pg_transform was created for compatibility and is currently unimplemented -4294967045 4294967192 0 pg_transform_oid_index was created for compatibility and is currently unimplemented -4294967043 4294967192 0 pg_transform_type_lang_index was created for compatibility and is currently unimplemented -4294967041 4294967192 0 triggers (empty - feature does not exist) -4294967042 4294967192 0 pg_trigger_oid_index was created for compatibility and is currently unimplemented -4294967040 4294967192 0 pg_trigger_tgconstraint_index was created for compatibility and is currently unimplemented -4294967036 4294967192 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967038 4294967192 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967039 4294967192 0 pg_ts_config_map_index was created for compatibility and is currently unimplemented -4294967037 4294967192 0 pg_ts_config_oid_index was created for compatibility and is currently unimplemented -4294967034 4294967192 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967035 4294967192 0 pg_ts_dict_oid_index was created for compatibility and is currently unimplemented -4294967032 4294967192 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967033 4294967192 0 pg_ts_parser_oid_index was created for compatibility and is currently unimplemented -4294967030 4294967192 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967031 4294967192 0 pg_ts_template_oid_index was created for compatibility and is currently unimplemented -4294967028 4294967192 0 scalar types (incomplete) -4294967029 4294967192 0 pg_type_oid_index was created for compatibility and is currently unimplemented -4294967023 4294967192 0 database users -4294967026 4294967192 0 local to remote user mapping (empty - feature does not exist) -4294967027 4294967192 0 pg_user_mapping_oid_index was created for compatibility and is currently unimplemented -4294967025 4294967192 0 pg_user_mapping_user_server_index was created for compatibility and is currently unimplemented -4294967024 4294967192 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967022 4294967192 0 view definitions (incomplete - see also information_schema.views) -4294967020 4294967192 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967019 4294967192 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967018 4294967192 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967294 4294967190 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967190 0 built-in functions (RAM/static) +4294967291 4294967190 0 contention information (cluster RPC; expensive!) +4294967249 4294967190 0 virtual table with database privileges +4294967290 4294967190 0 running queries visible by current user (cluster RPC; expensive!) +4294967288 4294967190 0 running sessions visible to current user (cluster RPC; expensive!) +4294967287 4294967190 0 cluster settings (RAM) +4294967289 4294967190 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967286 4294967190 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967285 4294967190 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967247 4294967190 0 virtual table with cross db references +4294967284 4294967190 0 databases accessible by the current user (KV scan) +4294967283 4294967190 0 telemetry counters (RAM; local node only) +4294967282 4294967190 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967280 4294967190 0 locally known gossiped health alerts (RAM; local node only) +4294967279 4294967190 0 locally known gossiped node liveness (RAM; local node only) +4294967278 4294967190 0 locally known edges in the gossip network (RAM; local node only) +4294967281 4294967190 0 locally known gossiped node details (RAM; local node only) +4294967277 4294967190 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967248 4294967190 0 virtual table with interleaved index information +4294967250 4294967190 0 virtual table to validate descriptors +4294967275 4294967190 0 decoded job metadata from system.jobs (KV scan) +4294967274 4294967190 0 node details across the entire cluster (cluster RPC; expensive!) +4294967273 4294967190 0 store details and status (cluster RPC; expensive!) +4294967272 4294967190 0 acquired table leases (RAM; local node only) +4294967293 4294967190 0 detailed identification strings (RAM, local node only) +4294967271 4294967190 0 contention information (RAM; local node only) +4294967276 4294967190 0 in-flight spans (RAM; local node only) +4294967267 4294967190 0 current values for metrics (RAM; local node only) +4294967270 4294967190 0 running queries visible by current user (RAM; local node only) +4294967262 4294967190 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967268 4294967190 0 running sessions visible by current user (RAM; local node only) +4294967258 4294967190 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967253 4294967190 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967269 4294967190 0 running user transactions visible by the current user (RAM; local node only) +4294967252 4294967190 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967266 4294967190 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967265 4294967190 0 comments for predefined virtual tables (RAM/static) +4294967264 4294967190 0 range metadata without leaseholder details (KV join; expensive!) +4294967261 4294967190 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967260 4294967190 0 session trace accumulated so far (RAM) +4294967259 4294967190 0 session variables (RAM) +4294967257 4294967190 0 details for all columns accessible by current user in current database (KV scan) +4294967256 4294967190 0 indexes accessible by current user in current database (KV scan) +4294967254 4294967190 0 stats for all tables accessible by current user in current database as of 10s ago +4294967255 4294967190 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967251 4294967190 0 decoded zone configurations from system.zones (KV scan) +4294967245 4294967190 0 roles for which the current user has admin option +4294967244 4294967190 0 roles available to the current user +4294967243 4294967190 0 character sets available in the current database +4294967242 4294967190 0 check constraints +4294967241 4294967190 0 identifies which character set the available collations are +4294967240 4294967190 0 shows the collations available in the current database +4294967239 4294967190 0 column privilege grants (incomplete) +4294967237 4294967190 0 columns with user defined types +4294967238 4294967190 0 table and view columns (incomplete) +4294967236 4294967190 0 columns usage by constraints +4294967235 4294967190 0 roles for the current user +4294967234 4294967190 0 column usage by indexes and key constraints +4294967233 4294967190 0 built-in function parameters (empty - introspection not yet supported) +4294967232 4294967190 0 foreign key constraints +4294967231 4294967190 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967230 4294967190 0 built-in functions (empty - introspection not yet supported) +4294967228 4294967190 0 schema privileges (incomplete; may contain excess users or roles) +4294967229 4294967190 0 database schemas (may contain schemata without permission) +4294967226 4294967190 0 sequences +4294967227 4294967190 0 exposes the session variables. +4294967225 4294967190 0 index metadata and statistics (incomplete) +4294967224 4294967190 0 table constraints +4294967223 4294967190 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967222 4294967190 0 tables and views +4294967221 4294967190 0 type privileges (incomplete; may contain excess users or roles) +4294967219 4294967190 0 grantable privileges (incomplete) +4294967220 4294967190 0 views (incomplete) +4294967216 4294967190 0 aggregated built-in functions (incomplete) +4294967217 4294967190 0 pg_aggregate_fnoid_index was created for compatibility and is currently unimplemented +4294967214 4294967190 0 index access methods (incomplete) +4294967215 4294967190 0 pg_am_oid_index was created for compatibility and is currently unimplemented +4294967210 4294967190 0 pg_amop was created for compatibility and is currently unimplemented +4294967213 4294967190 0 pg_amop_fam_strat_index was created for compatibility and is currently unimplemented +4294967212 4294967190 0 pg_amop_oid_index was created for compatibility and is currently unimplemented +4294967211 4294967190 0 pg_amop_opr_fam_index was created for compatibility and is currently unimplemented +4294967207 4294967190 0 pg_amproc was created for compatibility and is currently unimplemented +4294967209 4294967190 0 pg_amproc_fam_proc_index was created for compatibility and is currently unimplemented +4294967208 4294967190 0 pg_amproc_oid_index was created for compatibility and is currently unimplemented +4294967206 4294967190 0 column default values +4294967205 4294967190 0 pg_attrdef_adrelid_adnum_index was created for compatibility and is currently unimplemented +4294967204 4294967190 0 pg_attrdef_oid_index was created for compatibility and is currently unimplemented +4294967202 4294967190 0 table columns (incomplete - see also information_schema.columns) +4294967203 4294967190 0 pg_attribute_relid_attnum_index was created for compatibility and is currently unimplemented +4294967198 4294967190 0 role membership +4294967200 4294967190 0 pg_auth_members_member_role_index was created for compatibility and is currently unimplemented +4294967199 4294967190 0 pg_auth_members_role_member_index was created for compatibility and is currently unimplemented +4294967201 4294967190 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967197 4294967190 0 pg_authid_oid_index was created for compatibility and is currently unimplemented +4294967196 4294967190 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967195 4294967190 0 available extensions +4294967192 4294967190 0 casts (empty - needs filling out) +4294967194 4294967190 0 pg_cast_oid_index was created for compatibility and is currently unimplemented +4294967193 4294967190 0 pg_cast_source_target_index was created for compatibility and is currently unimplemented +4294967190 4294967190 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967191 4294967190 0 pg_class_oid_index was created for compatibility and is currently unimplemented +4294967189 4294967190 0 pg_class_tblspc_relfilenode_index was created for compatibility and is currently unimplemented +4294967187 4294967190 0 available collations (incomplete) +4294967188 4294967190 0 pg_collation_oid_index was created for compatibility and is currently unimplemented +4294967186 4294967190 0 pg_config was created for compatibility and is currently unimplemented +4294967182 4294967190 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967185 4294967190 0 pg_constraint_conparentid_index was created for compatibility and is currently unimplemented +4294967184 4294967190 0 pg_constraint_contypid_index was created for compatibility and is currently unimplemented +4294967183 4294967190 0 pg_constraint_oid_index was created for compatibility and is currently unimplemented +4294967179 4294967190 0 encoding conversions (empty - unimplemented) +4294967181 4294967190 0 pg_conversion_default_index was created for compatibility and is currently unimplemented +4294967180 4294967190 0 pg_conversion_oid_index was created for compatibility and is currently unimplemented +4294967178 4294967190 0 pg_cursors was created for compatibility and is currently unimplemented +4294967176 4294967190 0 available databases (incomplete) +4294967177 4294967190 0 pg_database_oid_index was created for compatibility and is currently unimplemented +4294967174 4294967190 0 pg_db_role_setting was created for compatibility and is currently unimplemented +4294967175 4294967190 0 pg_db_role_setting_databaseid_rol_index was created for compatibility and is currently unimplemented +4294967171 4294967190 0 default ACLs (empty - unimplemented) +4294967173 4294967190 0 pg_default_acl_oid_index was created for compatibility and is currently unimplemented +4294967172 4294967190 0 pg_default_acl_role_nsp_obj_index was created for compatibility and is currently unimplemented +4294967168 4294967190 0 dependency relationships (incomplete) +4294967170 4294967190 0 pg_depend_depender_index was created for compatibility and is currently unimplemented +4294967169 4294967190 0 pg_depend_reference_index was created for compatibility and is currently unimplemented +4294967166 4294967190 0 object comments +4294967167 4294967190 0 pg_description_o_c_o_index was created for compatibility and is currently unimplemented +4294967164 4294967190 0 enum types and labels (empty - feature does not exist) +4294967165 4294967190 0 pg_enum_oid_index was created for compatibility and is currently unimplemented +4294967163 4294967190 0 pg_enum_typid_sortorder_index was created for compatibility and is currently unimplemented +4294967161 4294967190 0 event triggers (empty - feature does not exist) +4294967162 4294967190 0 pg_event_trigger_oid_index was created for compatibility and is currently unimplemented +4294967159 4294967190 0 installed extensions (empty - feature does not exist) +4294967160 4294967190 0 pg_extension_oid_index was created for compatibility and is currently unimplemented +4294967158 4294967190 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967156 4294967190 0 foreign data wrappers (empty - feature does not exist) +4294967157 4294967190 0 pg_foreign_data_wrapper_oid_index was created for compatibility and is currently unimplemented +4294967154 4294967190 0 foreign servers (empty - feature does not exist) +4294967155 4294967190 0 pg_foreign_server_oid_index was created for compatibility and is currently unimplemented +4294967152 4294967190 0 foreign tables (empty - feature does not exist) +4294967153 4294967190 0 pg_foreign_table_relid_index was created for compatibility and is currently unimplemented +4294967151 4294967190 0 pg_group was created for compatibility and is currently unimplemented +4294967150 4294967190 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967147 4294967190 0 indexes (incomplete) +4294967149 4294967190 0 pg_index_indexrelid_index was created for compatibility and is currently unimplemented +4294967148 4294967190 0 pg_index_indrelid_index was created for compatibility and is currently unimplemented +4294967146 4294967190 0 index creation statements +4294967143 4294967190 0 table inheritance hierarchy (empty - feature does not exist) +4294967145 4294967190 0 pg_inherits_parent_index was created for compatibility and is currently unimplemented +4294967144 4294967190 0 pg_inherits_relid_seqno_index was created for compatibility and is currently unimplemented +4294967142 4294967190 0 pg_init_privs_o_c_o_index was created for compatibility and is currently unimplemented +4294967140 4294967190 0 available languages (empty - feature does not exist) +4294967141 4294967190 0 pg_language_oid_index was created for compatibility and is currently unimplemented +4294967137 4294967190 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967139 4294967190 0 pg_largeobject_loid_pn_index was created for compatibility and is currently unimplemented +4294967138 4294967190 0 pg_largeobject_metadata_oid_index was created for compatibility and is currently unimplemented +4294967136 4294967190 0 locks held by active processes (empty - feature does not exist) +4294967135 4294967190 0 available materialized views (empty - feature does not exist) +4294967133 4294967190 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967134 4294967190 0 pg_namespace_oid_index was created for compatibility and is currently unimplemented +4294967131 4294967190 0 opclass (empty - Operator classes not supported yet) +4294967132 4294967190 0 pg_opclass_oid_index was created for compatibility and is currently unimplemented +4294967129 4294967190 0 operators (incomplete) +4294967130 4294967190 0 pg_operator_oid_index was created for compatibility and is currently unimplemented +4294967127 4294967190 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967128 4294967190 0 pg_opfamily_oid_index was created for compatibility and is currently unimplemented +4294967126 4294967190 0 pg_partitioned_table_partrelid_index was created for compatibility and is currently unimplemented +4294967125 4294967190 0 pg_policies was created for compatibility and is currently unimplemented +4294967124 4294967190 0 pg_policy_oid_index was created for compatibility and is currently unimplemented +4294967123 4294967190 0 prepared statements +4294967122 4294967190 0 prepared transactions (empty - feature does not exist) +4294967120 4294967190 0 built-in functions (incomplete) +4294967121 4294967190 0 pg_proc_oid_index was created for compatibility and is currently unimplemented +4294967115 4294967190 0 pg_publication was created for compatibility and is currently unimplemented +4294967119 4294967190 0 pg_publication_oid_index was created for compatibility and is currently unimplemented +4294967116 4294967190 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967118 4294967190 0 pg_publication_rel_oid_index was created for compatibility and is currently unimplemented +4294967117 4294967190 0 pg_publication_rel_prrelid_prpubid_index was created for compatibility and is currently unimplemented +4294967114 4294967190 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967112 4294967190 0 range types (empty - feature does not exist) +4294967113 4294967190 0 pg_range_rngtypid_index was created for compatibility and is currently unimplemented +4294967109 4294967190 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967111 4294967190 0 pg_replication_origin_roiident_index was created for compatibility and is currently unimplemented +4294967110 4294967190 0 pg_replication_origin_roname_index was created for compatibility and is currently unimplemented +4294967107 4294967190 0 rewrite rules (empty - feature does not exist) +4294967108 4294967190 0 pg_rewrite_oid_index was created for compatibility and is currently unimplemented +4294967106 4294967190 0 database roles +4294967105 4294967190 0 pg_rules was created for compatibility and is currently unimplemented +4294967102 4294967190 0 security labels (empty - feature does not exist) +4294967103 4294967190 0 pg_seclabel_object_index was created for compatibility and is currently unimplemented +4294967104 4294967190 0 security labels (empty) +4294967100 4294967190 0 sequences (see also information_schema.sequences) +4294967101 4294967190 0 pg_sequence_seqrelid_index was created for compatibility and is currently unimplemented +4294967099 4294967190 0 session variables (incomplete) +4294967098 4294967190 0 pg_shadow was created for compatibility and is currently unimplemented +4294967093 4294967190 0 shared dependencies (empty - not implemented) +4294967095 4294967190 0 pg_shdepend_depender_index was created for compatibility and is currently unimplemented +4294967094 4294967190 0 pg_shdepend_reference_index was created for compatibility and is currently unimplemented +4294967097 4294967190 0 shared object comments +4294967092 4294967190 0 pg_shdescription_o_c_index was created for compatibility and is currently unimplemented +4294967091 4294967190 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967096 4294967190 0 shared security labels (empty - feature not supported) +4294967090 4294967190 0 pg_shseclabel_object_index was created for compatibility and is currently unimplemented +4294967089 4294967190 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967088 4294967190 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967087 4294967190 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967086 4294967190 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967085 4294967190 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967083 4294967190 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967084 4294967190 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967082 4294967190 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967081 4294967190 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967080 4294967190 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967079 4294967190 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967078 4294967190 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967077 4294967190 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967076 4294967190 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967075 4294967190 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967074 4294967190 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967073 4294967190 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967072 4294967190 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967071 4294967190 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967070 4294967190 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967069 4294967190 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967068 4294967190 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967067 4294967190 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967066 4294967190 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967065 4294967190 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967064 4294967190 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967063 4294967190 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967062 4294967190 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967061 4294967190 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967060 4294967190 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967059 4294967190 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967058 4294967190 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967057 4294967190 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967053 4294967190 0 pg_statistic_ext was created for compatibility and is currently unimplemented +4294967056 4294967190 0 pg_statistic_ext_data_stxoid_index was created for compatibility and is currently unimplemented +4294967055 4294967190 0 pg_statistic_ext_oid_index was created for compatibility and is currently unimplemented +4294967054 4294967190 0 pg_statistic_ext_relid_index was created for compatibility and is currently unimplemented +4294967052 4294967190 0 pg_statistic_relid_att_inh_index was created for compatibility and is currently unimplemented +4294967049 4294967190 0 pg_subscription was created for compatibility and is currently unimplemented +4294967051 4294967190 0 pg_subscription_oid_index was created for compatibility and is currently unimplemented +4294967050 4294967190 0 pg_subscription_rel_srrelid_srsubid_index was created for compatibility and is currently unimplemented +4294967048 4294967190 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967046 4294967190 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967047 4294967190 0 pg_tablespace_oid_index was created for compatibility and is currently unimplemented +4294967045 4294967190 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967044 4294967190 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967042 4294967190 0 pg_transform was created for compatibility and is currently unimplemented +4294967043 4294967190 0 pg_transform_oid_index was created for compatibility and is currently unimplemented +4294967041 4294967190 0 pg_transform_type_lang_index was created for compatibility and is currently unimplemented +4294967039 4294967190 0 triggers (empty - feature does not exist) +4294967040 4294967190 0 pg_trigger_oid_index was created for compatibility and is currently unimplemented +4294967038 4294967190 0 pg_trigger_tgconstraint_index was created for compatibility and is currently unimplemented +4294967034 4294967190 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967036 4294967190 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967037 4294967190 0 pg_ts_config_map_index was created for compatibility and is currently unimplemented +4294967035 4294967190 0 pg_ts_config_oid_index was created for compatibility and is currently unimplemented +4294967032 4294967190 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967033 4294967190 0 pg_ts_dict_oid_index was created for compatibility and is currently unimplemented +4294967030 4294967190 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967031 4294967190 0 pg_ts_parser_oid_index was created for compatibility and is currently unimplemented +4294967028 4294967190 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967029 4294967190 0 pg_ts_template_oid_index was created for compatibility and is currently unimplemented +4294967026 4294967190 0 scalar types (incomplete) +4294967027 4294967190 0 pg_type_oid_index was created for compatibility and is currently unimplemented +4294967021 4294967190 0 database users +4294967024 4294967190 0 local to remote user mapping (empty - feature does not exist) +4294967025 4294967190 0 pg_user_mapping_oid_index was created for compatibility and is currently unimplemented +4294967023 4294967190 0 pg_user_mapping_user_server_index was created for compatibility and is currently unimplemented +4294967022 4294967190 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967020 4294967190 0 view definitions (incomplete - see also information_schema.views) +4294967018 4294967190 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967017 4294967190 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967016 4294967190 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -3538,7 +3540,7 @@ indoption query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294967022 +test pg_views 4294967020 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/sequences b/pkg/sql/logictest/testdata/logic_test/sequences index 4c69588d3257..eee5781d1f77 100644 --- a/pkg/sql/logictest/testdata/logic_test/sequences +++ b/pkg/sql/logictest/testdata/logic_test/sequences @@ -1794,3 +1794,20 @@ DROP SEQUENCE s1 statement ok DROP TABLE s2 + +# Validate that cross DB sequences are detected by internal tables +statement ok +CREATE DATABASE db3; + +statement ok +CREATE SEQUENCE db3.s; + +statement ok +CREATE TABLE tDb3Ref (i INT PRIMARY KEY DEFAULT (nextval('db3.s'))); + +query TTTTTTT +SELECT * FROM "".crdb_internal.cross_db_references; +---- +db2 public seq db1 public t sequences owning table +db2 public seq2 db1 public t sequences owning table +test public tdb3ref db3 public s table column refers to sequence diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index 1bba390edcd3..782083aaf516 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -545,6 +545,7 @@ cluster_settings NULL cluster_transactions NULL create_statements NULL create_type_statements NULL +cross_db_references NULL databases NULL feature_usage NULL forward_dependencies NULL @@ -553,6 +554,7 @@ gossip_liveness NULL gossip_network NULL gossip_nodes NULL index_columns NULL +interleaved_indexes NULL invalid_objects NULL jobs NULL kv_node_status NULL diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index 942ba497c105..189731627020 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -902,3 +902,37 @@ DROP TYPE v4 statement ok DROP TABLE t + +# Validate that cross DB views are detected by internal tables + +statement ok +USE DB1; + +statement ok +CREATE SEQUENCE SQ1; + +statement ok +CREATE TYPE status AS ENUM ('open', 'closed', 'inactive'); + +statement ok +CREATE TABLE tval (val int primary key); + +statement ok +CREATE VIEW rv as select val from tval; + +statement ok; +USE DB2; + +statement ok; +CREATE VIEW vm as (select s.last_value, t.val as a, v.val as b, cast('open' as db1.status) from db1.tval as t, db1.sq1 as s, db1.rv as v) + +query TTTTTTT +select * from "".crdb_internal.cross_db_references order by object_database, object_schema, object_name, referenced_object_database, referenced_object_schema, referenced_object_name desc; +---- +db1 public sys system public descriptor view references table +db1 public sys2 system public descriptor view references table +db2 public v1 db1 public ab view references table +db2 public v2 db1 public ab view references table +db2 public vm db1 public tval view references table +db2 public vm db1 public sq1 view references table +db2 public vm db1 public rv view references table