diff --git a/pkg/sql/catalog/catalogkeys/keys.go b/pkg/sql/catalog/catalogkeys/keys.go index e9fb8329d556..c63d31a78e32 100644 --- a/pkg/sql/catalog/catalogkeys/keys.go +++ b/pkg/sql/catalog/catalogkeys/keys.go @@ -45,6 +45,8 @@ type CommentType int //go:generate stringer --type CommentType // Note: please add the new comment types to AllCommentTypes as well. +// Note: do not change the numeric values of this enum -- they correspond +// to stored values in system.comments. const ( // DatabaseCommentType comment on a database. DatabaseCommentType CommentType = 0 diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 10ee77bc25c8..1f62ea38a23c 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -737,6 +737,11 @@ func (tc *Collection) GetAllDatabases(ctx context.Context, txn *kv.Txn) (nstree. if err != nil { return nstree.Catalog{}, err } + + // FIXME: here probably I want to call LoadComments() with just + // DatabaseCommentType. But I only want to do so when the caller is + // interested in comments. Not all are. + ret, err := tc.aggregateAllLayers(ctx, txn, stored) if err != nil { return nstree.Catalog{}, err @@ -843,8 +848,14 @@ func (tc *Collection) GetAllInDatabase( if err != nil { return nstree.Catalog{}, err } + + // Also ensure the db desc itself is included, which ensures we can + // fetch its comment if any below. + // FIXME: this does not seem to work. Why? + ret.UpsertDescriptor(db) + var inDatabaseIDs catalog.DescriptorIDSet - _ = ret.ForEachDescriptor(func(desc catalog.Descriptor) error { + if err := ret.ForEachDescriptor(func(desc catalog.Descriptor) error { if desc.DescriptorType() == catalog.Schema { if dbID := desc.GetParentID(); dbID != descpb.InvalidID && dbID != db.GetID() { return nil @@ -855,11 +866,32 @@ func (tc *Collection) GetAllInDatabase( } } inDatabaseIDs.Add(desc.GetID()) + + // Also include all the comments for this object. + // FIXME: This does not seem to be the right place to call this -- + // it should load comments into `stored` _before_ the call + // to aggregateLayers(), so that shadowed comments don't get overwritten. + comments, err := tc.cr.LoadComments(ctx, txn, catalogkeys.AllCommentTypes, desc.GetID()) + if err != nil { + return err + } + ret.AddAll(comments) return nil - }) + }); err != nil { + return nstree.Catalog{}, err + } + return ret.FilterByIDs(inDatabaseIDs.Ordered()), nil } +// LoadComments extends the given catalog with all comments of the given type +// associated with descriptors already in the catalog. +func (tc *Collection) LoadComments( + ctx context.Context, txn *kv.Txn, commentTypes []catalogkeys.CommentType, descID descpb.ID, +) (nstree.Catalog, error) { + return tc.cr.LoadComments(ctx, txn, commentTypes, descID) +} + // GetAllTablesInDatabase is like GetAllInDatabase but filtered to tables. // Includes virtual objects. Does not include dropped objects. func (tc *Collection) GetAllTablesInDatabase( diff --git a/pkg/sql/catalog/internal/catkv/catalog_reader.go b/pkg/sql/catalog/internal/catkv/catalog_reader.go index 461129307890..ff40b14e8c46 100644 --- a/pkg/sql/catalog/internal/catkv/catalog_reader.go +++ b/pkg/sql/catalog/internal/catkv/catalog_reader.go @@ -81,6 +81,10 @@ type CatalogReader interface { ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, ) (nstree.Catalog, error) + // LoadComments returns a catalog with at least the comment of the + // given types for the given object. It may include more comments. + LoadComments(ctx context.Context, txn *kv.Txn, commentTypes []catalogkeys.CommentType, objID descpb.ID) (nstree.Catalog, error) + // GetByIDs reads the system.descriptor, system.comments and system.zone // entries for the desired IDs, but looks in the system database cache // first if there is one. @@ -159,6 +163,23 @@ func (cr catalogReader) ScanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalo return mc.Catalog, nil } +// LoadComments is part of the CatalogReader interface. +func (cr catalogReader) LoadComments( + ctx context.Context, txn *kv.Txn, commentTypes []catalogkeys.CommentType, objID descpb.ID, +) (nstree.Catalog, error) { + var mc nstree.MutableCatalog + cq := catalogQuery{codec: cr.codec} + err := cq.query(ctx, txn, &mc, func(codec keys.SQLCodec, b *kv.Batch) { + for _, ct := range commentTypes { + scan(ctx, b, catalogkeys.MakeObjectCommentsMetadataPrefix(codec, ct, objID)) + } + }) + if err != nil { + return nstree.Catalog{}, err + } + return mc.Catalog, nil +} + func (cr catalogReader) scanNamespace( ctx context.Context, txn *kv.Txn, prefix roachpb.Key, ) (nstree.Catalog, error) { diff --git a/pkg/sql/catalog/internal/catkv/catalog_reader_cached.go b/pkg/sql/catalog/internal/catkv/catalog_reader_cached.go index 0f4e0e2b0d4b..73d08166fef4 100644 --- a/pkg/sql/catalog/internal/catkv/catalog_reader_cached.go +++ b/pkg/sql/catalog/internal/catkv/catalog_reader_cached.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -61,6 +62,7 @@ type byIDStateValue struct { hasScanNamespaceForDatabaseEntries bool hasScanNamespaceForDatabaseSchemas bool hasGetDescriptorEntries bool + hasLoadComments bool } type byNameStateValue struct { @@ -177,6 +179,7 @@ func (c *cachedCatalogReader) ScanAll(ctx context.Context, txn *kv.Txn) (nstree. s.hasScanNamespaceForDatabaseEntries = true s.hasScanNamespaceForDatabaseSchemas = true s.hasGetDescriptorEntries = true + s.hasLoadComments = true c.byIDState[id] = s } for ni, s := range c.byNameState { @@ -288,6 +291,34 @@ func (c *cachedCatalogReader) ScanNamespaceForSchemaObjects( return read, nil } +// LoadComments is part of the CatalogReader interface. +func (c *cachedCatalogReader) LoadComments( + ctx context.Context, txn *kv.Txn, commentTypes []catalogkeys.CommentType, objID descpb.ID, +) (nstree.Catalog, error) { + if !c.byIDState[objID].hasLoadComments { + // Cache miss: need to retrieve the comments from underneath. + // In this case we retrieve all comment types, not just the one requested. + read, err := c.cr.LoadComments(ctx, txn, catalogkeys.AllCommentTypes, objID) + if err != nil { + return nstree.Catalog{}, err + } + if err := c.ensure(ctx, read); err != nil { + return nstree.Catalog{}, err + } + s := c.byIDState[objID] + s.hasLoadComments = true + c.byIDState[objID] = s + } + + var mc nstree.MutableCatalog + c.cache.ForEachCommentOnDescriptor(objID, func(key catalogkeys.CommentKey, cmt string) error { + mc.UpsertComment(key, cmt) + return nil + }) + + return mc.Catalog, nil +} + // GetByIDs is part of the CatalogReader interface. func (c *cachedCatalogReader) GetByIDs( ctx context.Context, diff --git a/pkg/sql/comprules/BUILD.bazel b/pkg/sql/comprules/BUILD.bazel index ffe11d570647..bbba51214078 100644 --- a/pkg/sql/comprules/BUILD.bazel +++ b/pkg/sql/comprules/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/sql/compengine", "//pkg/sql/lexbase", "//pkg/sql/scanner", + "//pkg/sql/sem/catconstants", ], ) diff --git a/pkg/sql/comprules/rules.go b/pkg/sql/comprules/rules.go index b07b8feb805f..181cd51147a4 100644 --- a/pkg/sql/comprules/rules.go +++ b/pkg/sql/comprules/rules.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/compengine" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/scanner" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" ) // GetCompMethods exposes the completion heuristics defined in this @@ -257,19 +258,20 @@ func completeObjectInCurrentDatabase( c.Trace("completing for %q (%d,%d), schema: %s", prefix, start, end, schema) const queryT = ` -WITH n AS (SELECT oid FROM pg_catalog.pg_namespace WHERE nspname %s), - t AS (SELECT oid, relname FROM pg_catalog.pg_class WHERE reltype != 0 AND relnamespace IN (TABLE n)) -SELECT relname AS completion, - 'relation' AS category, - substr(COALESCE(cc.comment, ''), e'[^\n]{0,80}') as description, - $2:::INT AS start, - $3:::INT AS end - FROM t -LEFT OUTER JOIN "".crdb_internal.kv_catalog_comments cc - ON t.oid = cc.object_id AND cc.type = 'TableCommentType' - WHERE left(relname, length($1:::STRING)) = $1::STRING + SELECT c.relname AS completion, + 'relation' AS category, + substr(d.description, ''), e'[^\n]{0,80}') as description, + $2:::INT AS start, + $3:::INT AS end + FROM pg_catalog.pg_class c + JOIN pg_catalog.pg_namespace n + ON c.relnamespace = n.oid AND n.nspname %s +LEFT OUTER JOIN crdb_internal.kv_catalog_comments d + ON t.oid = d.objoid AND d.classoid = %d + WHERE c.reltype != 0 + AND left(relname, length($1:::STRING)) = $1::STRING ` - query := fmt.Sprintf(queryT, schema) + query := fmt.Sprintf(queryT, schema, catconstants.PgCatalogClassTableID) iter, err := c.Query(ctx, query, prefix, start, end) return iter, err } @@ -298,17 +300,18 @@ func completeSchemaInCurrentDatabase( } c.Trace("completing for %q (%d,%d)", prefix, start, end) - const query = ` -SELECT nspname AS completion, - 'schema' AS category, - substr(COALESCE(cc.comment, ''), e'[^\n]{0,80}') as description, - $2:::INT AS start, - $3:::INT AS end - FROM pg_catalog.pg_namespace t -LEFT OUTER JOIN "".crdb_internal.kv_catalog_comments cc - ON t.oid = cc.object_id AND cc.type = 'SchemaCommentType' + const queryT = ` + SELECT n.nspname AS completion, + 'schema' AS category, + substr(COALESCE(d.description, ''), e'[^\n]{0,80}') as description, + $2:::INT AS start, + $3:::INT AS end + FROM pg_catalog.pg_namespace n +LEFT OUTER JOIN crdb_internal.kv_catalog_comments d + ON n.oid = d.objoid AND d.classoid = %d WHERE left(nspname, length($1:::STRING)) = $1::STRING ` + query := fmt.Sprintf(queryT, catconstants.PgCatalogNamespaceTableID) iter, err := c.Query(ctx, query, prefix, start, end) return iter, err } @@ -418,7 +421,7 @@ func completeObjectInOtherDatabase( } c.Trace("completing for %q (%d,%d), schema: %q, db: %q", prefix, start, end, schema, dbname) - const query = ` + const queryT = ` WITH t AS ( SELECT name, table_id FROM "".crdb_internal.tables @@ -433,8 +436,9 @@ SELECT name AS completion, $3:::INT AS end FROM t LEFT OUTER JOIN "".crdb_internal.kv_catalog_comments cc - ON t.table_id = cc.object_id AND cc.type = 'TableCommentType' + ON t.table_id = cc.objoid AND cc.classoid = %d ` + query := fmt.Sprintf(queryT, catconstants.PgCatalogClassTableID) iter, err := c.Query(ctx, query, prefix, start, end, dbname, schema) return iter, err } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 4df2bb26762c..2fe5573cf334 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -94,6 +94,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing/collector" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" + "github.com/lib/pq/oid" ) // CrdbInternalName is the name of the crdb_internal schema. @@ -116,6 +117,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalBackwardDependenciesTableID: crdbInternalBackwardDependenciesTable, catconstants.CrdbInternalBuildInfoTableID: crdbInternalBuildInfoTable, catconstants.CrdbInternalBuiltinFunctionsTableID: crdbInternalBuiltinFunctionsTable, + catconstants.CrdbInternalBuiltinFunctionCommentsTableID: crdbInternalBuiltinFunctionCommentsTable, catconstants.CrdbInternalCatalogCommentsTableID: crdbInternalCatalogCommentsTable, catconstants.CrdbInternalCatalogDescriptorTableID: crdbInternalCatalogDescriptorTable, catconstants.CrdbInternalCatalogNamespaceTableID: crdbInternalCatalogNamespaceTable, @@ -2737,7 +2739,8 @@ CREATE TABLE crdb_internal.builtin_functions ( signature STRING NOT NULL, category STRING NOT NULL, details STRING NOT NULL, - schema STRING NOT NULL + schema STRING NOT NULL, + oid OID NOT NULL )`, populate: func(ctx context.Context, _ *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { for _, name := range builtins.AllBuiltinNames() { @@ -2755,6 +2758,35 @@ CREATE TABLE crdb_internal.builtin_functions ( tree.NewDString(props.Category), tree.NewDString(f.Info), tree.NewDString(schema), + tree.NewDOid(f.Oid), + ); err != nil { + return err + } + } + } + return nil + }, +} + +// crdbInternalBuiltinFunctionCommentsTable exposes the built-in function +// comments, for use in pg_catalog. +var crdbInternalBuiltinFunctionCommentsTable = virtualSchemaTable{ + comment: "built-in functions (RAM/static)", + schema: ` +CREATE TABLE crdb_internal.builtin_function_comments ( + oid OID NOT NULL, + description STRING NOT NULL +)`, + populate: func(ctx context.Context, _ *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + for _, name := range builtins.AllBuiltinNames() { + _, overloads := builtinsregistry.GetBuiltinProperties(name) + for _, f := range overloads { + if f.Info == "" { + continue + } + if err := addRow( + tree.NewDOid(f.Oid), + tree.NewDString(f.Info), ); err != nil { return err } @@ -5134,50 +5166,99 @@ var crdbInternalCatalogCommentsTable = virtualSchemaTable{ comment: `like system.comments but overlaid with in-txn in-memory changes and including virtual objects`, schema: ` CREATE TABLE crdb_internal.kv_catalog_comments ( - type STRING NOT NULL, - object_id INT NOT NULL, - sub_id INT NOT NULL, - comment STRING NOT NULL + classoid OID NOT NULL, + objoid OID NOT NULL, + objsubid INT4 NOT NULL, + description STRING NOT NULL, + INDEX(classoid) WHERE classoid = ` + strconv.Itoa(catconstants.PgCatalogDatabaseTableID) + `:::oid )`, + indexes: []virtualIndex{ + { + populate: func(ctx context.Context, _ tree.Datum, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + // FIXME: when dbContext is not nil, we want a catalog with + // _only_ that db desc and is comment, if any. How to build + // it? + cat, err := p.Descriptors().GetAllDatabases(ctx, p.txn) + if err != nil { + return true, err + } + return true, populateCommentsTable(ctx, p, cat, addRow) + }, + }, + }, populate: func( ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error, ) error { - all, err := p.Descriptors().GetAll(ctx, p.Txn()) + var all nstree.Catalog + var err error + if dbContext != nil { + all, err = p.Descriptors().GetAllInDatabase(ctx, p.txn, dbContext) + } else { + all, err = p.Descriptors().GetAll(ctx, p.txn) + } if err != nil { return err } - // Delegate privilege check to system table. - { - sysTable := all.LookupDescriptor(systemschema.CommentsTable.GetID()) - if ok, err := p.HasPrivilege(ctx, sysTable, privilege.SELECT, p.User()); err != nil { - return err - } else if !ok { - return nil - } + return populateCommentsTable(ctx, p, all, addRow) + }, +} + +func populateCommentsTable( + ctx context.Context, p *planner, cat nstree.Catalog, addRow func(...tree.Datum) error, +) error { + // Delegate privilege check to system table. + { + sysTable, err := p.LookupTableByID(ctx, systemschema.CommentsTable.GetID()) + if err != nil { + return err } - // 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 - // retrieve those comments. - // TODO(knz): extend this with vtable column comments. - for _, ct := range catalogkeys.AllCommentTypes { - dct := tree.NewDString(ct.String()) - if err := all.ForEachComment(func(key catalogkeys.CommentKey, cmt string) error { - if ct != key.CommentType { - return nil - } - return addRow( - dct, - tree.NewDInt(tree.DInt(int64(key.ObjectID))), - tree.NewDInt(tree.DInt(int64(key.SubID))), - tree.NewDString(cmt), - ) - }); err != nil { + if ok, err := p.HasPrivilege(ctx, sysTable, privilege.SELECT, p.User()); err != nil { + return err + } else if !ok { + return nil + } + } + if err := cat.ForEachComment(func(key catalogkeys.CommentKey, cmt string) error { + var classOid, objOid *tree.DOid + objSubID := tree.DZero + switch key.CommentType { + case catalogkeys.DatabaseCommentType: + classOid = tree.NewDOid(catconstants.PgCatalogDatabaseTableID) + objOid = tree.NewDOid(oid.Oid(key.ObjectID)) + + case catalogkeys.SchemaCommentType: + classOid = tree.NewDOid(catconstants.PgCatalogNamespaceTableID) + objOid = tree.NewDOid(oid.Oid(key.ObjectID)) + + case catalogkeys.ColumnCommentType, catalogkeys.TableCommentType: + classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) + objOid = tree.NewDOid(oid.Oid(key.ObjectID)) + objSubID = tree.NewDInt(tree.DInt(key.SubID)) + + case catalogkeys.IndexCommentType: + classOid = tree.NewDOid(catconstants.PgCatalogIndexTableID) + objOid = makeOidHasher().IndexOid(descpb.ID(key.ObjectID), descpb.IndexID(key.SubID)) + + case catalogkeys.ConstraintCommentType: + tableDesc := cat.LookupDescriptor(catid.DescID(key.ObjectID)).(catalog.TableDescriptor) + schema := cat.LookupDescriptor(tableDesc.GetParentSchemaID()) + db := cat.LookupDescriptor(schema.GetParentID()) + c, err := tableDesc.FindConstraintWithID(descpb.ConstraintID(key.SubID)) + if err != nil { return err } + classOid = tree.NewDOid(catconstants.PgCatalogConstraintTableID) + objOid = getOIDFromConstraint(c, db.GetID(), schema.GetID(), tableDesc) } - return nil - }, + + return addRow( + classOid, objOid, objSubID, + tree.NewDString(cmt), + ) + }); err != nil { + return err + } + return nil } type marshaledJobMetadata struct { diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index b6cdaaceaf3b..dd3ca5092b5c 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" @@ -405,29 +406,26 @@ var informationSchemaColumnsTable = virtualSchemaTable{ 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 { + colCommentType := []catalogkeys.CommentType{catalogkeys.ColumnCommentType} // Get the collations for all comments of current database. - comments, err := getComments(ctx, p) - if err != nil { - return err - } - // Push all comments of columns into map. - commentMap := make(map[tree.DInt]map[tree.DInt]string) - for _, comment := range comments { - objID := tree.MustBeDInt(comment[0]) - objSubID := tree.MustBeDInt(comment[1]) - description := comment[2].String() - commentType := tree.MustBeDInt(comment[3]) - if commentType == 2 { - if commentMap[objID] == nil { - commentMap[objID] = make(map[tree.DInt]string) - } - commentMap[objID][objSubID] = description - } - } - return forEachTableDesc(ctx, p, dbContext, virtualMany, func( db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, table catalog.TableDescriptor, ) error { + // Push all comments of columns into map. + commentMap := make(map[uint32]string) + columnComments, err := p.Descriptors().LoadComments(ctx, p.Txn(), colCommentType, table.GetID()) + if err != nil { + return err + } + columnComments.ForEachCommentOnDescriptor( + table.GetID(), func(key catalogkeys.CommentKey, cmt string) error { + if key.CommentType != catalogkeys.ColumnCommentType { + return nil + } + commentMap[key.SubID] = cmt + return nil + }) + dbNameStr := tree.NewDString(db.GetName()) scNameStr := tree.NewDString(sc.GetName()) for _, column := range table.AccessibleColumns() { @@ -473,10 +471,10 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`, } } - // Match the comment belonging to current column from map,using table id and column id - tableID := tree.DInt(table.GetID()) - columnID := tree.DInt(column.GetID()) - description := commentMap[tableID][columnID] + description := tree.DNull + if cmt, ok := commentMap[uint32(column.GetPGAttributeNum())]; ok { + description = tree.NewDString(cmt) + } // udt_schema is set to pg_catalog for builtin types. If, however, the // type is a user defined type, then we should fill this value based on @@ -508,7 +506,7 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`, scNameStr, // table_schema tree.NewDString(table.GetName()), // table_name tree.NewDString(column.GetName()), // column_name - tree.NewDString(description), // column_comment + description, // column_comment tree.NewDInt(tree.DInt(column.GetPGAttributeNum())), // ordinal_position colDefault, // column_default yesOrNoDatum(column.IsNullable()), // is_nullable diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 103d0f031a6e..bb7692aaf901 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -17,6 +17,7 @@ import ( "fmt" "hash" "hash/fnv" + "strconv" "strings" "time" "unicode" @@ -24,11 +25,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catformat" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" @@ -150,7 +151,7 @@ var pgCatalog = virtualSchema{ catconstants.PgCatalogDbRoleSettingTableID: pgCatalogDbRoleSettingTable, catconstants.PgCatalogDefaultACLTableID: pgCatalogDefaultACLTable, catconstants.PgCatalogDependTableID: pgCatalogDependTable, - catconstants.PgCatalogDescriptionTableID: pgCatalogDescriptionTable, + catconstants.PgCatalogDescriptionTableID: pgCatalogDescriptionView, catconstants.PgCatalogEnumTableID: pgCatalogEnumTable, catconstants.PgCatalogEventTriggerTableID: pgCatalogEventTriggerTable, catconstants.PgCatalogExtensionTableID: pgCatalogExtensionTable, @@ -195,7 +196,7 @@ var pgCatalog = virtualSchema{ catconstants.PgCatalogSequencesTableID: pgCatalogSequencesTable, catconstants.PgCatalogSettingsTableID: pgCatalogSettingsTable, catconstants.PgCatalogShadowTableID: pgCatalogShadowTable, - catconstants.PgCatalogSharedDescriptionTableID: pgCatalogSharedDescriptionTable, + catconstants.PgCatalogSharedDescriptionTableID: pgCatalogSharedDescriptionView, catconstants.PgCatalogSharedSecurityLabelTableID: pgCatalogSharedSecurityLabelTable, catconstants.PgCatalogShdependTableID: pgCatalogShdependTable, catconstants.PgCatalogShmemAllocationsTableID: pgCatalogShmemAllocationsTable, @@ -1531,121 +1532,32 @@ 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) { - 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;`) -} - -var pgCatalogDescriptionTable = virtualSchemaTable{ - comment: `object comments -https://www.postgresql.org/docs/9.5/catalog-pg-description.html`, - schema: vtable.PGCatalogDescription, - populate: func( - ctx context.Context, - 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 := tableDesc.FindConstraintWithID(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 +var pgCatalogDescriptionView = virtualSchemaView{ + // TODO(knz): add an explanatory comment here when this issue is fixed: + // https://github.com/cockroachdb/cockroach/issues/95427 + // + // Note, the query uses `crdb_internal.kv_catalog_comments` without + // a database prefix. This is intentional: this ensures + // kv_catalog_comments only conains rows for the current database, + // which is what pg_description expects. + schema: ` +CREATE VIEW pg_catalog.pg_description AS SELECT + objoid, classoid, objsubid, description +FROM crdb_internal.kv_catalog_comments +WHERE classoid != ` + strconv.Itoa(catconstants.PgCatalogDatabaseTableID) + ` +UNION ALL +SELECT + oid AS objoid, + ` + strconv.Itoa(catconstants.PgCatalogProcTableID) + `:::oid AS classoid, + 0:::INT4 AS objsubid, + description AS description +FROM crdb_internal.builtin_function_comments +`, + resultColumns: colinfo.ResultColumns{ + {Name: "objoid", Typ: types.Oid}, + {Name: "classoid", Typ: types.Oid}, + {Name: "objsubid", Typ: types.Int4}, + {Name: "description", Typ: types.String}, }, } @@ -1696,32 +1608,19 @@ func getOIDFromConstraint( return oid } -var pgCatalogSharedDescriptionTable = virtualSchemaTable{ - comment: `shared object comments -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) - if err != nil { - return err - } - for _, comment := range comments { - commentType := catalogkeys.CommentType(tree.MustBeDInt(comment[3])) - if commentType != catalogkeys.DatabaseCommentType { - // Only database comments are exported in this table. - continue - } - classOid := tree.NewDOid(catconstants.PgCatalogDatabaseTableID) - objID := descpb.ID(tree.MustBeDInt(comment[0])) - if err := addRow( - tableOid(objID), - classOid, - comment[2]); err != nil { - return err - } - } - return nil +// Database comments. +// https://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html, +var pgCatalogSharedDescriptionView = virtualSchemaView{ + // TODO(knz): add an explanatory comment here when this issue is fixed: + // https://github.com/cockroachdb/cockroach/issues/95427 + schema: ` +CREATE VIEW pg_catalog.pg_shdescription AS SELECT objoid, classoid, description +FROM "".crdb_internal.kv_catalog_comments +WHERE classoid = ` + strconv.Itoa(catconstants.PgCatalogDatabaseTableID) + `:::oid`, + resultColumns: colinfo.ResultColumns{ + {Name: "objoid", Typ: types.Oid}, + {Name: "classoid", Typ: types.Oid}, + {Name: "description", Typ: types.String}, }, } diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index 97c4d9d2dd9b..e137239396d9 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -96,6 +96,7 @@ const ( CrdbInternalBackwardDependenciesTableID CrdbInternalBuildInfoTableID CrdbInternalBuiltinFunctionsTableID + CrdbInternalBuiltinFunctionCommentsTableID CrdbInternalCatalogCommentsTableID CrdbInternalCatalogDescriptorTableID CrdbInternalCatalogNamespaceTableID