diff --git a/docs/generated/sql/bnf/comment.bnf b/docs/generated/sql/bnf/comment.bnf index 31ac4179e833..a18d57fe3ce4 100644 --- a/docs/generated/sql/bnf/comment.bnf +++ b/docs/generated/sql/bnf/comment.bnf @@ -1,5 +1,6 @@ comment_stmt ::= 'COMMENT' 'ON' 'DATABASE' database_name 'IS' comment_text + | 'COMMENT' 'ON' 'SCHEMA' schema_name 'IS' comment_text | 'COMMENT' 'ON' 'TABLE' table_name 'IS' comment_text | 'COMMENT' 'ON' 'COLUMN' column_name 'IS' comment_text | 'COMMENT' 'ON' 'INDEX' table_index_name 'IS' comment_text diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index b6386511c91d..27c250564edf 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -55,6 +55,7 @@ copy_from_stmt ::= comment_stmt ::= 'COMMENT' 'ON' 'DATABASE' database_name 'IS' comment_text + | 'COMMENT' 'ON' 'SCHEMA' schema_name 'IS' comment_text | 'COMMENT' 'ON' 'TABLE' table_name 'IS' comment_text | 'COMMENT' 'ON' 'COLUMN' column_path 'IS' comment_text | 'COMMENT' 'ON' 'INDEX' table_index_name 'IS' comment_text @@ -272,6 +273,9 @@ comment_text ::= 'SCONST' | 'NULL' +schema_name ::= + name + column_path ::= name | prefixed_column_path @@ -1806,9 +1810,6 @@ alter_zone_partition_stmt ::= | 'ALTER' 'PARTITION' partition_name 'OF' 'INDEX' table_index_name set_zone_config | 'ALTER' 'PARTITION' partition_name 'OF' 'INDEX' table_name '@' '*' set_zone_config -schema_name ::= - name - opt_add_val_placement ::= 'BEFORE' 'SCONST' | 'AFTER' 'SCONST' diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index 2c6c86c987a4..6fd2ad502cae 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -850,7 +850,8 @@ func readPostgresStmt( case *tree.Insert, *tree.CopyFrom, *tree.Delete, copyData: // handled during the data ingestion pass. case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, - *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.SetVar, *tree.Analyze: + *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.SetVar, *tree.Analyze, + *tree.CommentOnSchema: // These are the statements that can be parsed by CRDB but are not // supported, or are not required to be processed, during an IMPORT. // - ignore txns. @@ -1344,7 +1345,8 @@ func (m *pgDumpReader) readFile( return wrapErrorWithUnsupportedHint(err) } case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, - *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.AlterSequence: + *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.AlterSequence, + *tree.CommentOnSchema: // handled during schema extraction. case *tree.SetVar, *tree.BeginTransaction, *tree.CommitTransaction, *tree.Analyze: // handled during schema extraction. diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 2c4ce11a0006..bf61c547d8d1 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -416,6 +416,7 @@ const ( TableCommentType = 1 ColumnCommentType = 2 IndexCommentType = 3 + SchemaCommentType = 4 ) const ( diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index c5eb99b5a6d8..c723e0726e3d 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "comment_on_column.go", "comment_on_database.go", "comment_on_index.go", + "comment_on_schema.go", "comment_on_table.go", "conn_executor.go", "conn_executor_exec.go", diff --git a/pkg/sql/comment_on_schema.go b/pkg/sql/comment_on_schema.go new file mode 100644 index 000000000000..9e2095dd5383 --- /dev/null +++ b/pkg/sql/comment_on_schema.go @@ -0,0 +1,95 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" +) + +type commentOnSchemaNode struct { + n *tree.CommentOnSchema + schemaDesc catalog.SchemaDescriptor +} + +// CommentOnSchema add comment on a schema. +// Privileges: CREATE on scheme. +// notes: postgres requires CREATE on the scheme. +func (p *planner) CommentOnSchema(ctx context.Context, n *tree.CommentOnSchema) (planNode, error) { + if err := checkSchemaChangeEnabled( + ctx, + p.ExecCfg(), + "COMMENT ON SCHEMA", + ); err != nil { + return nil, err + } + + // Users can't create a schema without being connected to a DB. + dbName := p.CurrentDatabase() + if dbName == "" { + return nil, pgerror.New(pgcode.UndefinedDatabase, + "cannot comment schema without being connected to a database") + } + + db, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + dbName, tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return nil, err + } + + schemaDesc, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.txn, + db.GetSchemaID(string(n.Name)), tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return nil, err + } + + if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil { + return nil, err + } + + return &commentOnSchemaNode{n: n, schemaDesc: schemaDesc}, nil +} + +func (n *commentOnSchemaNode) startExec(params runParams) error { + if n.n.Comment != nil { + _, err := params.p.extendedEvalCtx.ExecCfg.InternalExecutor.ExecEx( + params.ctx, + "set-schema-comment", + params.p.Txn(), + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "UPSERT INTO system.comments VALUES ($1, $2, 0, $3)", + keys.SchemaCommentType, + n.schemaDesc.GetID(), + *n.n.Comment) + if err != nil { + return err + } + } else { + err := params.p.removeSchemaComment(params.ctx, n.schemaDesc.GetID()) + if err != nil { + return err + } + } + + return nil +} + +func (n *commentOnSchemaNode) Next(runParams) (bool, error) { return false, nil } +func (n *commentOnSchemaNode) Values() tree.Datums { return tree.Datums{} } +func (n *commentOnSchemaNode) Close(context.Context) {} diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 1aa20005fb1c..515824834167 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -26,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" @@ -212,6 +214,9 @@ func (p *planner) dropSchemaImpl( } // Mark the descriptor as dropped. sc.State = descpb.DescriptorState_DROP + if err := p.removeSchemaComment(ctx, sc.GetID()); err != nil { + return err + } return p.writeSchemaDesc(ctx, sc) } @@ -245,6 +250,19 @@ func (p *planner) createDropSchemaJob( return err } +func (p *planner) removeSchemaComment(ctx context.Context, schemaID descpb.ID) error { + _, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.ExecEx( + ctx, + "delete-schema-comment", + p.txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "DELETE FROM system.comments WHERE type=$1 AND object_id=$2 AND sub_id=0", + keys.SchemaCommentType, + schemaID) + + return err +} + func (n *dropSchemaNode) Next(params runParams) (bool, error) { return false, nil } func (n *dropSchemaNode) Values() tree.Datums { return tree.Datums{} } func (n *dropSchemaNode) Close(ctx context.Context) {} diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 4f106f71ac0c..cb4af9ba0fd1 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -5074,3 +5074,14 @@ SELECT sequenceowner, cache_size FROM pg_sequences WHERE sequencename = 'serial' ---- sequenceowner cache_size anyuser 10 + +statement ok +COMMENT ON SCHEMA test_schema is 'testing schema'; + +query T colnames +SELECT obj_description(objoid) + FROM pg_catalog.pg_description + WHERE description = 'testing schema' +---- +obj_description +testing schema diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index fe2f002f568f..3b39bff032ef 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -868,3 +868,35 @@ SHOW TABLES statement ok DROP DATABASE samename CASCADE; + +# Verify schema comments +subtest schema_comments + +user root + +statement ok +CREATE DATABASE comment_db + +statement ok +USE comment_db + +statement ok +CREATE SCHEMA foo + +statement ok +COMMENT ON SCHEMA foo IS 'bar' + +query T +SELECT comment FROM system.comments LIMIT 1 +---- +bar + +statement ok +DROP SCHEMA foo + +query T +SELECT comment FROM system.comments LIMIT 1 +---- + +statement ok +DROP DATABASE comment_db diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 76e7a83b2ce5..7ad640f8f5a1 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -112,6 +112,8 @@ func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, return p.CommentOnColumn(ctx, n) case *tree.CommentOnDatabase: return p.CommentOnDatabase(ctx, n) + case *tree.CommentOnSchema: + return p.CommentOnSchema(ctx, n) case *tree.CommentOnIndex: return p.CommentOnIndex(ctx, n) case *tree.CommentOnTable: @@ -238,6 +240,7 @@ func init() { &tree.AlterRoleSet{}, &tree.CommentOnColumn{}, &tree.CommentOnDatabase{}, + &tree.CommentOnSchema{}, &tree.CommentOnIndex{}, &tree.CommentOnTable{}, &tree.CreateDatabase{}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 01e4e62f2d57..7fef95d8200e 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -3342,6 +3342,10 @@ comment_stmt: { $$.val = &tree.CommentOnDatabase{Name: tree.Name($4), Comment: $6.strPtr()} } +| COMMENT ON SCHEMA schema_name IS comment_text + { + $$.val = &tree.CommentOnSchema{Name: tree.Name($4), Comment: $6.strPtr()} + } | COMMENT ON TABLE table_name IS comment_text { $$.val = &tree.CommentOnTable{Table: $4.unresolvedObjectName(), Comment: $6.strPtr()} diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index f96d9dcde478..f027393f7fcb 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -1390,6 +1390,9 @@ https://www.postgresql.org/docs/9.5/catalog-pg-description.html`, case keys.DatabaseCommentType: // Database comments are exported in pg_shdescription. continue + case keys.SchemaCommentType: + objID = tree.NewDOid(tree.MustBeDInt(objID)) + classOid = tree.NewDOid(catconstants.PgCatalogNamespaceTableID) case keys.ColumnCommentType, keys.TableCommentType: objID = tree.NewDOid(tree.MustBeDInt(objID)) classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 4309dec00fb1..13ffb0b535ee 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -55,7 +55,7 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error) case *tree.AlterIndex, *tree.AlterTable, *tree.AlterSequence, *tree.Analyze, *tree.BeginTransaction, - *tree.CommentOnColumn, *tree.CommentOnDatabase, *tree.CommentOnIndex, *tree.CommentOnTable, + *tree.CommentOnColumn, *tree.CommentOnDatabase, *tree.CommentOnIndex, *tree.CommentOnTable, *tree.CommentOnSchema, *tree.CommitTransaction, *tree.CopyFrom, *tree.CreateDatabase, *tree.CreateIndex, *tree.CreateView, *tree.CreateSequence, diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index deb1d256c7f8..a7e53812d4f3 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -1942,6 +1942,8 @@ func getCatalogOidForComments(catalogName string) (id int, ok bool) { return catconstants.PgCatalogClassTableID, true case "pg_database": return catconstants.PgCatalogDatabaseTableID, true + case "pg_namespace": + return catconstants.PgCatalogNamespaceTableID, true default: // We currently only support comments on pg_class objects // (columns, tables) in this context. diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 5f5b5525a441..5d68d94f4f71 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "comment_on_column.go", "comment_on_database.go", "comment_on_index.go", + "comment_on_schema.go", "comment_on_table.go", "constant.go", "constant_eval.go", diff --git a/pkg/sql/sem/tree/comment_on_schema.go b/pkg/sql/sem/tree/comment_on_schema.go new file mode 100644 index 000000000000..eae57975b20d --- /dev/null +++ b/pkg/sql/sem/tree/comment_on_schema.go @@ -0,0 +1,37 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree + +import "github.com/cockroachdb/cockroach/pkg/sql/lexbase" + +// CommentOnSchema represents an COMMENT ON SCHEMA statement. +type CommentOnSchema struct { + Name Name + Comment *string +} + +// Format implements the NodeFormatter interface. +func (n *CommentOnSchema) Format(ctx *FmtCtx) { + ctx.WriteString("COMMENT ON SCHEMA ") + ctx.FormatNode(&n.Name) + ctx.WriteString(" IS ") + if n.Comment != nil { + // TODO(knz): Replace all this with ctx.FormatNode + // when COMMENT supports expressions. + if ctx.flags.HasFlags(FmtHideConstants) { + ctx.WriteByte('_') + } else { + lexbase.EncodeSQLStringWithFlags(&ctx.Buffer, *n.Comment, ctx.flags.EncodeFlags()) + } + } else { + ctx.WriteString("NULL") + } +} diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 9e5cb2a1f0b3..54dc5e716d4f 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -479,6 +479,15 @@ func (*CommentOnDatabase) StatementType() StatementType { return TypeDDL } // StatementTag returns a short string identifying the type of statement. func (*CommentOnDatabase) StatementTag() string { return "COMMENT ON DATABASE" } +// StatementReturnType implements the Statement interface. +func (*CommentOnSchema) StatementReturnType() StatementReturnType { return DDL } + +// StatementType implements the Statement interface. +func (*CommentOnSchema) StatementType() StatementType { return TypeDDL } + +// StatementTag returns a short string identifying the type of statement. +func (*CommentOnSchema) StatementTag() string { return "COMMENT ON SCHEMA" } + // StatementReturnType implements the Statement interface. func (*CommentOnIndex) StatementReturnType() StatementReturnType { return DDL } @@ -1622,6 +1631,7 @@ func (n *CancelSessions) String() string { return AsString(n) } func (n *CannedOptPlan) String() string { return AsString(n) } func (n *CommentOnColumn) String() string { return AsString(n) } func (n *CommentOnDatabase) String() string { return AsString(n) } +func (n *CommentOnSchema) String() string { return AsString(n) } func (n *CommentOnIndex) String() string { return AsString(n) } func (n *CommentOnTable) String() string { return AsString(n) } func (n *CommitTransaction) String() string { return AsString(n) } diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 44b785a49674..8cff6d0b9feb 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -358,6 +358,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&commentOnDatabaseNode{}): "comment on database", reflect.TypeOf(&commentOnIndexNode{}): "comment on index", reflect.TypeOf(&commentOnTableNode{}): "comment on table", + reflect.TypeOf(&commentOnSchemaNode{}): "comment on schema", reflect.TypeOf(&controlJobsNode{}): "control jobs", reflect.TypeOf(&controlSchedulesNode{}): "control schedules", reflect.TypeOf(&createDatabaseNode{}): "create database",