Skip to content

Commit

Permalink
sql: support COMMENT ON SCHEMA
Browse files Browse the repository at this point in the history
This change adds support for SCHEMA commenting.

Release note (sql change): This change adds associating
comment to SQL schema using PostgreSQL's COMMENT ON SCHEMA syntax.
  • Loading branch information
ekalinin committed Aug 17, 2021
1 parent ee3efd6 commit edc8951
Show file tree
Hide file tree
Showing 18 changed files with 229 additions and 6 deletions.
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/comment.bnf
Original file line number Diff line number Diff line change
@@ -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
7 changes: 4 additions & 3 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -272,6 +273,9 @@ comment_text ::=
'SCONST'
| 'NULL'

schema_name ::=
name

column_path ::=
name
| prefixed_column_path
Expand Down Expand Up @@ -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'
Expand Down
6 changes: 4 additions & 2 deletions pkg/ccl/importccl/read_import_pgdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand Down
1 change: 1 addition & 0 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -416,6 +416,7 @@ const (
TableCommentType = 1
ColumnCommentType = 2
IndexCommentType = 3
SchemaCommentType = 4
)

const (
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
95 changes: 95 additions & 0 deletions pkg/sql/comment_on_schema.go
Original file line number Diff line number Diff line change
@@ -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) {}
18 changes: 18 additions & 0 deletions pkg/sql/drop_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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"
Expand Down Expand Up @@ -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)
}

Expand Down Expand Up @@ -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) {}
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -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
32 changes: 32 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema
Original file line number Diff line number Diff line change
Expand Up @@ -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
3 changes: 3 additions & 0 deletions pkg/sql/opaque.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down Expand Up @@ -238,6 +240,7 @@ func init() {
&tree.AlterRoleSet{},
&tree.CommentOnColumn{},
&tree.CommentOnDatabase{},
&tree.CommentOnSchema{},
&tree.CommentOnIndex{},
&tree.CommentOnTable{},
&tree.CreateDatabase{},
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -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()}
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/plan_opt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/sem/builtins/pg_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -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_description":
return catconstants.PgCatalogDescriptionTableID, true
default:
// We currently only support comments on pg_class objects
// (columns, tables) in this context.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
37 changes: 37 additions & 0 deletions pkg/sql/sem/tree/comment_on_schema.go
Original file line number Diff line number Diff line change
@@ -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")
}
}
10 changes: 10 additions & 0 deletions pkg/sql/sem/tree/stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 }

Expand Down Expand Up @@ -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) }
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/walk.go
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down

0 comments on commit edc8951

Please sign in to comment.