diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 572258e3f11e..1c9f40c739a0 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -1476,7 +1476,7 @@ func (s *adminServer) DataDistribution( // and deleted tables (as opposed to e.g. information_schema) because we are interested // in the data for all ranges, not just ranges for visible tables. userName := s.getUser(req) - tablesQuery := `SELECT name, table_id, database_name, drop_time FROM "".crdb_internal.tables` + tablesQuery := `SELECT name, table_id, database_name, drop_time FROM "".crdb_internal.tables WHERE schema_name = 'public'` rows1, _ /* cols */, err := s.server.internalExecutor.QueryWithUser( ctx, "admin-replica-matrix", nil /* txn */, userName, tablesQuery, ) diff --git a/pkg/sql/comment_on_column.go b/pkg/sql/comment_on_column.go index 8eb1d352627a..6ef3c70bbcb8 100644 --- a/pkg/sql/comment_on_column.go +++ b/pkg/sql/comment_on_column.go @@ -24,7 +24,7 @@ import ( type commentOnColumnNode struct { n *tree.CommentOnColumn - tableDesc *MutableTableDescriptor + tableDesc *ImmutableTableDescriptor } // CommentOnColumn add comment on a column. @@ -34,7 +34,7 @@ func (p *planner) CommentOnColumn(ctx context.Context, n *tree.CommentOnColumn) if n.ColumnItem.TableName != nil { tableName = n.ColumnItem.TableName.ToTableName() } - tableDesc, err := p.ResolveMutableTableDescriptor(ctx, &tableName, true, requireTableDesc) + tableDesc, err := p.ResolveUncachedTableDescriptor(ctx, &tableName, true, requireTableDesc) if err != nil { return nil, err } diff --git a/pkg/sql/comment_on_table.go b/pkg/sql/comment_on_table.go index 8be2e32782ba..7ea80317de42 100644 --- a/pkg/sql/comment_on_table.go +++ b/pkg/sql/comment_on_table.go @@ -24,7 +24,7 @@ import ( type commentOnTableNode struct { n *tree.CommentOnTable - tableDesc *MutableTableDescriptor + tableDesc *ImmutableTableDescriptor } // CommentOnTable add comment on a table. @@ -32,7 +32,7 @@ type commentOnTableNode struct { // notes: postgres requires CREATE on the table. // mysql requires ALTER, CREATE, INSERT on the table. func (p *planner) CommentOnTable(ctx context.Context, n *tree.CommentOnTable) (planNode, error) { - tableDesc, err := p.ResolveMutableTableDescriptor(ctx, &n.Table, true, requireTableDesc) + tableDesc, err := p.ResolveUncachedTableDescriptor(ctx, &n.Table, true, requireTableDesc) if err != nil { return nil, err } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b6e32378c1ea..1e907e9b6a39 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -75,6 +75,7 @@ var crdbInternal = virtualSchema{ sqlbase.CrdbInternalLocalSessionsTableID: crdbInternalLocalSessionsTable, sqlbase.CrdbInternalLocalMetricsTableID: crdbInternalLocalMetricsTable, sqlbase.CrdbInternalPartitionsTableID: crdbInternalPartitionsTable, + sqlbase.CrdbInternalPredefinedCommentsTableID: crdbInternalPredefinedCommentsTable, sqlbase.CrdbInternalRangesNoLeasesTableID: crdbInternalRangesNoLeasesTable, sqlbase.CrdbInternalRangesViewID: crdbInternalRangesView, sqlbase.CrdbInternalRuntimeInfoTableID: crdbInternalRuntimeInfoTable, @@ -91,6 +92,7 @@ var crdbInternal = virtualSchema{ } var crdbInternalBuildInfoTable = virtualSchemaTable{ + comment: `detailed identification strings (RAM, local node only)`, schema: ` CREATE TABLE crdb_internal.node_build_info ( node_id INT NOT NULL, @@ -123,6 +125,7 @@ CREATE TABLE crdb_internal.node_build_info ( } var crdbInternalRuntimeInfoTable = virtualSchemaTable{ + comment: `server parameters, useful to construct connection URLs (RAM, local node only)`, schema: ` CREATE TABLE crdb_internal.node_runtime_info ( node_id INT NOT NULL, @@ -181,12 +184,13 @@ CREATE TABLE crdb_internal.node_runtime_info ( } var crdbInternalTablesTable = virtualSchemaTable{ + comment: `table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)`, schema: ` CREATE TABLE crdb_internal.tables ( table_id INT NOT NULL, parent_id INT NOT NULL, name STRING NOT NULL, - database_name STRING NOT NULL, + database_name STRING, version INT NOT NULL, mod_time TIMESTAMP NOT NULL, mod_time_logical DECIMAL NOT NULL, @@ -195,7 +199,8 @@ CREATE TABLE crdb_internal.tables ( sc_lease_node_id INT, sc_lease_expiration_time TIMESTAMP, drop_time TIMESTAMP, - audit_mode STRING NOT NULL + audit_mode STRING NOT NULL, + schema_name STRING NOT NULL )`, populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { descs, err := p.Tables().getAllDescriptors(ctx, p.txn) @@ -210,21 +215,8 @@ CREATE TABLE crdb_internal.tables ( dbNames[db.ID] = db.Name } } - // Note: we do not use forEachTableDesc() here because we want to - // include added and dropped descriptors. - for _, desc := range descs { - table, ok := desc.(*sqlbase.TableDescriptor) - if !ok || p.CheckAnyPrivilege(ctx, table) != nil { - continue - } - dbName := dbNames[table.GetParentID()] - if dbName == "" { - // The parent database was deleted. This is possible e.g. when - // a database is dropped with CASCADE, and someone queries - // this virtual table before the dropped table descriptors are - // effectively deleted. - dbName = fmt.Sprintf("[%d]", table.GetParentID()) - } + + addDesc := func(table *sqlbase.TableDescriptor, dbName tree.Datum, scName string) error { leaseNodeDatum := tree.DNull leaseExpDatum := tree.DNull if table.Lease != nil { @@ -239,11 +231,11 @@ CREATE TABLE crdb_internal.tables ( timeutil.Unix(0, table.DropTime), time.Nanosecond, ) } - if err := addRow( + return addRow( tree.NewDInt(tree.DInt(int64(table.ID))), tree.NewDInt(tree.DInt(int64(table.GetParentID()))), tree.NewDString(table.Name), - tree.NewDString(dbName), + dbName, tree.NewDInt(tree.DInt(int64(table.Version))), tree.MakeDTimestamp(timeutil.Unix(0, table.ModificationTime.WallTime), time.Microsecond), tree.TimestampToDecimal(table.ModificationTime), @@ -253,15 +245,48 @@ CREATE TABLE crdb_internal.tables ( leaseExpDatum, dropTimeDatum, tree.NewDString(table.AuditMode.String()), - ); err != nil { + tree.NewDString(scName)) + } + + // Note: we do not use forEachTableDesc() here because we want to + // include added and dropped descriptors. + for _, desc := range descs { + table, ok := desc.(*sqlbase.TableDescriptor) + if !ok || p.CheckAnyPrivilege(ctx, table) != nil { + continue + } + dbName := dbNames[table.GetParentID()] + if dbName == "" { + // The parent database was deleted. This is possible e.g. when + // a database is dropped with CASCADE, and someone queries + // this virtual table before the dropped table descriptors are + // effectively deleted. + dbName = fmt.Sprintf("[%d]", table.GetParentID()) + } + if err := addDesc(table, tree.NewDString(dbName), "public"); err != nil { return err } } + + // Also add all the virtual descriptors. + vt := p.getVirtualTabler() + vEntries := vt.getEntries() + for _, virtSchemaName := range vt.getSchemaNames() { + e := vEntries[virtSchemaName] + for _, tName := range e.orderedDefNames { + vTableEntry := e.defs[tName] + if err := addDesc(vTableEntry.desc, tree.DNull, virtSchemaName); err != nil { + return err + } + } + } + return nil }, } var crdbInternalSchemaChangesTable = virtualSchemaTable{ + comment: `ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)`, schema: ` CREATE TABLE crdb_internal.schema_changes ( table_id INT NOT NULL, @@ -324,6 +349,7 @@ CREATE TABLE crdb_internal.schema_changes ( } var crdbInternalLeasesTable = virtualSchemaTable{ + comment: `acquired table leases (RAM; local node only)`, schema: ` CREATE TABLE crdb_internal.leases ( node_id INT NOT NULL, @@ -391,6 +417,7 @@ func tsOrNull(micros int64) tree.Datum { } var crdbInternalJobsTable = virtualSchemaTable{ + comment: `decoded job metadata from system.jobs (KV scan)`, schema: ` CREATE TABLE crdb_internal.jobs ( job_id INT, @@ -524,6 +551,7 @@ func (s stmtList) Less(i, j int) bool { } var crdbInternalStmtStatsTable = virtualSchemaTable{ + comment: `statement statistics (RAM; local node only)`, schema: ` CREATE TABLE crdb_internal.node_statement_statistics ( node_id INT NOT NULL, @@ -635,6 +663,7 @@ CREATE TABLE crdb_internal.node_statement_statistics ( // crdbInternalSessionTraceTable exposes the latest trace collected on this // session (via SET TRACING={ON/OFF}) var crdbInternalSessionTraceTable = virtualSchemaTable{ + comment: `session trace accumulated so far (RAM)`, schema: ` CREATE TABLE crdb_internal.session_trace ( span_idx INT NOT NULL, -- The span's index. @@ -667,6 +696,7 @@ CREATE TABLE crdb_internal.session_trace ( // crdbInternalClusterSettingsTable exposes the list of current // cluster settings. var crdbInternalClusterSettingsTable = virtualSchemaTable{ + comment: `cluster settings (RAM)`, schema: ` CREATE TABLE crdb_internal.cluster_settings ( variable STRING NOT NULL, @@ -695,6 +725,7 @@ CREATE TABLE crdb_internal.cluster_settings ( // crdbInternalSessionVariablesTable exposes the session variables. var crdbInternalSessionVariablesTable = virtualSchemaTable{ + comment: `session variables (RAM)`, schema: ` CREATE TABLE crdb_internal.session_variables ( variable STRING NOT NULL, @@ -742,7 +773,8 @@ func (p *planner) makeSessionsRequest(ctx context.Context) serverpb.ListSessions // crdbInternalLocalQueriesTable exposes the list of running queries // on the current node. The results are dependent on the current user. var crdbInternalLocalQueriesTable = virtualSchemaTable{ - schema: fmt.Sprintf(queriesSchemaPattern, "node_queries"), + comment: "running queries visible by current user (RAM; local node only)", + schema: fmt.Sprintf(queriesSchemaPattern, "node_queries"), populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { req := p.makeSessionsRequest(ctx) response, err := p.extendedEvalCtx.StatusServer.ListLocalSessions(ctx, &req) @@ -756,7 +788,8 @@ var crdbInternalLocalQueriesTable = virtualSchemaTable{ // crdbInternalClusterQueriesTable exposes the list of running queries // on the entire cluster. The result is dependent on the current user. var crdbInternalClusterQueriesTable = virtualSchemaTable{ - schema: fmt.Sprintf(queriesSchemaPattern, "cluster_queries"), + comment: "running queries visible by current user (cluster RPC; expensive!)", + schema: fmt.Sprintf(queriesSchemaPattern, "cluster_queries"), populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { req := p.makeSessionsRequest(ctx) response, err := p.extendedEvalCtx.StatusServer.ListSessions(ctx, &req) @@ -839,7 +872,8 @@ CREATE TABLE crdb_internal.%s ( // crdbInternalLocalSessionsTable exposes the list of running sessions // on the current node. The results are dependent on the current user. var crdbInternalLocalSessionsTable = virtualSchemaTable{ - schema: fmt.Sprintf(sessionsSchemaPattern, "node_sessions"), + comment: "running sessions visible by current user (RAM; local node only)", + schema: fmt.Sprintf(sessionsSchemaPattern, "node_sessions"), populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { req := p.makeSessionsRequest(ctx) response, err := p.extendedEvalCtx.StatusServer.ListLocalSessions(ctx, &req) @@ -853,7 +887,8 @@ var crdbInternalLocalSessionsTable = virtualSchemaTable{ // crdbInternalClusterSessionsTable exposes the list of running sessions // on the entire cluster. The result is dependent on the current user. var crdbInternalClusterSessionsTable = virtualSchemaTable{ - schema: fmt.Sprintf(sessionsSchemaPattern, "cluster_sessions"), + comment: "running sessions visible to current user (cluster RPC; expensive!)", + schema: fmt.Sprintf(sessionsSchemaPattern, "cluster_sessions"), populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { req := p.makeSessionsRequest(ctx) response, err := p.extendedEvalCtx.StatusServer.ListSessions(ctx, &req) @@ -961,12 +996,12 @@ func populateSessionsTable( // crdbInternalLocalMetricsTable exposes a snapshot of the metrics on the // current node. var crdbInternalLocalMetricsTable = virtualSchemaTable{ + comment: "current values for metrics (RAM; local node only)", schema: `CREATE TABLE crdb_internal.node_metrics ( store_id INT NULL, -- the store, if any, for this metric name STRING NOT NULL, -- name of the metric value FLOAT NOT NULL -- value of the metric )`, - populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { if err := p.RequireSuperUser(ctx, "read crdb_internal.node_metrics"); err != nil { return err @@ -1001,6 +1036,7 @@ var crdbInternalLocalMetricsTable = virtualSchemaTable{ // crdbInternalBuiltinFunctionsTable exposes the built-in function // metadata. var crdbInternalBuiltinFunctionsTable = virtualSchemaTable{ + comment: "built-in functions (RAM/static)", schema: ` CREATE TABLE crdb_internal.builtin_functions ( function STRING NOT NULL, @@ -1029,6 +1065,7 @@ CREATE TABLE crdb_internal.builtin_functions ( // crdbInternalCreateStmtsTable exposes the CREATE TABLE/CREATE VIEW // statements. var crdbInternalCreateStmtsTable = virtualSchemaTable{ + comment: `CREATE and ALTER statements for all tables accessible by current user in current database (KV scan)`, schema: ` CREATE TABLE crdb_internal.create_statements ( database_id INT, @@ -1139,6 +1176,7 @@ CREATE TABLE crdb_internal.create_statements ( // crdbInternalTableColumnsTable exposes the column descriptors. var crdbInternalTableColumnsTable = virtualSchemaTable{ + comment: "details for all columns accessible by current user in current database (KV scan)", schema: ` CREATE TABLE crdb_internal.table_columns ( descriptor_id INT, @@ -1181,6 +1219,7 @@ CREATE TABLE crdb_internal.table_columns ( // crdbInternalTableIndexesTable exposes the index descriptors. var crdbInternalTableIndexesTable = virtualSchemaTable{ + comment: "indexes accessible by current user in current database (KV scan)", schema: ` CREATE TABLE crdb_internal.table_indexes ( descriptor_id INT, @@ -1227,6 +1266,7 @@ CREATE TABLE crdb_internal.table_indexes ( // crdbInternalIndexColumnsTable exposes the index columns. var crdbInternalIndexColumnsTable = virtualSchemaTable{ + comment: "index columns for all indexes accessible by current user in current database (KV scan)", schema: ` CREATE TABLE crdb_internal.index_columns ( descriptor_id INT, @@ -1339,6 +1379,7 @@ CREATE TABLE crdb_internal.index_columns ( // crdbInternalBackwardDependenciesTable exposes the backward // inter-descriptor dependencies. var crdbInternalBackwardDependenciesTable = virtualSchemaTable{ + comment: "backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan)", schema: ` CREATE TABLE crdb_internal.backward_dependencies ( descriptor_id INT, @@ -1450,6 +1491,7 @@ CREATE TABLE crdb_internal.backward_dependencies ( // crdbInternalFeatureUsage exposes the telemetry counters. var crdbInternalFeatureUsage = virtualSchemaTable{ + comment: "telemetry counters (RAM; local node only)", schema: ` CREATE TABLE crdb_internal.feature_usage ( feature_name STRING NOT NULL, @@ -1472,6 +1514,7 @@ CREATE TABLE crdb_internal.feature_usage ( // crdbInternalForwardDependenciesTable exposes the forward // inter-descriptor dependencies. var crdbInternalForwardDependenciesTable = virtualSchemaTable{ + comment: "forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan)", schema: ` CREATE TABLE crdb_internal.forward_dependencies ( descriptor_id INT, @@ -1608,6 +1651,7 @@ FROM crdb_internal.ranges_no_leases // crdbInternalRangesNoLeasesTable exposes system ranges without the // `lease_holder` information. var crdbInternalRangesNoLeasesTable = virtualSchemaTable{ + comment: `range metadata without leaseholder details (KV join; expensive!)`, schema: ` CREATE TABLE crdb_internal.ranges_no_leases ( range_id INT NOT NULL, @@ -1718,6 +1762,7 @@ CREATE TABLE crdb_internal.ranges_no_leases ( // as a hidden field by the CLI for backwards compatibility. Use zone_name // instead. var crdbInternalZonesTable = virtualSchemaTable{ + comment: "decoded zone configurations from system.zones (KV scan)", schema: ` CREATE TABLE crdb_internal.zones ( zone_id INT NOT NULL, @@ -1817,6 +1862,7 @@ CREATE TABLE crdb_internal.zones ( // crdbInternalGossipNodesTable exposes local information about the cluster nodes. var crdbInternalGossipNodesTable = virtualSchemaTable{ + comment: "locally known gossiped node details (RAM; local node only)", schema: ` CREATE TABLE crdb_internal.gossip_nodes ( node_id INT NOT NULL, @@ -1939,6 +1985,7 @@ CREATE TABLE crdb_internal.gossip_nodes ( // liveness. The data exposed in this table can be stale/incomplete because // gossip doesn't provide guarantees around freshness or consistency. var crdbInternalGossipLivenessTable = virtualSchemaTable{ + comment: "locally known gossiped node liveness (RAM; local node only)", schema: ` CREATE TABLE crdb_internal.gossip_liveness ( node_id INT NOT NULL, @@ -2009,6 +2056,7 @@ CREATE TABLE crdb_internal.gossip_liveness ( // crdbInternalGossipAlertsTable exposes current health alerts in the cluster. var crdbInternalGossipAlertsTable = virtualSchemaTable{ + comment: "locally known gossiped health alerts (RAM; local node only)", schema: ` CREATE TABLE crdb_internal.gossip_alerts ( node_id INT NOT NULL, @@ -2074,6 +2122,7 @@ CREATE TABLE crdb_internal.gossip_alerts ( // crdbInternalGossipNetwork exposes the local view of the gossip network (i.e // the gossip client connections from source_id node to target_id node). var crdbInternalGossipNetworkTable = virtualSchemaTable{ + comment: "locally known edges in the gossip network (RAM; local node only)", schema: ` CREATE TABLE crdb_internal.gossip_network ( source_id INT NOT NULL, -- source node of a gossip connection @@ -2146,6 +2195,7 @@ func addPartitioningRows( // crdbInternalPartitionsTable decodes and exposes the partitions of each // table. var crdbInternalPartitionsTable = virtualSchemaTable{ + comment: "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)", schema: ` CREATE TABLE crdb_internal.partitions ( table_id INT NOT NULL, @@ -2168,6 +2218,7 @@ CREATE TABLE crdb_internal.partitions ( // crdbInternalKVNodeStatusTable exposes information from the status server about the cluster nodes. var crdbInternalKVNodeStatusTable = virtualSchemaTable{ + comment: "node details across the entire cluster (cluster RPC; expensive!)", schema: ` CREATE TABLE crdb_internal.kv_node_status ( node_id INT NOT NULL, @@ -2281,6 +2332,7 @@ CREATE TABLE crdb_internal.kv_node_status ( // crdbInternalKVStoreStatusTable exposes information about the cluster stores. var crdbInternalKVStoreStatusTable = virtualSchemaTable{ + comment: "store details and status (cluster RPC; expensive!)", schema: ` CREATE TABLE crdb_internal.kv_store_status ( node_id INT NOT NULL, @@ -2390,3 +2442,47 @@ CREATE TABLE crdb_internal.kv_store_status ( return nil }, } + +// crdbInternalPredefinedComments exposes the predefined +// comments for virtual tables. This is used by SHOW TABLES WITH COMMENT +// as fall-back when system.comments is silent. +// TODO(knz): extend this with vtable column comments. +var crdbInternalPredefinedCommentsTable = virtualSchemaTable{ + comment: `comments for predefined virtual tables (RAM/static)`, + schema: ` +CREATE TABLE crdb_internal.predefined_comments ( + TYPE INT, + OBJECT_ID INT, + SUB_ID INT, + COMMENT STRING +)`, + populate: func( + ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error, + ) error { + tableCommentKey := tree.NewDInt(keys.TableCommentType) + vt := p.getVirtualTabler() + vEntries := vt.getEntries() + vSchemaNames := vt.getSchemaNames() + + for _, virtSchemaName := range vSchemaNames { + e := vEntries[virtSchemaName] + + for _, tName := range e.orderedDefNames { + vTableEntry := e.defs[tName] + table := vTableEntry.desc + + if vTableEntry.comment != "" { + if err := addRow( + tableCommentKey, + tree.NewDInt(tree.DInt(table.ID)), + zeroVal, + tree.NewDString(vTableEntry.comment)); err != nil { + return err + } + } + } + } + + return nil + }, +} diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index a3d3cbf73a28..afd35d3e0029 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -19,6 +19,7 @@ import ( "sort" "strconv" + "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/sem/tree" @@ -200,6 +201,9 @@ func validateInformationSchemaTable(table *sqlbase.TableDescriptor) error { } var informationSchemaAdministrableRoleAuthorizations = virtualSchemaTable{ + comment: `roles for which the current user has admin option +` + base.DocsURL("information-schema.html#administrable_role_authorizations") + ` +https://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html`, schema: vtable.InformationSchemaAdministrableRoleAuthorizations, populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { currentUser := p.SessionData().User @@ -229,6 +233,9 @@ var informationSchemaAdministrableRoleAuthorizations = virtualSchemaTable{ } var informationSchemaApplicableRoles = virtualSchemaTable{ + comment: `roles available to the current user +` + base.DocsURL("information-schema.html#applicable_roles") + ` +https://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html`, schema: vtable.InformationSchemaApplicableRoles, populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { currentUser := p.SessionData().User @@ -254,6 +261,9 @@ var informationSchemaApplicableRoles = virtualSchemaTable{ } var informationSchemaColumnPrivileges = virtualSchemaTable{ + comment: `column privilege grants (incomplete) +` + base.DocsURL("information-schema.html#column_privileges") + ` +https://www.postgresql.org/docs/9.5/infoschema-column-privileges.html`, schema: vtable.InformationSchemaColumnPrivileges, 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 { @@ -286,6 +296,9 @@ var informationSchemaColumnPrivileges = virtualSchemaTable{ } var informationSchemaColumnsTable = virtualSchemaTable{ + comment: `table and view columns (incomplete) +` + base.DocsURL("information-schema.html#columns") + ` +https://www.postgresql.org/docs/9.5/infoschema-columns.html`, schema: vtable.InformationSchemaColumns, 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 { @@ -325,9 +338,10 @@ var informationSchemaColumnsTable = virtualSchemaTable{ }, } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-enabled-roles.html -// MySQL: missing var informationSchemaEnabledRoles = virtualSchemaTable{ + comment: `roles for the current user +` + base.DocsURL("information-schema.html#enabled_roles") + ` +https://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html`, schema: ` CREATE TABLE information_schema.enabled_roles ( ROLE_NAME STRING NOT NULL @@ -383,9 +397,9 @@ func datetimePrecision(colType sqlbase.ColumnType) tree.Datum { return tree.DNull } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-constraint-column-usage.html -// MySQL: missing var informationSchemaConstraintColumnUsageTable = virtualSchemaTable{ + comment: `columns usage by constraints +https://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html`, schema: ` CREATE TABLE information_schema.constraint_column_usage ( TABLE_CATALOG STRING NOT NULL, @@ -441,9 +455,11 @@ CREATE TABLE information_schema.constraint_column_usage ( }, } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-key-column-usage.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/key-column-usage-table.html var informationSchemaKeyColumnUsageTable = virtualSchemaTable{ + comment: `column usage by indexes and key constraints +` + base.DocsURL("information-schema.html#key_column_usage") + ` +https://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html`, schema: ` CREATE TABLE information_schema.key_column_usage ( CONSTRAINT_CATALOG STRING NOT NULL, @@ -511,6 +527,8 @@ CREATE TABLE information_schema.key_column_usage ( // Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-parameters.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/parameters-table.html var informationSchemaParametersTable = virtualSchemaTable{ + comment: `built-in function parameters (empty - introspection not yet supported) +https://www.postgresql.org/docs/9.5/infoschema-parameters.html`, schema: ` CREATE TABLE information_schema.parameters ( SPECIFIC_CATALOG STRING, @@ -585,9 +603,11 @@ func dStringForFKAction(action sqlbase.ForeignKeyReference_Action) tree.Datum { panic(errors.Errorf("unexpected ForeignKeyReference_Action: %v", action)) } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-referential-constraints.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/referential-constraints-table.html var informationSchemaReferentialConstraintsTable = virtualSchemaTable{ + comment: `foreign key constraints +` + base.DocsURL("information-schema.html#referential_constraints") + ` +https://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html`, schema: ` CREATE TABLE information_schema.referential_constraints ( CONSTRAINT_CATALOG STRING NOT NULL, @@ -651,6 +671,9 @@ CREATE TABLE information_schema.referential_constraints ( // Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-role-table-grants.html // MySQL: missing var informationSchemaRoleTableGrants = virtualSchemaTable{ + comment: `privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +` + base.DocsURL("information-schema.html#role_table_grants") + ` +https://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html`, schema: ` CREATE TABLE information_schema.role_table_grants ( GRANTOR STRING, @@ -668,9 +691,10 @@ CREATE TABLE information_schema.role_table_grants ( populate: populateTablePrivileges, } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-routines.html // MySQL: https://dev.mysql.com/doc/mysql-infoschema-excerpt/5.7/en/routines-table.html var informationSchemaRoutineTable = virtualSchemaTable{ + comment: `built-in functions (empty - introspection not yet supported) +https://www.postgresql.org/docs/9.5/infoschema-routines.html`, schema: ` CREATE TABLE information_schema.routines ( SPECIFIC_CATALOG STRING, @@ -760,9 +784,11 @@ CREATE TABLE information_schema.routines ( }, } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-schemata.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/schemata-table.html var informationSchemaSchemataTable = virtualSchemaTable{ + comment: `database schemas (may contain schemata without permission) +` + base.DocsURL("information-schema.html#schemata") + ` +https://www.postgresql.org/docs/9.5/infoschema-schemata.html`, schema: vtable.InformationSchemaSchemata, populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { return forEachDatabaseDesc(ctx, p, dbContext, func(db *sqlbase.DatabaseDescriptor) error { @@ -778,9 +804,10 @@ var informationSchemaSchemataTable = virtualSchemaTable{ }, } -// Postgres: missing // MySQL: https://dev.mysql.com/doc/refman/5.7/en/schema-privileges-table.html var informationSchemaSchemataTablePrivileges = virtualSchemaTable{ + comment: `schema privileges (incomplete; may contain excess users or roles) +` + base.DocsURL("information-schema.html#schema_privileges"), schema: ` CREATE TABLE information_schema.schema_privileges ( GRANTEE STRING NOT NULL, @@ -833,9 +860,10 @@ func dStringForIndexDirection(dir sqlbase.IndexDescriptor_Direction) tree.Datum panic("unreachable") } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-sequences.html -// MySQL: missing var informationSchemaSequences = virtualSchemaTable{ + comment: `sequences +` + base.DocsURL("information-schema.html#sequences") + ` +https://www.postgresql.org/docs/9.5/infoschema-sequences.html`, schema: ` CREATE TABLE information_schema.sequences ( SEQUENCE_CATALOG STRING NOT NULL, @@ -878,6 +906,8 @@ CREATE TABLE information_schema.sequences ( // Postgres: missing // MySQL: https://dev.mysql.com/doc/refman/5.7/en/statistics-table.html var informationSchemaStatisticsTable = virtualSchemaTable{ + comment: `index metadata and statistics (incomplete) +` + base.DocsURL("information-schema.html#statistics"), schema: ` CREATE TABLE information_schema.statistics ( TABLE_CATALOG STRING NOT NULL, @@ -974,9 +1004,11 @@ CREATE TABLE information_schema.statistics ( }, } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-table-constraints.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/table-constraints-table.html var informationSchemaTableConstraintTable = virtualSchemaTable{ + comment: `table constraints +` + base.DocsURL("information-schema.html#table_constraints") + ` +https://www.postgresql.org/docs/9.5/infoschema-table-constraints.html`, schema: ` CREATE TABLE information_schema.table_constraints ( CONSTRAINT_CATALOG STRING NOT NULL, @@ -1026,9 +1058,11 @@ CREATE TABLE information_schema.table_constraints ( }, } -// Postgres: missing +// Postgres: not provided // MySQL: https://dev.mysql.com/doc/refman/5.7/en/user-privileges-table.html +// TODO(knz): this introspection facility is of dubious utility. var informationSchemaUserPrivileges = virtualSchemaTable{ + comment: `grantable privileges (incomplete)`, schema: ` CREATE TABLE information_schema.user_privileges ( GRANTEE STRING NOT NULL, @@ -1057,9 +1091,11 @@ CREATE TABLE information_schema.user_privileges ( }, } -// Postgres: https://www.postgresql.org/docs/9.6/static/infoschema-table-privileges.html // MySQL: https://dev.mysql.com/doc/refman/5.7/en/table-privileges-table.html var informationSchemaTablePrivileges = virtualSchemaTable{ + comment: `privileges granted on table or views (incomplete; may contain excess users or roles) +` + base.DocsURL("information-schema.html#table_privileges") + ` +https://www.postgresql.org/docs/9.5/infoschema-table-privileges.html`, schema: ` CREATE TABLE information_schema.table_privileges ( GRANTOR STRING, @@ -1112,6 +1148,9 @@ var ( ) var informationSchemaTablesTable = virtualSchemaTable{ + comment: `tables and views +` + base.DocsURL("information-schema.html#tables") + ` +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, @@ -1146,6 +1185,9 @@ var informationSchemaTablesTable = virtualSchemaTable{ // 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{ + comment: `views (incomplete) +` + base.DocsURL("information-schema.html#views") + ` +https://www.postgresql.org/docs/9.5/infoschema-views.html`, schema: ` CREATE TABLE information_schema.views ( TABLE_CATALOG STRING NOT NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function b/pkg/sql/logictest/testdata/logic_test/builtin_function index aff8f0225013..f0af167a9f33 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function @@ -1912,14 +1912,36 @@ FOREIGN KEY (a) REFERENCES pg_indexdef_test (a) ON DELETE CASCADE CHECK (c > a) UNIQUE (b ASC) -# These functions always return NULL since we don't support comments. -query TTTT +# These functions always return NULL since we don't support comments on vtable columns and databases. +query TT SELECT col_description('pg_class'::regclass::oid, 2), - obj_description('pg_class'::regclass::oid, 'pg_class'), - obj_description('pg_class'::regclass::oid), shobj_description('pg_class'::regclass::oid, 'pg_class') ---- -NULL NULL NULL NULL +NULL NULL + +# vtable comments are supported +query TT +SELECT regexp_replace(obj_description('pg_class'::regclass::oid), e' .*', '') AS comment1, + regexp_replace(obj_description('pg_class'::regclass::oid, 'test'), e' .*', '') AS comment2 +---- +tables tables + +# Regular table column comments are supported. +statement ok +CREATE TABLE t(x INT); + +statement ok +COMMENT ON TABLE t IS 'waa' + +statement ok +COMMENT ON COLUMN t.x IS 'woo' + +query TTT +SELECT obj_description('t'::regclass::oid), + obj_description('t'::regclass::oid, 'test'), + col_description('t'::regclass, 1) +---- +waa waa woo # Check that base function names are also visible in namespace pg_catalog. query I diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index c2a4fc8aac3b..6564a9320683 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -36,6 +36,7 @@ node_runtime_info node_sessions node_statement_statistics partitions +predefined_comments ranges ranges_no_leases schema_changes @@ -71,11 +72,11 @@ SELECT * FROM crdb_internal.schema_changes ---- table_id parent_id name type target_id target_name state direction -query IITTITRTTTTTT colnames +query IITTITRTTTTTTT colnames SELECT * FROM crdb_internal.tables WHERE NAME = 'namespace' ---- -table_id parent_id name database_name version mod_time mod_time_logical format_version state sc_lease_node_id sc_lease_expiration_time drop_time audit_mode -2 1 namespace system 1 1970-01-01 00:00:00 +0000 +0000 0E-10 InterleavedFormatVersion PUBLIC NULL NULL NULL DISABLED +table_id parent_id name database_name version mod_time mod_time_logical format_version state sc_lease_node_id sc_lease_expiration_time drop_time audit_mode schema_name +2 1 namespace system 1 1970-01-01 00:00:00 +0000 +0000 0E-10 InterleavedFormatVersion PUBLIC NULL NULL NULL DISABLED public # Verify that table names are not double escaped. diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index f63ba3ac3755..d218655064eb 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -57,6 +57,7 @@ test crdb_internal node_runtime_info public S test crdb_internal node_sessions public SELECT test crdb_internal node_statement_statistics public SELECT test crdb_internal partitions public SELECT +test crdb_internal predefined_comments public SELECT test crdb_internal ranges public SELECT test crdb_internal ranges_no_leases public SELECT test crdb_internal schema_changes public SELECT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 2ffdaa48f03a..99c92ae11056 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -235,6 +235,7 @@ crdb_internal node_runtime_info crdb_internal node_sessions crdb_internal node_statement_statistics crdb_internal partitions +crdb_internal predefined_comments crdb_internal ranges crdb_internal ranges_no_leases crdb_internal schema_changes @@ -361,6 +362,7 @@ node_runtime_info node_sessions node_statement_statistics partitions +predefined_comments ranges ranges_no_leases schema_changes @@ -494,6 +496,7 @@ system crdb_internal node_runtime_info SYSTEM VIE system crdb_internal node_sessions SYSTEM VIEW NO 1 system crdb_internal node_statement_statistics SYSTEM VIEW NO 1 system crdb_internal partitions SYSTEM VIEW NO 1 +system crdb_internal predefined_comments SYSTEM VIEW NO 1 system crdb_internal ranges SYSTEM VIEW NO 1 system crdb_internal ranges_no_leases SYSTEM VIEW NO 1 system crdb_internal schema_changes SYSTEM VIEW NO 1 @@ -593,8 +596,9 @@ user testuser query TTTTTI colnames SELECT * FROM other_db.information_schema.tables WHERE table_catalog = 'other_db' AND table_schema = 'public' ---- -table_catalog table_schema table_name table_type is_insertable_into version -other_db public xyz BASE TABLE YES 6 +table_catalog table_schema table_name table_type is_insertable_into version +other_db public xyz BASE TABLE YES 6 + user root @@ -607,9 +611,9 @@ user testuser query TTTTTI colnames SELECT * FROM other_db.information_schema.tables WHERE table_catalog = 'other_db' AND table_schema = 'public' ORDER BY 1, 3 ---- -table_catalog table_schema table_name table_type is_insertable_into version -other_db public abc VIEW NO 2 -other_db public xyz BASE TABLE YES 6 +table_catalog table_schema table_name table_type is_insertable_into version +other_db public abc VIEW NO 2 +other_db public xyz BASE TABLE YES 6 user root @@ -1072,6 +1076,7 @@ NULL public system crdb_internal node_runtime_info NULL public system crdb_internal node_sessions SELECT NULL YES NULL public system crdb_internal node_statement_statistics SELECT NULL YES NULL public system crdb_internal partitions SELECT NULL YES +NULL public system crdb_internal predefined_comments SELECT NULL YES NULL public system crdb_internal ranges SELECT NULL YES NULL public system crdb_internal ranges_no_leases SELECT NULL YES NULL public system crdb_internal schema_changes SELECT NULL YES @@ -1311,6 +1316,7 @@ NULL public system crdb_internal node_runtime_info NULL public system crdb_internal node_sessions SELECT NULL YES NULL public system crdb_internal node_statement_statistics SELECT NULL YES NULL public system crdb_internal partitions SELECT NULL YES +NULL public system crdb_internal predefined_comments SELECT NULL YES NULL public system crdb_internal ranges SELECT NULL YES NULL public system crdb_internal ranges_no_leases SELECT NULL YES NULL public system crdb_internal schema_changes SELECT NULL YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 0be4f379e61b..cad6ba2391fc 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -771,8 +771,8 @@ FROM pg_catalog.pg_depend ORDER BY objid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967233 178791267 0 4294967235 450499961 0 n -4294967233 3318155331 0 4294967235 450499960 0 n +4294967232 178791267 0 4294967234 450499961 0 n +4294967232 3318155331 0 4294967234 450499960 0 n # All entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. @@ -784,7 +784,7 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967233 4294967235 pg_constraint pg_class +4294967232 4294967234 pg_constraint pg_class # All entries in pg_depend are foreign key constraints that reference an index # in pg_class. @@ -1257,9 +1257,101 @@ testuser 2264919399 false false false false ******** NU ## pg_catalog.pg_description query OOIT colnames -SELECT objoid, classoid, objsubid, description FROM pg_catalog.pg_description ----- -objoid classoid objsubid description +SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS description + FROM pg_catalog.pg_description +---- +objoid classoid objsubid description +4294967294 0 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 0 0 built-in functions (RAM/static) +4294967291 0 0 running queries visible by current user (cluster RPC; expensive!) +4294967290 0 0 running sessions visible to current user (cluster RPC; expensive!) +4294967289 0 0 cluster settings (RAM) +4294967288 0 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967287 0 0 telemetry counters (RAM; local node only) +4294967286 0 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967284 0 0 locally known gossiped health alerts (RAM; local node only) +4294967283 0 0 locally known gossiped node liveness (RAM; local node only) +4294967282 0 0 locally known edges in the gossip network (RAM; local node only) +4294967285 0 0 locally known gossiped node details (RAM; local node only) +4294967281 0 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967280 0 0 decoded job metadata from system.jobs (KV scan) +4294967279 0 0 node details across the entire cluster (cluster RPC; expensive!) +4294967278 0 0 store details and status (cluster RPC; expensive!) +4294967277 0 0 acquired table leases (RAM; local node only) +4294967293 0 0 detailed identification strings (RAM, local node only) +4294967274 0 0 current values for metrics (RAM; local node only) +4294967276 0 0 running queries visible by current user (RAM; local node only) +4294967269 0 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967275 0 0 running sessions visible by current user (RAM; local node only) +4294967265 0 0 statement statistics (RAM; local node only) +4294967273 0 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967272 0 0 comments for predefined virtual tables (RAM/static) +4294967271 0 0 range metadata without leaseholder details (KV join; expensive!) +4294967268 0 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967267 0 0 session trace accumulated so far (RAM) +4294967266 0 0 session variables (RAM) +4294967264 0 0 details for all columns accessible by current user in current database (KV scan) +4294967263 0 0 indexes accessible by current user in current database (KV scan) +4294967262 0 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967261 0 0 decoded zone configurations from system.zones (KV scan) +4294967259 0 0 roles for which the current user has admin option +4294967258 0 0 roles available to the current user +4294967257 0 0 column privilege grants (incomplete) +4294967256 0 0 table and view columns (incomplete) +4294967255 0 0 columns usage by constraints +4294967254 0 0 roles for the current user +4294967253 0 0 column usage by indexes and key constraints +4294967252 0 0 built-in function parameters (empty - introspection not yet supported) +4294967251 0 0 foreign key constraints +4294967250 0 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967249 0 0 built-in functions (empty - introspection not yet supported) +4294967247 0 0 schema privileges (incomplete; may contain excess users or roles) +4294967248 0 0 database schemas (may contain schemata without permission) +4294967246 0 0 sequences +4294967245 0 0 index metadata and statistics (incomplete) +4294967244 0 0 table constraints +4294967243 0 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967242 0 0 tables and views +4294967240 0 0 grantable privileges (incomplete) +4294967241 0 0 views (incomplete) +4294967238 0 0 index access methods (incomplete) +4294967237 0 0 column default values +4294967236 0 0 table columns (incomplete - see also information_schema.columns) +4294967235 0 0 role membership +4294967234 0 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967233 0 0 available collations (incomplete) +4294967232 0 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967231 0 0 available databases (incomplete) +4294967230 0 0 dependency relationships (incomplete) +4294967229 0 0 object comments +4294967227 0 0 enum types and labels (empty - feature does not exist) +4294967226 0 0 installed extensions (empty - feature does not exist) +4294967225 0 0 foreign data wrappers (empty - feature does not exist) +4294967224 0 0 foreign servers (empty - feature does not exist) +4294967223 0 0 foreign tables (empty - feature does not exist) +4294967222 0 0 indexes (incomplete) +4294967221 0 0 index creation statements +4294967220 0 0 table inheritance hierarchy (empty - feature does not exist) +4294967219 0 0 available languages (empty - feature does not exist) +4294967218 0 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967217 0 0 operators (incomplete) +4294967216 0 0 built-in functions (incomplete) +4294967215 0 0 range types (empty - feature does not exist) +4294967214 0 0 rewrite rules (empty - feature does not exist) +4294967213 0 0 database roles +4294967202 0 0 security labels (empty - feature does not exist) +4294967212 0 0 sequences (see also information_schema.sequences) +4294967211 0 0 session variables (incomplete) +4294967228 0 0 shared object comments (empty - feature does not exist) +4294967201 0 0 shared security labels (empty - feature not supported) +4294967203 0 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967208 0 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967207 0 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967206 0 0 triggers (empty - feature does not exist) +4294967205 0 0 scalar types (incomplete) +4294967210 0 0 database users +4294967209 0 0 local to remote user mapping (empty - feature does not exist) +4294967204 0 0 view definitions (incomplete - see also information_schema.views) ## pg_catalog.pg_shdescription @@ -1839,4 +1931,3 @@ SELECT pg_catalog.set_config('woo', 'woo', false) query error configuration setting.*not supported SELECT pg_catalog.set_config('vacuum_cost_delay', '0', false) - diff --git a/pkg/sql/logictest/testdata/logic_test/pgoidtype b/pkg/sql/logictest/testdata/logic_test/pgoidtype index 40bdeb7fc827..75f574306998 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgoidtype +++ b/pkg/sql/logictest/testdata/logic_test/pgoidtype @@ -61,7 +61,7 @@ pg_constraint pg_constraint query OO SELECT 'pg_constraint '::REGCLASS, '"pg_constraint"'::REGCLASS::OID ---- -pg_constraint 4294967233 +pg_constraint 4294967232 query O SELECT 4061301040::REGCLASS @@ -73,7 +73,7 @@ SELECT oid, oid::regclass, oid::regclass::int, oid::regclass::int::regclass, oid FROM pg_class WHERE relname = 'pg_constraint' ---- -4294967233 pg_constraint 4294967233 pg_constraint pg_constraint +4294967232 pg_constraint 4294967232 pg_constraint pg_constraint query OOOO SELECT 'upper'::REGPROC, 'upper'::REGPROCEDURE, 'pg_catalog.upper'::REGPROCEDURE, 'upper'::REGPROC::OID @@ -160,7 +160,7 @@ pg_constraint query OO SELECT ('pg_constraint')::REGCLASS, ('pg_constraint')::REGCLASS::OID ---- -pg_constraint 4294967233 +pg_constraint 4294967232 ## Test visibility of pg_* via oid casts. diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index a9d6e9006baa..32b218ee3d5c 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -175,21 +175,21 @@ query TT colnames SELECT * FROM [SHOW TABLES FROM system WITH COMMENT] ---- table_name comment -comments NULL -descriptor NULL -eventlog NULL -jobs NULL -lease NULL -locations NULL -namespace NULL -rangelog NULL -role_members NULL -settings NULL -table_statistics NULL -ui NULL -users NULL -web_sessions NULL -zones NULL +namespace · +descriptor · +users · +zones · +settings · +lease · +eventlog · +rangelog · +ui · +jobs · +web_sessions · +table_statistics · +locations · +role_members · +comments · query ITTT colnames SELECT node_id, user_name, application_name, active_queries diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index cc02f3eae60e..6fb4cedbfff5 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -120,12 +120,12 @@ c INT8 false NULL · {pr query TT SHOW TABLES FROM test WITH COMMENT ---- -a a_comment -b NULL -c NULL -d NULL -e NULL -f NULL +a a_comment +b · +c · +d · +e · +f · statement ok SET DATABASE = "" diff --git a/pkg/sql/logictest/testdata/planner_test/explain b/pkg/sql/logictest/testdata/planner_test/explain index ccd5f8dd2201..12c460372164 100644 --- a/pkg/sql/logictest/testdata/planner_test/explain +++ b/pkg/sql/logictest/testdata/planner_test/explain @@ -167,24 +167,22 @@ sort · · query TTT SELECT * FROM [EXPLAIN SHOW TABLES WITH COMMENT] WHERE field != 'size' ---- -sort · · - │ order +table_schema,+table_name - └── render · · - └── join · · - │ type left outer - │ equality (table_id) = (object_id) - ├── filter · · - │ │ filter ((state = 'PUBLIC') OR (state IS NULL)) AND ((database_name = 'test') OR (database_name IS NULL)) - │ └── join · · - │ │ type left outer - │ │ equality (table_name, table_catalog) = (name, database_name) - │ ├── filter · · - │ │ │ filter table_schema = 'public' - │ │ └── values · · - │ └── values · · - └── scan · · -· table comments@primary -· spans ALL +render · · + └── join · · + │ type left outer + │ equality (oid) = (objoid) + ├── join · · + │ │ type inner + │ │ equality (relnamespace) = (oid) + │ ├── filter · · + │ │ │ filter relkind IN ('r', 'v') + │ │ └── values · · + │ └── filter · · + │ │ filter nspname = 'public' + │ └── values · · + └── filter · · + │ filter objsubid = 0 + └── values · · query TTT SELECT * FROM [EXPLAIN SHOW DATABASE] WHERE field != 'size' diff --git a/pkg/sql/logictest/testdata/planner_test/join b/pkg/sql/logictest/testdata/planner_test/join index 0d63286ddc06..617b396cd577 100644 --- a/pkg/sql/logictest/testdata/planner_test/join +++ b/pkg/sql/logictest/testdata/planner_test/join @@ -388,7 +388,7 @@ SELECT level, node_type, field, description FROM [EXPLAIN (VERBOSE) SELECT 10 · type inner 10 · equality (refobjid) = (oid) 11 filter · · -11 · filter (dep.classid = 4294967233) AND (dep.refclassid = 4294967235) +11 · filter (dep.classid = 4294967232) AND (dep.refclassid = 4294967234) 11 filter · · 11 · filter pkic.relkind = 'i' diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 60a797aa7460..3971b60bb85f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -157,24 +157,22 @@ sort · · query TTT SELECT * FROM [EXPLAIN SHOW TABLES WITH COMMENT] WHERE field != 'size' ---- -sort · · - │ order +table_schema,+table_name - └── render · · - └── join · · - │ type left outer - │ equality (table_id) = (object_id) - ├── filter · · - │ │ filter ((state = 'PUBLIC') OR (state IS NULL)) AND ((database_name = 'test') OR (database_name IS NULL)) - │ └── join · · - │ │ type left outer - │ │ equality (table_name, table_catalog) = (name, database_name) - │ ├── filter · · - │ │ │ filter table_schema = 'public' - │ │ └── values · · - │ └── values · · - └── scan · · -· table comments@primary -· spans ALL +render · · + └── join · · + │ type left outer + │ equality (oid) = (objoid) + ├── join · · + │ │ type inner + │ │ equality (relnamespace) = (oid) + │ ├── filter · · + │ │ │ filter relkind IN ('r', 'v') + │ │ └── values · · + │ └── filter · · + │ │ filter nspname = 'public' + │ └── values · · + └── filter · · + │ filter objsubid = 0 + └── values · · query TTT SELECT * FROM [EXPLAIN SHOW DATABASE] WHERE field != 'size' diff --git a/pkg/sql/opt/exec/execbuilder/testdata/join b/pkg/sql/opt/exec/execbuilder/testdata/join index 406860ad628e..23a42e0564ec 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/join +++ b/pkg/sql/opt/exec/execbuilder/testdata/join @@ -376,7 +376,7 @@ SELECT level, node_type, field, description FROM [EXPLAIN (VERBOSE) SELECT 10 · type inner 10 · equality (refobjid) = (oid) 11 filter · · -11 · filter (classid = 4294967233) AND (refclassid = 4294967235) +11 · filter (classid = 4294967232) AND (refclassid = 4294967234) 12 virtual table · · 12 · source · 11 filter · · diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index ecd54ad21360..e1f9aaff6a6a 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -1986,7 +1986,8 @@ comment_stmt: comment_text: SCONST { - $$.val = &$1 + t := $1 + $$.val = &t } | NULL { diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index fe9dff0dd65c..89f5875bf059 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -232,6 +232,8 @@ var pgCatalog = virtualSchema{ // See: https://www.postgresql.org/docs/9.5/static/catalog-pg-am.html and // https://www.postgresql.org/docs/9.6/static/catalog-pg-am.html. var pgCatalogAmTable = virtualSchemaTable{ + comment: `index access methods (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-am.html`, schema: ` CREATE TABLE pg_catalog.pg_am ( oid OID, @@ -309,8 +311,9 @@ CREATE TABLE pg_catalog.pg_am ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-attrdef.html. var pgCatalogAttrDefTable = virtualSchemaTable{ + comment: `column default values +https://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html`, schema: ` CREATE TABLE pg_catalog.pg_attrdef ( oid OID, @@ -343,8 +346,9 @@ CREATE TABLE pg_catalog.pg_attrdef ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-attribute.html. var pgCatalogAttributeTable = virtualSchemaTable{ + comment: `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, @@ -421,8 +425,9 @@ CREATE TABLE pg_catalog.pg_attribute ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-auth-members.html. var pgCatalogAuthMembersTable = virtualSchemaTable{ + comment: `role membership +https://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html`, schema: ` CREATE TABLE pg_catalog.pg_auth_members ( roleid OID, @@ -453,8 +458,9 @@ var ( relPersistencePermanent = tree.NewDString("p") ) -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-class.html. var pgCatalogClassTable = virtualSchemaTable{ + comment: `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, @@ -572,8 +578,9 @@ CREATE TABLE pg_catalog.pg_class ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-collation.html. var pgCatalogCollationTable = virtualSchemaTable{ + comment: `available collations (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-collation.html`, schema: ` CREATE TABLE pg_catalog.pg_collation ( oid OID, @@ -646,8 +653,9 @@ var ( } ) -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-constraint.html. var pgCatalogConstraintTable = virtualSchemaTable{ + comment: `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, @@ -854,8 +862,9 @@ func colIDArrayToVector(arr []sqlbase.ColumnID) (tree.Datum, error) { return tree.NewDIntVectorFromDArray(tree.MustBeDArray(dArr)), nil } -// See https://www.postgresql.org/docs/9.6/static/catalog-pg-database.html. var pgCatalogDatabaseTable = virtualSchemaTable{ + comment: `available databases (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-database.html`, schema: ` CREATE TABLE pg_catalog.pg_database ( oid OID, @@ -913,8 +922,6 @@ var ( pgClassTableName = tree.MakeTableNameWithSchema("", tree.Name(pgCatalogName), tree.Name("pg_class")) ) -// See https://www.postgresql.org/docs/9.6/static/catalog-pg-depend.html. -// // pg_depend is a fairly complex table that details many different kinds of // relationships between database objects. We do not implement the vast // majority of this table, as it is mainly used by pgjdbc to address a @@ -924,6 +931,8 @@ var ( // provide those rows in pg_depend that track the dependency of foreign key // constraints on their supporting index entries in pg_class. var pgCatalogDependTable = virtualSchemaTable{ + comment: `dependency relationships (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-depend.html`, schema: ` CREATE TABLE pg_catalog.pg_depend ( classid OID, @@ -987,8 +996,9 @@ CREATE TABLE pg_catalog.pg_depend ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-description.html. var pgCatalogDescriptionTable = virtualSchemaTable{ + comment: `object comments +https://www.postgresql.org/docs/9.5/catalog-pg-description.html`, schema: ` CREATE TABLE pg_catalog.pg_description ( objoid OID, @@ -1005,57 +1015,33 @@ CREATE TABLE pg_catalog.pg_description ( ctx, "select-comments", p.EvalContext().Txn, - "SELECT object_id, sub_id, comment FROM system.comments") + `SELECT COALESCE(pc.object_id, sc.object_id) AS object_id, + COALESCE(pc.sub_id, sc.sub_id) AS sub_id, + COALESCE(pc.comment, sc.comment) AS comment + FROM (SELECT * FROM system.comments) AS sc + FULL JOIN (SELECT * FROM crdb_internal.predefined_comments) AS pc + ON (pc.object_id = sc.object_id AND pc.sub_id = sc.sub_id AND pc.type = sc.type)`) if err != nil { return err } - commentMap := make(map[tree.DInt]tree.Datums) for _, comment := range comments { - id := *comment[0].(*tree.DInt) - commentMap[id] = comment - } - - err = forEachTableDescWithTableLookup( - ctx, - p, - dbContext, - hideVirtual, - func( - db *sqlbase.DatabaseDescriptor, - scName string, - table *sqlbase.TableDescriptor, - tableLookup tableLookupFn) error { - if comment, ok := commentMap[tree.DInt(table.ID)]; ok { - return addRow( - defaultOid(table.ID), - oidZero, - comment[1], - comment[2]) - } - - return nil - }) - if err != nil { - return err - } - - return forEachDatabaseDesc(ctx, p, nil /*all databases*/, func(db *sqlbase.DatabaseDescriptor) error { - if comment, ok := commentMap[tree.DInt(db.ID)]; ok { - return addRow( - defaultOid(db.ID), - oidZero, - comment[1], - comment[2]) + objID := sqlbase.ID(*comment[0].(*tree.DInt)) + if err := addRow( + defaultOid(objID), + oidZero, + comment[1], + comment[2]); err != nil { + return err } - - return nil - }) + } + return nil }, } -// See: https://www.postgresql.org/docs/current/static/catalog-pg-shdescription.html. var pgCatalogSharedDescriptionTable = virtualSchemaTable{ + comment: `shared object comments (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html`, schema: ` CREATE TABLE pg_catalog.pg_shdescription ( objoid OID, @@ -1068,8 +1054,9 @@ CREATE TABLE pg_catalog.pg_shdescription ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-enum.html. var pgCatalogEnumTable = virtualSchemaTable{ + comment: `enum types and labels (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-enum.html`, schema: ` CREATE TABLE pg_catalog.pg_enum ( oid OID, @@ -1083,8 +1070,9 @@ CREATE TABLE pg_catalog.pg_enum ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-extension.html. var pgCatalogExtensionTable = virtualSchemaTable{ + comment: `installed extensions (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-extension.html`, schema: ` CREATE TABLE pg_catalog.pg_extension ( extname NAME, @@ -1101,8 +1089,9 @@ CREATE TABLE pg_catalog.pg_extension ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-foreign-data-wrapper.html. var pgCatalogForeignDataWrapperTable = virtualSchemaTable{ + comment: `foreign data wrappers (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html`, schema: ` CREATE TABLE pg_catalog.pg_foreign_data_wrapper ( oid OID, @@ -1119,8 +1108,9 @@ CREATE TABLE pg_catalog.pg_foreign_data_wrapper ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-foreign-server.html. var pgCatalogForeignServerTable = virtualSchemaTable{ + comment: `foreign servers (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html`, schema: ` CREATE TABLE pg_catalog.pg_foreign_server ( oid OID, @@ -1138,8 +1128,9 @@ CREATE TABLE pg_catalog.pg_foreign_server ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-foreign-table.html. var pgCatalogForeignTableTable = virtualSchemaTable{ + comment: `foreign tables (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html`, schema: ` CREATE TABLE pg_catalog.pg_foreign_table ( ftrelid OID, @@ -1172,8 +1163,9 @@ func makeZeroedIntVector(size int) (tree.Datum, error) { return tree.NewDIntVectorFromDArray(intArray), nil } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-index.html. var pgCatalogIndexTable = virtualSchemaTable{ + comment: `indexes (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, schema: ` CREATE TABLE pg_catalog.pg_index ( indexrelid OID, @@ -1258,11 +1250,11 @@ CREATE TABLE pg_catalog.pg_index ( }, } -// See: https://www.postgresql.org/docs/9.6/static/view-pg-indexes.html. -// -// Note that crdb_oid is an extension of the schema to much more easily map -// index OIDs to the corresponding index definition. var pgCatalogIndexesTable = virtualSchemaTable{ + comment: `index creation statements +https://www.postgresql.org/docs/9.5/view-pg-indexes.html`, + // Note: crdb_oid is an extension of the schema to much more easily map + // index OIDs to the corresponding index definition. schema: ` CREATE TABLE pg_catalog.pg_indexes ( crdb_oid OID, @@ -1354,8 +1346,9 @@ func indexDefFromDescriptor( return indexDef.String(), nil } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-inherits.html. var pgCatalogInheritsTable = virtualSchemaTable{ + comment: `table inheritance hierarchy (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-inherits.html`, schema: ` CREATE TABLE pg_catalog.pg_inherits ( inhrelid OID, @@ -1368,8 +1361,9 @@ CREATE TABLE pg_catalog.pg_inherits ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-language.html. var pgCatalogLanguageTable = virtualSchemaTable{ + comment: `available languages (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-language.html`, schema: ` CREATE TABLE pg_catalog.pg_language ( oid OID, @@ -1388,8 +1382,9 @@ CREATE TABLE pg_catalog.pg_language ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-namespace.html. var pgCatalogNamespaceTable = virtualSchemaTable{ + comment: `available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +https://www.postgresql.org/docs/9.5/catalog-pg-namespace.html`, schema: ` CREATE TABLE pg_catalog.pg_namespace ( oid OID, @@ -1421,8 +1416,9 @@ var ( _ = postfixKind ) -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-operator.html. var pgCatalogOperatorTable = virtualSchemaTable{ + comment: `operators (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-operator.html`, schema: ` CREATE TABLE pg_catalog.pg_operator ( oid OID, @@ -1535,8 +1531,9 @@ var ( _ = proArgModeTable ) -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-proc.html. var pgCatalogProcTable = virtualSchemaTable{ + comment: `built-in functions (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`, schema: ` CREATE TABLE pg_catalog.pg_proc ( oid OID, @@ -1692,8 +1689,9 @@ CREATE TABLE pg_catalog.pg_proc ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-range.html. var pgCatalogRangeTable = virtualSchemaTable{ + comment: `range types (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-range.html`, schema: ` CREATE TABLE pg_catalog.pg_range ( rngtypid OID, @@ -1711,8 +1709,9 @@ CREATE TABLE pg_catalog.pg_range ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-rewrite.html. var pgCatalogRewriteTable = virtualSchemaTable{ + comment: `rewrite rules (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html`, schema: ` CREATE TABLE pg_catalog.pg_rewrite ( oid OID, @@ -1730,8 +1729,9 @@ CREATE TABLE pg_catalog.pg_rewrite ( }, } -// See: https://www.postgresql.org/docs/9.6/static/view-pg-roles.html. var pgCatalogRolesTable = virtualSchemaTable{ + comment: `database roles +https://www.postgresql.org/docs/9.5/view-pg-roles.html`, schema: ` CREATE TABLE pg_catalog.pg_roles ( oid OID, @@ -1779,8 +1779,9 @@ CREATE TABLE pg_catalog.pg_roles ( }, } -// See: https://www.postgresql.org/docs/10/static/catalog-pg-sequence.html. var pgCatalogSequencesTable = virtualSchemaTable{ + comment: `sequences (see also information_schema.sequences) +https://www.postgresql.org/docs/9.5/catalog-pg-sequence.html`, schema: ` CREATE TABLE pg_catalog.pg_sequence ( seqrelid OID, @@ -1818,8 +1819,9 @@ var ( settingsCtxUser = tree.NewDString("user") ) -// See: https://www.postgresql.org/docs/9.6/static/view-pg-settings.html. var pgCatalogSettingsTable = virtualSchemaTable{ + comment: `session variables (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-settings.html`, schema: ` CREATE TABLE pg_catalog.pg_settings ( name STRING, @@ -1890,8 +1892,9 @@ CREATE TABLE pg_catalog.pg_settings ( }, } -// See: https://www.postgresql.org/docs/9.6/static/view-pg-tables.html. var pgCatalogTablesTable = virtualSchemaTable{ + comment: `tables summary (see also information_schema.tables, pg_catalog.pg_class) +https://www.postgresql.org/docs/9.5/view-pg-tables.html`, schema: ` CREATE TABLE pg_catalog.pg_tables ( schemaname NAME, @@ -1926,8 +1929,9 @@ CREATE TABLE pg_catalog.pg_tables ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-tablespace.html. var pgCatalogTablespaceTable = virtualSchemaTable{ + comment: `available tablespaces (incomplete; concept inapplicable to CockroachDB) +https://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html`, schema: ` CREATE TABLE pg_catalog.pg_tablespace ( oid OID, @@ -1949,8 +1953,9 @@ CREATE TABLE pg_catalog.pg_tablespace ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-trigger.html. var pgCatalogTriggerTable = virtualSchemaTable{ + comment: `triggers (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-trigger.html`, schema: ` CREATE TABLE pg_catalog.pg_trigger ( oid OID, @@ -2021,8 +2026,9 @@ var ( typDelim = tree.NewDString(",") ) -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-type.html. var pgCatalogTypeTable = virtualSchemaTable{ + comment: `scalar types (incomplete) +https://www.postgresql.org/docs/9.5/catalog-pg-type.html`, schema: ` CREATE TABLE pg_catalog.pg_type ( oid OID, @@ -2137,8 +2143,9 @@ CREATE TABLE pg_catalog.pg_type ( }, } -// See: https://www.postgresql.org/docs/9.6/static/view-pg-user.html. var pgCatalogUserTable = virtualSchemaTable{ + comment: `database users +https://www.postgresql.org/docs/9.5/view-pg-user.html`, schema: ` CREATE TABLE pg_catalog.pg_user ( usename NAME, @@ -2174,8 +2181,9 @@ CREATE TABLE pg_catalog.pg_user ( }, } -// See: https://www.postgresql.org/docs/9.6/static/catalog-pg-user-mapping.html. var pgCatalogUserMappingTable = virtualSchemaTable{ + comment: `local to remote user mapping (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html`, schema: ` CREATE TABLE pg_catalog.pg_user_mapping ( oid OID, @@ -2190,8 +2198,9 @@ CREATE TABLE pg_catalog.pg_user_mapping ( }, } -// See: https://www.postgresql.org/docs/9.6/static/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW var pgCatalogStatActivityTable = virtualSchemaTable{ + comment: `backend access statistics (empty - monitoring works differently in CockroachDB) +https://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW`, schema: ` CREATE TABLE pg_catalog.pg_stat_activity ( datid OID, @@ -2220,9 +2229,9 @@ CREATE TABLE pg_catalog.pg_stat_activity ( }, } -// -// See https://www.postgresql.org/docs/current/static/catalog-pg-seclabel.html var pgCatalogSecurityLabelTable = virtualSchemaTable{ + comment: `security labels (empty - feature does not exist) +https://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html`, schema: ` CREATE TABLE pg_catalog.pg_seclabel ( objoid OID, @@ -2237,8 +2246,9 @@ CREATE TABLE pg_catalog.pg_seclabel ( }, } -// See https://www.postgresql.org/docs/current/static/catalog-pg-shseclabel.html var pgCatalogSharedSecurityLabelTable = virtualSchemaTable{ + comment: `shared security labels (empty - feature not supported) +https://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html`, schema: ` CREATE TABLE pg_catalog.pg_shseclabel ( objoid OID, @@ -2317,8 +2327,9 @@ func typCategory(typ types.T) tree.Datum { return datumToTypeCategory[reflect.TypeOf(types.UnwrapType(typ))] } -// See: https://www.postgresql.org/docs/9.6/static/view-pg-views.html. var pgCatalogViewsTable = virtualSchemaTable{ + comment: `view definitions (incomplete - see also information_schema.views) +https://www.postgresql.org/docs/9.5/view-pg-views.html`, schema: ` CREATE TABLE pg_catalog.pg_views ( schemaname NAME, diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index e86d82ee6d58..a987b2882c77 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -712,17 +712,16 @@ var pgBuiltins = map[string]builtinDefinition{ Types: tree.ArgTypes{{"table_oid", types.Oid}, {"column_number", types.Int}}, ReturnType: tree.FixedReturnType(types.String), Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { - oid, ok := args[0].(*tree.DOid) - if !ok { + if *args[1].(*tree.DInt) == 0 { + // column ID 0 never exists, and we don't want the query + // below to pick up the table comment by accident. return tree.DNull, nil } - r, err := ctx.InternalExecutor.QueryRow( ctx.Ctx(), "pg_get_coldesc", ctx.Txn, - "SELECT description FROM pg_catalog.pg_description WHERE objoid=$1 AND objsubid=$2 LIMIT 1;", - oid.DInt, - args[1]) + "SELECT description FROM pg_catalog.pg_description WHERE objoid=$1 AND objsubid=$2 LIMIT 1", + args[0], args[1]) if err != nil { return nil, err } @@ -740,30 +739,17 @@ var pgBuiltins = map[string]builtinDefinition{ Types: tree.ArgTypes{{"object_oid", types.Oid}}, ReturnType: tree.FixedReturnType(types.String), Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { - oid, ok := args[0].(*tree.DOid) - if !ok { - return tree.DNull, nil - } - - r, err := ctx.InternalExecutor.QueryRow( - ctx.Ctx(), "pg_get_objdesc", - ctx.Txn, - "SELECT description FROM pg_catalog.pg_description WHERE objoid=$1 LIMIT 1", oid.DInt) - if err != nil { - return nil, err - } - if len(r) == 0 { - return tree.DNull, nil - } - return r[0], nil + return getPgObjDesc(ctx, &ctx.SessionData.Database, int(args[0].(*tree.DOid).DInt)) }, Info: notUsableInfo, }, tree.Overload{ Types: tree.ArgTypes{{"object_oid", types.Oid}, {"catalog_name", types.String}}, ReturnType: tree.FixedReturnType(types.String), - Fn: func(_ *tree.EvalContext, _ tree.Datums) (tree.Datum, error) { - return tree.DNull, nil + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + return getPgObjDesc(ctx, + (*string)(args[1].(*tree.DString)), + int(args[0].(*tree.DOid).DInt)) }, Info: notUsableInfo, }, @@ -1600,3 +1586,23 @@ func setSessionVar(ctx *tree.EvalContext, settingName, newVal string, isLocal bo } return ctx.SessionAccessor.SetSessionVar(ctx.Context, settingName, newVal) } + +func getPgObjDesc(ctx *tree.EvalContext, dbName *string, oid int) (tree.Datum, error) { + r, err := ctx.InternalExecutor.QueryRow( + ctx.Ctx(), "pg_get_objdesc", ctx.Txn, + fmt.Sprintf(` +SELECT description + FROM %[1]s.pg_catalog.pg_description + WHERE objoid = %[2]d + AND objsubid = 0 + LIMIT 1`, + (*tree.Name)(dbName), + oid)) + if err != nil { + return nil, err + } + if len(r) == 0 { + return tree.DNull, nil + } + return r[0], nil +} diff --git a/pkg/sql/show_tables.go b/pkg/sql/show_tables.go index a1f8b0aa4035..26c67cdb0a87 100644 --- a/pkg/sql/show_tables.go +++ b/pkg/sql/show_tables.go @@ -42,34 +42,20 @@ func (p *planner) ShowTables(ctx context.Context, n *tree.ShowTables) (planNode, var query string if n.WithComment { - // TODO(knz): this 3-way join is painful and would really benefit - // from vtables having a real table ID (#32963) so that we don't - // have to use `information_schema.tables` and simply join - // `crdb_internal.tables` with `system.comments` instead. const getTablesQuery = ` SELECT - i.table_name, c.comment -FROM - %[1]s.information_schema.tables AS i - LEFT JOIN crdb_internal.tables AS t - ON - i.table_name = t.name - AND i.table_catalog = t.database_name - LEFT JOIN system.comments AS c - ON t.table_id = c.object_id -WHERE - table_schema = %[2]s - AND (t.state = %[3]s OR t.state IS NULL) - AND (t.database_name = %[4]s OR t.database_name IS NULL) -ORDER BY - table_schema, table_name` + pc.relname AS table_name, + COALESCE(pd.description, '') AS comment + FROM %[1]s.pg_catalog.pg_class AS pc + JOIN %[1]s.pg_catalog.pg_namespace AS ns ON (ns.oid = pc.relnamespace) +LEFT JOIN %[1]s.pg_catalog.pg_description AS pd ON (pc.oid = pd.objoid AND pd.objsubid = 0) +WHERE ns.nspname = %[2]s + AND pc.relkind IN ('r', 'v')` query = fmt.Sprintf( getTablesQuery, &n.CatalogName, - lex.EscapeSQLString(n.Schema()), - lex.EscapeSQLString(sqlbase.TableDescriptor_PUBLIC.String()), - lex.EscapeSQLString(n.CatalogName.Normalize())) + lex.EscapeSQLString(n.Schema())) } else { const getTablesQuery = ` diff --git a/pkg/sql/sqlbase/constants.go b/pkg/sql/sqlbase/constants.go index b6f4691df6e8..2602b1287f99 100644 --- a/pkg/sql/sqlbase/constants.go +++ b/pkg/sql/sqlbase/constants.go @@ -57,6 +57,7 @@ const ( CrdbInternalLocalSessionsTableID CrdbInternalLocalMetricsTableID CrdbInternalPartitionsTableID + CrdbInternalPredefinedCommentsTableID CrdbInternalRangesNoLeasesTableID CrdbInternalRangesViewID CrdbInternalRuntimeInfoTableID diff --git a/pkg/sql/stats/automatic_stats.go b/pkg/sql/stats/automatic_stats.go index e3eb914a992b..eecdafe4d0b7 100644 --- a/pkg/sql/stats/automatic_stats.go +++ b/pkg/sql/stats/automatic_stats.go @@ -273,7 +273,7 @@ func (r *Refresher) ensureAllTables( // Use a historical read so as to disable txn contention resolution. getAllTablesQuery := fmt.Sprintf( - `SELECT table_id FROM crdb_internal.tables AS OF SYSTEM TIME '-%s'`, + `SELECT table_id FROM crdb_internal.tables AS OF SYSTEM TIME '-%s' WHERE schema_name = 'public'`, initialTableCollectionDelay) rows, err := r.ex.Query( diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index abece6357b6b..ebcac50157e5 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -55,6 +55,7 @@ type virtualSchemaDef interface { initVirtualTableDesc( ctx context.Context, st *cluster.Settings, id sqlbase.ID, ) (sqlbase.TableDescriptor, error) + getComment() string } // virtualSchemaTable represents a table within a virtualSchema. @@ -63,6 +64,9 @@ type virtualSchemaTable struct { // each virtualSchemaTable. schema string + // comment represents comment of virtual schema table. + comment string + // populate, if non-nil, is a function that is used when creating a // valuesNode. This function eagerly loads every row of the virtual table // during initialization of the valuesNode. @@ -126,6 +130,11 @@ func (t virtualSchemaTable) initVirtualTableDesc( return mutDesc.TableDescriptor, err } +// getComment is part of the virtualSchemaDef interface. +func (t virtualSchemaTable) getComment() string { + return t.comment +} + // getSchema is part of the virtualSchemaDef interface. func (v virtualSchemaView) getSchema() string { return v.schema @@ -155,6 +164,11 @@ func (v virtualSchemaView) initVirtualTableDesc( return mutDesc.TableDescriptor, err } +// getComment is part of the virtualSchemaDef interface. +func (v virtualSchemaView) getComment() string { + return "" +} + // virtualSchemas holds a slice of statically registered virtualSchema objects. // // When adding a new virtualSchema, define a virtualSchema in a separate file, and @@ -190,6 +204,7 @@ type virtualSchemaEntry struct { type virtualDefEntry struct { virtualDef virtualSchemaDef desc *sqlbase.TableDescriptor + comment string validWithNoDatabaseContext bool } @@ -319,6 +334,7 @@ func NewVirtualSchemaHolder( virtualDef: def, desc: &tableDesc, validWithNoDatabaseContext: schema.validWithNoDatabaseContext, + comment: def.getComment(), } orderedDefNames = append(orderedDefNames, tableDesc.Name) }