Skip to content

Commit

Permalink
Merge #32442
Browse files Browse the repository at this point in the history
32442: sql: implement `COMMENT ON TABLE` r=knz a=hueypark

Informs  #19472.

This patch introduces support for table comments.

The syntax to set or delete a comment is the same as postgres:
`COMMENT ON TABLE ... IS ...`. See:
postgresql.org/docs/9.1/sql-comment.html

This also makes `pg_catalog.pg_description` and `obj_description()` do
the right thing for compatibility with 3rd party clients.

This is supported by a new system table `system.comments`, which is
extensible to support comments on other database objects than tables:

- its `type` column indicates the type of object, to distinguish
  between db, table, column and others. For now just one type is
  defined.
- `object_id`: table or database ID, relative to the `type`.
- `sub_id`: when a comment is placed on an object "inside" another, eg
  a column inside a table.
- `comment`: the comment proper.

This design of `system.comments` mimics pg's own `pg_description`
which uses the same schema.

Release note (sql change): CockroachDB now supports associating
comments to SQL tables using PostgreSQL's `COMMENT ON TABLE`
syntax. This also provides proper support for pg's
`pg_catalog.pg_description` and built-in function `obj_description()`.

Release note (sql change): The `SHOW TABLES` statement
now supports printing out table comments using the optional phrase
`WITH COMMENT`, e.g `SHOW TABLES FROM mydb WITH COMMENT`.

Co-authored-by: Jaewan Park <jaewan.huey.park@gmail.com>
  • Loading branch information
craig[bot] and hueypark committed Dec 9, 2018
2 parents fd28ed9 + 70c8f59 commit 783767c
Show file tree
Hide file tree
Showing 36 changed files with 848 additions and 184 deletions.
18 changes: 15 additions & 3 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ stmt ::=
'HELPTOKEN'
| preparable_stmt
| copy_from_stmt
| comment_stmt
| execute_stmt
| deallocate_stmt
| discard_stmt
Expand Down Expand Up @@ -46,6 +47,9 @@ preparable_stmt ::=
copy_from_stmt ::=
'COPY' table_name opt_column_list 'FROM' 'STDIN'

comment_stmt ::=
'COMMENT' 'ON' 'TABLE' table_name 'IS' comment_text

execute_stmt ::=
'EXECUTE' table_alias_name execute_param_clause

Expand Down Expand Up @@ -198,6 +202,10 @@ opt_column_list ::=
'(' name_list ')'
|

comment_text ::=
'SCONST'
| 'NULL'

table_alias_name ::=
name

Expand Down Expand Up @@ -488,9 +496,9 @@ show_stats_stmt ::=
'SHOW' 'STATISTICS' 'FOR' 'TABLE' table_name

show_tables_stmt ::=
'SHOW' 'TABLES' 'FROM' name '.' name
| 'SHOW' 'TABLES' 'FROM' name
| 'SHOW' 'TABLES'
'SHOW' 'TABLES' 'FROM' name '.' name with_comment
| 'SHOW' 'TABLES' 'FROM' name with_comment
| 'SHOW' 'TABLES' with_comment

show_trace_stmt ::=
'SHOW' opt_compact 'TRACE' 'FOR' 'SESSION'
Expand Down Expand Up @@ -1097,6 +1105,10 @@ table_name_with_index ::=
table_name '@' index_name
| table_name

with_comment ::=
'WITH' 'COMMENT'
|

opt_compact ::=
'COMPACT'
|
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/role
Original file line number Diff line number Diff line change
Expand Up @@ -718,10 +718,10 @@ CREATE USER public
statement error role name "public" is reserved
CREATE ROLE public

statement error cannot drop special role public
statement error cannot drop user or role public: grants still exist on system.public.comments
DROP USER public

statement error cannot drop special role public
statement error cannot drop user or role public: grants still exist on system.public.comments
DROP ROLE public

statement error role public does not exist
Expand Down
2 changes: 2 additions & 0 deletions pkg/cli/cli_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2334,10 +2334,12 @@ writing ` + os.DevNull + `
debug/nodes/1/ranges/18
debug/nodes/1/ranges/19
debug/nodes/1/ranges/20
debug/nodes/1/ranges/21
debug/reports/problemranges
debug/schema/defaultdb@details
debug/schema/postgres@details
debug/schema/system@details
debug/schema/system/comments
debug/schema/system/descriptor
debug/schema/system/eventlog
debug/schema/system/jobs
Expand Down
6 changes: 6 additions & 0 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -328,4 +328,10 @@ const (
LocationsTableID = 21
LivenessRangesID = 22
RoleMembersTableID = 23
CommentsTableID = 24

// CommentType is type for system.comments
// DatabaseCommentType = 0
TableCommentType = 1
// ColumnCommentType = 2
)
94 changes: 94 additions & 0 deletions pkg/sql/comment_on_table.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package sql

import (
"context"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

type commentOnTableNode struct {
n *tree.CommentOnTable
tableDesc *MutableTableDescriptor
}

// CommentOnTable add comment on a table.
// Privileges: CREATE on table.
// 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)
if err != nil {
return nil, err
}

if err := p.CheckPrivilege(ctx, tableDesc, privilege.CREATE); err != nil {
return nil, err
}

return &commentOnTableNode{n: n, tableDesc: tableDesc}, nil
}

func (n *commentOnTableNode) startExec(params runParams) error {
if n.n.Comment != nil {
_, err := params.p.extendedEvalCtx.ExecCfg.InternalExecutor.Exec(
params.ctx,
"upsert-comment",
params.p.Txn(),
"UPSERT INTO system.comments VALUES ($1, $2, 0, $3)",
keys.TableCommentType,
n.tableDesc.ID,
*n.n.Comment)
if err != nil {
return err
}
} else {
_, err := params.p.extendedEvalCtx.ExecCfg.InternalExecutor.Exec(
params.ctx,
"delete-comment",
params.p.Txn(),
"DELETE FROM system.comments WHERE type=$1 AND object_id=$2 AND sub_id=0",
keys.TableCommentType,
n.tableDesc.ID)
if err != nil {
return err
}
}

return MakeEventLogger(params.extendedEvalCtx.ExecCfg).InsertEventRecord(
params.ctx,
params.p.txn,
EventLogCommentOnTable,
int32(n.tableDesc.ID),
int32(params.extendedEvalCtx.NodeID),
struct {
TableName string
Statement string
User string
Comment *string
}{
n.n.Table.FQString(),
n.n.String(),
params.SessionData().User,
n.n.Comment},
)
}

func (n *commentOnTableNode) Next(runParams) (bool, error) { return false, nil }
func (n *commentOnTableNode) Values() tree.Datums { return tree.Datums{} }
func (n *commentOnTableNode) Close(context.Context) {}
113 changes: 113 additions & 0 deletions pkg/sql/comment_on_table_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package sql_test

import (
"context"
gosql "database/sql"
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

func TestCommentOnTable(t *testing.T) {
defer leaktest.AfterTest(t)()

params, _ := tests.CreateTestServerParams()
s, db, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.TODO())

if _, err := db.Exec(`
CREATE DATABASE d;
SET DATABASE = d;
CREATE TABLE t (i INT );
`); err != nil {
t.Fatal(err)
}

testCases := []struct {
exec string
query string
expect gosql.NullString
}{
{
`COMMENT ON TABLE t IS 'foo'`,
`SELECT obj_description('t'::regclass)`,
gosql.NullString{String: `foo`, Valid: true},
},
{
`TRUNCATE t`,
`SELECT obj_description('t'::regclass)`,
gosql.NullString{String: `foo`, Valid: true},
},
{
`COMMENT ON TABLE t IS NULL`,
`SELECT obj_description('t'::regclass)`,
gosql.NullString{Valid: false},
},
}

for _, tc := range testCases {
if _, err := db.Exec(tc.exec); err != nil {
t.Fatal(err)
}

row := db.QueryRow(tc.query)
var comment gosql.NullString
if err := row.Scan(&comment); err != nil {
t.Fatal(err)
}
if tc.expect != comment {
t.Fatalf("expected comment %v, got %v", tc.expect, comment)
}
}
}

func TestCommentOnTableWhenDrop(t *testing.T) {
defer leaktest.AfterTest(t)()

params, _ := tests.CreateTestServerParams()
s, db, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.TODO())

if _, err := db.Exec(`
CREATE DATABASE d;
SET DATABASE = d;
CREATE TABLE t (i INT );
`); err != nil {
t.Fatal(err)
}

if _, err := db.Exec(`COMMENT ON TABLE t IS 'foo'`); err != nil {
t.Fatal(err)
}

if _, err := db.Exec(`DROP TABLE t`); err != nil {
t.Fatal(err)
}

row := db.QueryRow(`SELECT comment FROM system.comments LIMIT 1`)
var comment string
err := row.Scan(&comment)
if err != gosql.ErrNoRows {
if err != nil {
t.Fatal(err)
}

t.Fatal("dropped comment remain comment")
}
}
22 changes: 21 additions & 1 deletion pkg/sql/drop_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,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/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
Expand Down Expand Up @@ -313,7 +314,12 @@ func (p *planner) dropTableImpl(
droppedViews = append(droppedViews, viewDesc.Name)
}

err := p.initiateDropTable(ctx, tableDesc, true /* drain name */)
err := p.removeComment(ctx, tableDesc)
if err != nil {
return droppedViews, err
}

err = p.initiateDropTable(ctx, tableDesc, true /* drain name */)
return droppedViews, err
}

Expand Down Expand Up @@ -475,3 +481,17 @@ func removeMatchingReferences(
}
return updatedRefs
}

func (p *planner) removeComment(
ctx context.Context, tableDesc *sqlbase.MutableTableDescriptor,
) error {
_, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Exec(
ctx,
"delete-comment",
p.txn,
"DELETE FROM system.comments WHERE type=$1 AND object_id=$2 AND sub_id=0",
keys.TableCommentType,
tableDesc.ID)

return err
}
2 changes: 2 additions & 0 deletions pkg/sql/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@ const (
EventLogTruncateTable EventLogType = "truncate_table"
// EventLogAlterTable is recorded when a table is altered.
EventLogAlterTable EventLogType = "alter_table"
// EventLogCommentOnTable is recorded when a table is commented.
EventLogCommentOnTable EventLogType = "comment_on_table"

// EventLogCreateIndex is recorded when an index is created.
EventLogCreateIndex EventLogType = "create_index"
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/expand_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -350,6 +350,7 @@ func doExpandPlan(
case *alterTableNode:
case *alterSequenceNode:
case *alterUserSetPasswordNode:
case *commentOnTableNode:
case *renameColumnNode:
case *renameDatabaseNode:
case *renameIndexNode:
Expand Down Expand Up @@ -860,6 +861,7 @@ func (p *planner) simplifyOrderings(plan planNode, usefulOrdering sqlbase.Column
case *alterTableNode:
case *alterSequenceNode:
case *alterUserSetPasswordNode:
case *commentOnTableNode:
case *renameColumnNode:
case *renameDatabaseNode:
case *renameIndexNode:
Expand Down
20 changes: 20 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/grant_table
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,21 @@ system public NULL admin GRANT
system public NULL admin SELECT
system public NULL root GRANT
system public NULL root SELECT
system public comments admin DELETE
system public comments admin GRANT
system public comments admin INSERT
system public comments admin SELECT
system public comments admin UPDATE
system public comments public DELETE
system public comments public GRANT
system public comments public INSERT
system public comments public SELECT
system public comments public UPDATE
system public comments root DELETE
system public comments root GRANT
system public comments root INSERT
system public comments root SELECT
system public comments root UPDATE
system public descriptor admin GRANT
system public descriptor admin SELECT
system public descriptor root GRANT
Expand Down Expand Up @@ -315,6 +330,11 @@ system pg_catalog NULL root GRANT
system pg_catalog NULL root SELECT
system public NULL root GRANT
system public NULL root SELECT
system public comments root DELETE
system public comments root GRANT
system public comments root INSERT
system public comments root SELECT
system public comments root UPDATE
system public descriptor root GRANT
system public descriptor root SELECT
system public eventlog root DELETE
Expand Down
Loading

0 comments on commit 783767c

Please sign in to comment.