From dc93f55bdf5bbf3f4c96bc60cf8eb125c641d90e Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Thu, 11 May 2023 04:15:27 +0000 Subject: [PATCH] pg_catalog: add virtual indexes and optimize existing ones Previously, the pg_description table was missing a virtual index on objoid, with the addition of descriptions from all builtins this table has much more data by default. As a result any query joining with this table is slow. To address this, this patch will first add a virtual index on the index description table, and update existing incomplete indexes on OID's to skip over builtins that refer to builtin functions. Release note (performance improvement): improve performance when joining with the pg_description table --- .../testdata/benchmark_expectations | 2 +- pkg/cli/clisqlshell/testdata/describe | 2 + pkg/sql/catalog/nstree/catalog.go | 2 + pkg/sql/crdb_internal.go | 12 - pkg/sql/information_schema.go | 2 +- .../logictest/testdata/logic_test/comment_on | 2 +- .../testdata/logic_test/crdb_internal_catalog | 2 +- pkg/sql/opt/exec/execbuilder/testdata/explain | 11 +- pkg/sql/pg_catalog.go | 266 +++++++++++------- pkg/sql/pg_metadata_test.go | 9 +- pkg/sql/vtable/pg_catalog.go | 3 +- 11 files changed, 183 insertions(+), 130 deletions(-) diff --git a/pkg/bench/rttanalysis/testdata/benchmark_expectations b/pkg/bench/rttanalysis/testdata/benchmark_expectations index d71fce0faed9..a1450d2d9cae 100644 --- a/pkg/bench/rttanalysis/testdata/benchmark_expectations +++ b/pkg/bench/rttanalysis/testdata/benchmark_expectations @@ -77,7 +77,7 @@ exp,benchmark 12,ORMQueries/hasura_column_descriptions_8_tables 5,ORMQueries/hasura_column_descriptions_modified 4,ORMQueries/information_schema._pg_index_position -134,ORMQueries/introspection_description_join +132,ORMQueries/introspection_description_join 4,ORMQueries/pg_attribute 4,ORMQueries/pg_class 6,ORMQueries/pg_is_other_temp_schema diff --git a/pkg/cli/clisqlshell/testdata/describe b/pkg/cli/clisqlshell/testdata/describe index 800927c78af1..6a93877cee15 100644 --- a/pkg/cli/clisqlshell/testdata/describe +++ b/pkg/cli/clisqlshell/testdata/describe @@ -303,6 +303,7 @@ pg_catalog,pg_db_role_setting,table,admin,NULL pg_catalog,pg_default_acl,table,admin,NULL pg_catalog,pg_depend,table,admin,NULL pg_catalog,pg_description,table,admin,NULL +pg_catalog,pg_description_objoid_idx,index,admin,NULL pg_catalog,pg_enum,table,admin,NULL pg_catalog,pg_event_trigger,table,admin,NULL pg_catalog,pg_extension,table,admin,NULL @@ -563,6 +564,7 @@ pg_catalog,pg_depend,table,admin,NULL,permanent,prefix,"dependency relationships https://www.postgresql.org/docs/9.5/catalog-pg-depend.html" pg_catalog,pg_description,table,admin,NULL,permanent,prefix,"object comments https://www.postgresql.org/docs/9.5/catalog-pg-description.html" +pg_catalog,pg_description_objoid_idx,index,admin,NULL,permanent,prefix, pg_catalog,pg_enum,table,admin,NULL,permanent,prefix,"enum types and labels (empty - feature does not exist) https://www.postgresql.org/docs/9.5/catalog-pg-enum.html" pg_catalog,pg_event_trigger,table,admin,NULL,permanent,prefix,"event triggers (empty - feature does not exist) diff --git a/pkg/sql/catalog/nstree/catalog.go b/pkg/sql/catalog/nstree/catalog.go index 1c307b1a6d19..d5f86dd72c35 100644 --- a/pkg/sql/catalog/nstree/catalog.go +++ b/pkg/sql/catalog/nstree/catalog.go @@ -35,6 +35,8 @@ type Catalog struct { // immutable catalogs that are incomeplete and only contain comment information. type CommentCatalog interface { ForEachComment(fn func(key catalogkeys.CommentKey, cmt string) error) error + ForEachCommentOnDescriptor( + id descpb.ID, fn func(key catalogkeys.CommentKey, cmt string) error) error LookupComment(key catalogkeys.CommentKey) (_ string, found bool) } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index a667d5922bfd..f7fc0a5b078d 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -5415,18 +5415,6 @@ CREATE TABLE crdb_internal.kv_catalog_comments ( if err != nil { return err } - // Delegate privilege check to system table. - { - sysTable, err := p.Descriptors().ByIDWithLeased(p.txn).Get().Table(ctx, systemschema.CommentsTable.GetID()) - if err != nil { - return err - } - if ok, err := p.HasPrivilege(ctx, sysTable, privilege.SELECT, p.User()); err != nil { - return err - } else if !ok { - return nil - } - } // Loop over all comment entries. // NB if ever anyone were to extend this table to carry column // comments, make sure to update pg_catalog.col_description to diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 2527ac7b7da9..ca1e607dc889 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -407,7 +407,7 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`, schema: vtable.InformationSchemaColumns, populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { // Get the collations for all comments of current database. - comments, err := getComments(ctx, p) + comments, err := getComments(ctx, p, descpb.InvalidID /* all comments */) if err != nil { return err } diff --git a/pkg/sql/logictest/testdata/logic_test/comment_on b/pkg/sql/logictest/testdata/logic_test/comment_on index 54c18958e809..e50f50ba34f4 100644 --- a/pkg/sql/logictest/testdata/logic_test/comment_on +++ b/pkg/sql/logictest/testdata/logic_test/comment_on @@ -266,7 +266,7 @@ CREATE TABLE t_99316(a INT); statement ok INSERT INTO system.comments VALUES (4294967122, 't_99316'::regclass::OID, 0, 'bar'); -statement error pgcode XX000 select-comments: invalid comment type 4294967122 +statement error pgcode XX000 internal error: select-comments: invalid comment type 4294967122 SELECT * FROM pg_catalog.pg_description WHERE objoid = 't'::regclass::OID; statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index c6fc81b30ac8..0eb7bc6441e8 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -281,7 +281,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967093 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "extname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "extowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "extnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "extrelocatable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "extversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "extconfig", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "extcondition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967093, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} 4294967094 {"table": {"columns": [{"id": 1, "name": "evtname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "evtevent", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "evtowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "evtfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "evtenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "evttags", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967094, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} 4294967095 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "enumtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "enumsortorder", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 4, "name": "enumlabel", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967095, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} -4294967096 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967096, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} +4294967096 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967096, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["objoid"], "name": "pg_description_objoid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["classoid", "objsubid", "description"], "version": 3}], "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} 4294967097 {"table": {"columns": [{"id": 1, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967097, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} 4294967098 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "defaclrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "defaclnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "defaclobjtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "defaclacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967098, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} 4294967099 {"table": {"columns": [{"id": 1, "name": "setconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 2, "name": "setdatabase", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "setrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967099, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967118, "version": "1"}} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index d15f43404507..2df593699ba1 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -436,14 +436,11 @@ vectorized: true │ │ │ └── • virtual table │ │ │ table: pg_class@pg_class_oid_idx │ │ │ - │ │ └── • sort - │ │ │ order: +objoid + │ │ └── • filter + │ │ │ filter: (objsubid = 0) AND (classoid = 4294967106) │ │ │ - │ │ └── • filter - │ │ │ filter: (objsubid = 0) AND (classoid = 4294967106) - │ │ │ - │ │ └── • virtual table - │ │ table: pg_description@primary + │ │ └── • virtual table + │ │ table: pg_description@pg_description_objoid_idx │ │ │ └── • virtual table │ table: table_row_statistics@primary diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 426606ee506c..57d3cf49dbc1 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -1125,9 +1125,10 @@ func makeAllRelationsVirtualTableWithDescriptorIDIndex( return false, errors.AssertionFailedf("unexpected type %T for table id column in virtual table %s", unwrappedConstraint, schemaDef) } - table, err := p.LookupTableByID(ctx, id) + desc, err := p.byIDGetterBuilder().WithoutNonPublic().Get().Desc(ctx, id) if err != nil { - if sqlerrors.IsUndefinedRelationError(err) { + if errors.Is(err, catalog.ErrDescriptorNotFound) || + catalog.HasInactiveDescriptorError(err) { // No table found, so no rows. In this case, we'll fall back to the // full table scan if the index isn't complete - see the // indexContainsNonTableDescriptorIDs parameter. @@ -1136,15 +1137,27 @@ func makeAllRelationsVirtualTableWithDescriptorIDIndex( } return false, err } + // If the descriptor is not a table, then we have a complete result + // from this virtual index. We can mark the result as populated, + // because we know the underlying descriptor will generate no rows, since + // the ID being queried is *not* a table. + table, ok := desc.(catalog.TableDescriptor) + if !ok { + return true, nil + } // Don't include tables that aren't in the current database unless // they're virtual, dropped tables, or ones that the user can't see. canSeeDescriptor, err := userCanSeeDescriptor(ctx, p, table, db, true /*allowAdding*/) if err != nil { return false, err } + // Skip over tables from a different DB, ones which aren't visible + // or are dropped. From a virtual index viewpoint, we will consider + // this result set as populated, since the underlying full table will + // also skip the same descriptors. if (!table.IsVirtualTable() && table.GetParentID() != db.GetID()) || table.Dropped() || !canSeeDescriptor { - return false, nil + return true, nil } h := makeOidHasher() scResolver := oneAtATimeSchemaResolver{p: p, ctx: ctx} @@ -1569,30 +1582,147 @@ https://www.postgresql.org/docs/9.5/catalog-pg-depend.html`, }, } -// getComments returns all comments in the database. A comment is represented -// as a datum row, containing object id, sub id (column id in the case of -// columns), comment text, and comment type (keys.FooCommentType). -func getComments(ctx context.Context, p *planner) ([]tree.Datums, error) { +// getComments returns all comments in the database when descriptorID is +// descpb.InvalidID, otherwise only comments from a target descriptor. +// A comment is represented as a datum row, containing object id, sub id +// (column id in the case of columns), comment text, and comment type +// (keys.FooCommentType). +func getComments(ctx context.Context, p *planner, descriptorID descpb.ID) ([]tree.Datums, error) { + queryText := strings.Builder{} + var queryArgs []interface{} + queryText.WriteString(`SELECT + object_id, + sub_id, + comment, + CASE type + WHEN 'DatabaseCommentType' THEN 0 + WHEN 'TableCommentType' THEN 1 + WHEN 'ColumnCommentType' THEN 2 + WHEN 'IndexCommentType' THEN 3 + WHEN 'SchemaCommentType' THEN 4 + WHEN 'ConstraintCommentType' THEN 5 + END + AS type + FROM + "".crdb_internal.kv_catalog_comments`) + // If necessary filter by ID. + if descriptorID != descpb.InvalidID { + queryText.WriteString(` +WHERE object_id=$1`) + queryArgs = []interface{}{descriptorID} + } + return p.InternalSQLTxn().QueryBufferedEx( ctx, "select-comments", p.Txn(), sessiondata.NodeUserSessionDataOverride, - `SELECT - object_id, - sub_id, - comment, - CASE type - WHEN 'DatabaseCommentType' THEN 0 - WHEN 'TableCommentType' THEN 1 - WHEN 'ColumnCommentType' THEN 2 - WHEN 'IndexCommentType' THEN 3 - WHEN 'SchemaCommentType' THEN 4 - WHEN 'ConstraintCommentType' THEN 5 - END - AS type - FROM - "".crdb_internal.kv_catalog_comments;`) + queryText.String(), + queryArgs...) +} + +// populatePgCatalogFromComments populates the for the pg_description table +// based on a set of fetched comments. +func populatePgCatalogFromComments( + ctx context.Context, + p *planner, + dbContext catalog.DatabaseDescriptor, + addRow func(...tree.Datum) error, + id oid.Oid, +) (populated bool, err error) { + comments, err := getComments(ctx, p, descpb.ID(id)) + if err != nil { + return false, err + } + for _, comment := range comments { + populated = true + objID := comment[0] + objSubID := comment[1] + description := comment[2] + commentType := catalogkeys.CommentType(tree.MustBeDInt(comment[3])) + + classOid := oidZero + + switch commentType { + case catalogkeys.DatabaseCommentType: + // Database comments are exported in pg_shdescription. + continue + case catalogkeys.SchemaCommentType: + // TODO: The type conversion to oid.Oid is safe since we use desc IDs + // for this, but it's not ideal. The backing column for objId should be + // changed to use the OID type. + objID = tree.NewDOid(oid.Oid(tree.MustBeDInt(objID))) + classOid = tree.NewDOid(catconstants.PgCatalogNamespaceTableID) + case catalogkeys.ColumnCommentType, catalogkeys.TableCommentType: + // TODO: The type conversion to oid.Oid is safe since we use desc IDs + // for this, but it's not ideal. The backing column for objId should be + // changed to use the OID type. + objID = tree.NewDOid(oid.Oid(tree.MustBeDInt(objID))) + classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) + case catalogkeys.ConstraintCommentType: + tableDesc, err := p.Descriptors().ByIDWithLeased(p.txn).WithoutNonPublic().Get().Table(ctx, descpb.ID(tree.MustBeDInt(objID))) + if err != nil { + return false, err + } + schema, err := p.Descriptors().ByIDWithLeased(p.txn).WithoutNonPublic().Get().Schema(ctx, tableDesc.GetParentSchemaID()) + if err != nil { + return false, err + } + c, err := catalog.MustFindConstraintByID(tableDesc, descpb.ConstraintID(tree.MustBeDInt(objSubID))) + if err != nil { + return false, err + } + objID = getOIDFromConstraint(c, dbContext.GetID(), schema.GetID(), tableDesc) + objSubID = tree.DZero + classOid = tree.NewDOid(catconstants.PgCatalogConstraintTableID) + case catalogkeys.IndexCommentType: + objID = makeOidHasher().IndexOid( + descpb.ID(tree.MustBeDInt(objID)), + descpb.IndexID(tree.MustBeDInt(objSubID))) + objSubID = tree.DZero + classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) + } + if err := addRow( + objID, + classOid, + objSubID, + description); err != nil { + return false, err + } + } + + if id != 0 && populated { + return populated, nil + } + + // Populate rows based on builtins as well. + fmtOverLoad := func(builtin tree.Overload) error { + return addRow( + tree.NewDOid(builtin.Oid), + tree.NewDOid(catconstants.PgCatalogProcTableID), + tree.DZero, + tree.NewDString(builtin.Info), + ) + } + // For direct lookups match with builtins, if this fails + // we will do a full scan, since the OID might be an index + // or constraint. + if id != 0 { + builtin, matched := tree.OidToQualifiedBuiltinOverload[id] + if !matched { + return matched, err + } + return true, fmtOverLoad(*builtin.Overload) + } + for _, name := range builtins.AllBuiltinNames() { + _, overloads := builtinsregistry.GetBuiltinProperties(name) + for _, builtin := range overloads { + if err := fmtOverLoad(builtin); err != nil { + return false, err + } + } + } + return true, nil } var pgCatalogDescriptionTable = virtualSchemaTable{ @@ -1604,85 +1734,17 @@ https://www.postgresql.org/docs/9.5/catalog-pg-description.html`, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { - - // This is less efficient than it has to be - if we see performance problems - // here, we can push the filter into the query that getComments runs, - // instead of filtering client-side below. - comments, err := getComments(ctx, p) - if err != nil { - return err - } - for _, comment := range comments { - objID := comment[0] - objSubID := comment[1] - description := comment[2] - commentType := catalogkeys.CommentType(tree.MustBeDInt(comment[3])) - - classOid := oidZero - - switch commentType { - case catalogkeys.DatabaseCommentType: - // Database comments are exported in pg_shdescription. - continue - case catalogkeys.SchemaCommentType: - // TODO: The type conversion to oid.Oid is safe since we use desc IDs - // for this, but it's not ideal. The backing column for objId should be - // changed to use the OID type. - objID = tree.NewDOid(oid.Oid(tree.MustBeDInt(objID))) - classOid = tree.NewDOid(catconstants.PgCatalogNamespaceTableID) - case catalogkeys.ColumnCommentType, catalogkeys.TableCommentType: - // TODO: The type conversion to oid.Oid is safe since we use desc IDs - // for this, but it's not ideal. The backing column for objId should be - // changed to use the OID type. - objID = tree.NewDOid(oid.Oid(tree.MustBeDInt(objID))) - classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) - case catalogkeys.ConstraintCommentType: - tableDesc, err := p.Descriptors().ByIDWithLeased(p.txn).WithoutNonPublic().Get().Table(ctx, descpb.ID(tree.MustBeDInt(objID))) - if err != nil { - return err - } - schema, err := p.Descriptors().ByIDWithLeased(p.txn).WithoutNonPublic().Get().Schema(ctx, tableDesc.GetParentSchemaID()) - if err != nil { - return err - } - c, err := catalog.MustFindConstraintByID(tableDesc, descpb.ConstraintID(tree.MustBeDInt(objSubID))) - if err != nil { - return err - } - objID = getOIDFromConstraint(c, dbContext.GetID(), schema.GetID(), tableDesc) - objSubID = tree.DZero - classOid = tree.NewDOid(catconstants.PgCatalogConstraintTableID) - case catalogkeys.IndexCommentType: - objID = makeOidHasher().IndexOid( - descpb.ID(tree.MustBeDInt(objID)), - descpb.IndexID(tree.MustBeDInt(objSubID))) - objSubID = tree.DZero - classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) - } - if err := addRow( - objID, - classOid, - objSubID, - description); err != nil { - return err - } - } - - // Also add all built-in comments. - for _, name := range builtins.AllBuiltinNames() { - _, overloads := builtinsregistry.GetBuiltinProperties(name) - for _, builtin := range overloads { - if err := addRow( - tree.NewDOid(builtin.Oid), - tree.NewDOid(catconstants.PgCatalogProcTableID), - tree.DZero, - tree.NewDString(builtin.Info), - ); err != nil { - return err - } - } - } - return nil + _, err := populatePgCatalogFromComments(ctx, p, dbContext, addRow, 0) + return err + }, + indexes: []virtualIndex{ + { + incomplete: true, + populate: func(ctx context.Context, unwrappedConstraint tree.Datum, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + oid := tree.MustBeDOid(unwrappedConstraint) + return populatePgCatalogFromComments(ctx, p, dbContext, addRow, oid.Oid) + }, + }, }, } @@ -1739,7 +1801,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html`, schema: vtable.PGCatalogSharedDescription, populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { // See comment above - could make this more efficient if necessary. - comments, err := getComments(ctx, p) + comments, err := getComments(ctx, p, descpb.InvalidID /* all comments */) if err != nil { return err } diff --git a/pkg/sql/pg_metadata_test.go b/pkg/sql/pg_metadata_test.go index a2fa09bbaa10..7c7ddbeb6a73 100644 --- a/pkg/sql/pg_metadata_test.go +++ b/pkg/sql/pg_metadata_test.go @@ -222,10 +222,11 @@ var none = struct{}{} var mappedPopulateFunctions = map[string]string{ // Currently pg_type cannot be found automatically by this code because it is // not the populate function. Same for pg_proc. - "addPGTypeRow": "PGCatalogType", - "addPgProcUDFRow": "PGCatalogProc", - "addPgProcBuiltinRow": "PgCatalogProc", - "addRowForTimezoneNames": "PgCatalogTimezoneNames", + "addPGTypeRow": "PGCatalogType", + "addPgProcUDFRow": "PGCatalogProc", + "addPgProcBuiltinRow": "PgCatalogProc", + "addRowForTimezoneNames": "PgCatalogTimezoneNames", + "populatePgCatalogFromComments": "PGCatalogDescription", } // schemaCodeFixer have specific configurations to fix the files with virtual diff --git a/pkg/sql/vtable/pg_catalog.go b/pkg/sql/vtable/pg_catalog.go index d3135747fa25..db1ff042a937 100644 --- a/pkg/sql/vtable/pg_catalog.go +++ b/pkg/sql/vtable/pg_catalog.go @@ -313,7 +313,8 @@ CREATE TABLE pg_catalog.pg_description ( objoid OID, classoid OID, objsubid INT4, - description STRING + description STRING, + INDEX(objoid) )` // PGCatalogSharedDescription describes the schema of the