Skip to content

Commit

Permalink
sql/schemachanger: implement COMMENT ON in new schema changer
Browse files Browse the repository at this point in the history
This commit implements COMMENT ON with the new schema changer
framework along with a few minor changes required for the
functionality:
(1) Adds `Comment` as an element attribute so that existing comment
can be distinguished from new comment for an upsert.
(2) Add methods to resolve constraint by name.
(3) Add methods to resolve index with best effort given a naked name.

Release note: None
  • Loading branch information
chengxiong-ruan committed Apr 5, 2022
1 parent 3324c28 commit 87eee63
Show file tree
Hide file tree
Showing 27 changed files with 590 additions and 21 deletions.
48 changes: 48 additions & 0 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -675,6 +675,27 @@ func (b *builderState) ResolveIndex(
})
}

// ResolveTableWithIndexBestEffort implements the scbuildstmt.NameResolver interface.
// Current database is used if no explicit db is specified and public schema is used if
// no explicit schema is specified.
func (b *builderState) ResolveTableWithIndexBestEffort(
indexName tree.Name, prefix tree.ObjectNamePrefix, p scbuildstmt.ResolveParams,
) scbuildstmt.ElementResultSet {
// Use public schema by default.
if !prefix.ExplicitSchema {
prefix.SchemaName = "public"
prefix.ExplicitSchema = true
}
// Use current database by default.
if !prefix.ExplicitCatalog {
prefix.CatalogName = tree.Name(b.cr.CurrentDatabase())
prefix.ExplicitCatalog = true
}

_, tableName := b.cr.MayResolveIndex(b.ctx, indexName, prefix)
return b.ResolveTable(tableName.ToUnresolvedObjectName(), p)
}

// ResolveColumn implements the scbuildstmt.NameResolver interface.
func (b *builderState) ResolveColumn(
relationID catid.DescID, columnName tree.Name, p scbuildstmt.ResolveParams,
Expand Down Expand Up @@ -702,6 +723,33 @@ func (b *builderState) ResolveColumn(
})
}

// ResolveConstraint implements the scbuildstmt.NameResolver interface.
func (b *builderState) ResolveConstraint(
relationID catid.DescID, constraintName tree.Name, p scbuildstmt.ResolveParams,
) scbuildstmt.ElementResultSet {
b.ensureDescriptor(relationID)
c := b.descCache[relationID]
rel := c.desc.(catalog.TableDescriptor)
var constraintID catid.ConstraintID
scpb.ForEachConstraintName(c.ers, func(status scpb.Status, _ scpb.TargetStatus, e *scpb.ConstraintName) {
if e.TableID == relationID && tree.Name(e.Name) == constraintName {
constraintID = e.ConstraintID
}
})
if constraintID == 0 {
if p.IsExistenceOptional {
return nil
}
panic(pgerror.Newf(pgcode.UndefinedObject,
"constraint %q of relation %q does not exist", constraintName, rel.GetName()))
}

return c.ers.Filter(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) bool {
idI, _ := screl.Schema.GetAttribute(screl.ConstraintID, e)
return idI != nil && idI.(catid.ConstraintID) == constraintID
})
}

func (b *builderState) ensureDescriptor(id catid.DescID) {
if _, found := b.descCache[id]; found {
return
Expand Down
15 changes: 14 additions & 1 deletion pkg/sql/schemachanger/scbuild/builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ func run(
withDependencies func(*testing.T, serverutils.TestServerInterface, *sqlutils.SQLRunner, func(scbuild.Dependencies)),
) string {
switch d.Cmd {
case "create-table", "create-view", "create-type", "create-sequence", "create-schema", "create-database":
case "create-table", "create-view", "create-type", "create-sequence", "create-schema", "create-database", "comment-on":
stmts, err := parser.Parse(d.Input)
require.NoError(t, err)
require.Len(t, stmts, 1)
Expand Down Expand Up @@ -142,6 +142,19 @@ func run(
t.Logf("created relation with id %d", tableID)
}

return ""
case "descriptor-metadata":
stmts, err := parser.Parse(d.Input)
require.NoError(t, err)
for _, stmt := range stmts {
switch stmt.AST.(type) {
case *tree.CommentOnDatabase, *tree.CommentOnSchema, *tree.CommentOnTable, *tree.CommentOnColumn,
*tree.CommentOnIndex, *tree.CommentOnConstraint:
tdb.Exec(t, stmt.SQL)
default:
t.Fatal("not a supported descriptor metadata statement")
}
}
return ""
case "build":
if a := d.CmdArgs; len(a) > 0 && a[0].Key == "skip" {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/schemachanger/scbuild/dependencies.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,10 @@ type CatalogReader interface {
// MayResolveTable looks up a table by name.
MayResolveTable(ctx context.Context, name tree.UnresolvedObjectName) (catalog.ResolvedObjectPrefix, catalog.TableDescriptor)

// MayResolveIndex looks up an index using a naked index name with database
// and schema prefix. Resolved index and the owner table name are returned.
MayResolveIndex(ctx context.Context, indexName tree.Name, prefix tree.ObjectNamePrefix) (catalog.Index, tree.TableName)

// MayResolveType looks up a type by name.
MayResolveType(ctx context.Context, name tree.UnresolvedObjectName) (catalog.ResolvedObjectPrefix, catalog.TypeDescriptor)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"alter_table.go",
"alter_table_add_column.go",
"comment_on.go",
"create_index.go",
"dependencies.go",
"drop_database.go",
Expand Down
230 changes: 230 additions & 0 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/comment_on.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,230 @@
// Copyright 2022 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 scbuildstmt

import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

// CommentOnDatabase implements COMMENT ON DATABASE xxx IS xxx statement.
func CommentOnDatabase(b BuildCtx, n *tree.CommentOnDatabase) {
dc := &scpb.DatabaseComment{}
if n.Comment != nil {
dc.Comment = *n.Comment
}

dbElements := b.ResolveDatabase(n.Name, ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
})
dbElements.ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Database:
dc.DatabaseID = t.DatabaseID
case *scpb.DatabaseComment:
if len(dc.Comment) == 0 {
b.Drop(e)
}
}
})

if len(dc.Comment) > 0 {
b.Add(dc)
}
}

// CommentOnSchema implements COMMENT ON SCHEMA xxx IS xxx statement;
func CommentOnSchema(b BuildCtx, n *tree.CommentOnSchema) {
sc := &scpb.SchemaComment{}
if n.Comment != nil {
sc.Comment = *n.Comment
}

schemaElements := b.ResolveSchema(
n.Name,
ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
},
)

schemaElements.ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Schema:
sc.SchemaID = t.SchemaID
case *scpb.SchemaComment:
if len(sc.Comment) == 0 {
b.Drop(e)
}
}
})

if len(sc.Comment) > 0 {
b.Add(sc)
}
}

// CommentOnTable implements COMMENT ON TABLE xxx IS xxx statement.
func CommentOnTable(b BuildCtx, n *tree.CommentOnTable) {
tc := &scpb.TableComment{}
if n.Comment != nil {
tc.Comment = *n.Comment
}

tableElements := b.ResolveTable(n.Table, ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
})

tableElements.ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Table:
tc.TableID = t.TableID
case *scpb.TableComment:
if len(tc.Comment) == 0 {
b.Drop(e)
}
}
})

if len(tc.Comment) > 0 {
b.Add(tc)
}
}

// CommentOnColumn implements COMMENT ON COLUMN xxx IS xxx statement.
func CommentOnColumn(b BuildCtx, n *tree.CommentOnColumn) {
cc := &scpb.ColumnComment{}
if n.Comment != nil {
cc.Comment = *n.Comment
}

params := ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
}
tableElements := b.ResolveTable(n.ColumnItem.TableName, params)
tableElements.ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Table:
cc.TableID = t.TableID
}
})

columnElements := b.ResolveColumn(cc.TableID, n.ColumnItem.ColumnName, params)

columnElements.ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Column:
cc.ColumnID = t.ColumnID
case *scpb.ColumnComment:
if len(cc.Comment) == 0 {
b.Drop(e)
}
}
})

if len(cc.Comment) > 0 {
b.Add(cc)
}
}

// CommentOnIndex implements COMMENT ON INDEX xxx iS xxx statement.
func CommentOnIndex(b BuildCtx, n *tree.CommentOnIndex) {
ic := &scpb.IndexComment{}
if n.Comment != nil {
ic.Comment = *n.Comment
}

params := ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
}

var tableElements ElementResultSet
if len(n.Index.Table.ObjectName) > 0 {
tableElements = b.ResolveTable(n.Index.Table.ToUnresolvedObjectName(), params)
} else {
tableElements = b.ResolveTableWithIndexBestEffort(tree.Name(n.Index.Index), n.Index.Table.ObjectNamePrefix, params)
}

tableElements.ForEachElementStatus(func(_ scpb.Status, target scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Table:
// Mutate the AST to have the fully resolved name from above, which will be
// used for both event logging and errors. This is enforced by validation.
n.Index.Table.ObjectNamePrefix = b.NamePrefix(t)
ic.TableID = t.TableID
}
})

indexElements := b.ResolveIndex(ic.TableID, tree.Name(n.Index.Index), params)

indexElements.ForEachElementStatus(func(_ scpb.Status, target scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.SecondaryIndex:
ic.IndexID = t.IndexID
case *scpb.PrimaryIndex:
ic.IndexID = t.IndexID
case *scpb.IndexComment:
if len(ic.Comment) == 0 {
b.Drop(e)
}
}
})

if len(ic.Comment) > 0 {
b.Add(ic)
}
}

// CommentOnConstraint implements COMMENT ON CONSTRAINT xxx ON table_name IS xxx
// statement.
func CommentOnConstraint(b BuildCtx, n *tree.CommentOnConstraint) {
params := ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
}
cc := &scpb.ConstraintComment{}
if n.Comment != nil {
cc.Comment = *n.Comment
}

tableElements := b.ResolveTable(n.Table, params)
tableElements.ForEachElementStatus(func(_ scpb.Status, target scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.Table:
cc.TableID = t.TableID
}
})

constraintElements := b.ResolveConstraint(cc.TableID, n.Constraint, params)
constraintElements.ForEachElementStatus(func(_ scpb.Status, target scpb.TargetStatus, e scpb.Element) {
switch t := e.(type) {
case *scpb.CheckConstraint:
cc.ConstraintID = t.ConstraintID
case *scpb.ForeignKeyConstraint:
cc.ConstraintID = t.ConstraintID
case *scpb.UniqueWithoutIndexConstraint:
cc.ConstraintID = t.ConstraintID
case *scpb.ConstraintComment:
if len(cc.Comment) == 0 {
b.Drop(e)
}
}
})

if len(cc.Comment) > 0 {
b.Add(cc)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -261,6 +261,12 @@ type NameResolver interface {
// ResolveIndex retrieves an index by name and returns its elements.
ResolveIndex(relationID catid.DescID, indexName tree.Name, p ResolveParams) ElementResultSet

// ResolveTableWithIndexBestEffort retrieves a table which contains the target index and returns its elements.
ResolveTableWithIndexBestEffort(indexName tree.Name, prefix tree.ObjectNamePrefix, p ResolveParams) ElementResultSet

// ResolveColumn retrieves a column by name and returns its elements.
ResolveColumn(relationID catid.DescID, columnName tree.Name, p ResolveParams) ElementResultSet

// ResolveConstraint retrieves a constraint by name and returns its elements.
ResolveConstraint(relationID catid.DescID, constraintName tree.Name, p ResolveParams) ElementResultSet
}
22 changes: 14 additions & 8 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,14 +46,20 @@ var supportedStatements = map[reflect.Type]supportedStatement{
// Alter table will have commands individually whitelisted via the
// supportedAlterTableStatements list, so wwe will consider it fully supported
// here.
reflect.TypeOf((*tree.AlterTable)(nil)): {AlterTable, true},
reflect.TypeOf((*tree.CreateIndex)(nil)): {CreateIndex, false},
reflect.TypeOf((*tree.DropDatabase)(nil)): {DropDatabase, true},
reflect.TypeOf((*tree.DropSchema)(nil)): {DropSchema, true},
reflect.TypeOf((*tree.DropSequence)(nil)): {DropSequence, true},
reflect.TypeOf((*tree.DropTable)(nil)): {DropTable, true},
reflect.TypeOf((*tree.DropType)(nil)): {DropType, true},
reflect.TypeOf((*tree.DropView)(nil)): {DropView, true},
reflect.TypeOf((*tree.AlterTable)(nil)): {AlterTable, true},
reflect.TypeOf((*tree.CreateIndex)(nil)): {CreateIndex, false},
reflect.TypeOf((*tree.DropDatabase)(nil)): {DropDatabase, true},
reflect.TypeOf((*tree.DropSchema)(nil)): {DropSchema, true},
reflect.TypeOf((*tree.DropSequence)(nil)): {DropSequence, true},
reflect.TypeOf((*tree.DropTable)(nil)): {DropTable, true},
reflect.TypeOf((*tree.DropType)(nil)): {DropType, true},
reflect.TypeOf((*tree.DropView)(nil)): {DropView, true},
reflect.TypeOf((*tree.CommentOnDatabase)(nil)): {CommentOnDatabase, false},
reflect.TypeOf((*tree.CommentOnSchema)(nil)): {CommentOnSchema, false},
reflect.TypeOf((*tree.CommentOnTable)(nil)): {CommentOnTable, false},
reflect.TypeOf((*tree.CommentOnColumn)(nil)): {CommentOnColumn, false},
reflect.TypeOf((*tree.CommentOnIndex)(nil)): {CommentOnIndex, false},
reflect.TypeOf((*tree.CommentOnConstraint)(nil)): {CommentOnConstraint, false},
}

func init() {
Expand Down
Loading

0 comments on commit 87eee63

Please sign in to comment.