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 13, 2022
1 parent 851decc commit 22ba279
Show file tree
Hide file tree
Showing 33 changed files with 811 additions and 35 deletions.
18 changes: 15 additions & 3 deletions pkg/sql/descmetadata/comment_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@ package descmetadata

import (
"context"
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
Expand All @@ -33,6 +35,7 @@ type metadataCache struct {
objIDsChecked map[catid.DescID]struct{}
}

// Get implements the scdecomp.DescriptorCommentCache interface.
func (mf metadataCache) Get(
ctx context.Context, objID catid.DescID, subID descpb.ID, commentType keys.CommentType,
) (comment string, ok bool, err error) {
Expand Down Expand Up @@ -89,14 +92,23 @@ func (mf metadataCache) LoadCommentsForObjects(
return errors.Errorf("unexpected descriptor type %q for fetching comment", descType)
}

var buf strings.Builder
_, _ = fmt.Fprintf(&buf, `SELECT type, object_id, sub_id, comment FROM system.comments WHERE type IN (%d`, commentTypes[0])
for _, typ := range commentTypes[1:] {
_, _ = fmt.Fprintf(&buf, ", %d", typ)
}
_, _ = fmt.Fprintf(&buf, `) AND object_id IN (%d`, uncheckedObjIDs[0])
for _, id := range uncheckedObjIDs[1:] {
_, _ = fmt.Fprintf(&buf, ", %d", id)
}
buf.WriteString(")")

rows, err := mf.ie.QueryBufferedEx(
ctx,
"mf-get-table-comments",
mf.txn,
sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT type, obj_id, sub_id, comment FROM system.comments WHERE type IN ($1) AND object_id IN ($2)",
commentTypes,
objIDs,
buf.String(),
)

if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/scbuild/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,7 +169,7 @@ func newBuilderState(ctx context.Context, d Dependencies, initial scpb.CurrentSt
output: make([]elementState, 0, len(initial.Current)),
descCache: make(map[catid.DescID]*cachedDesc),
tempSchemas: make(map[catid.DescID]catalog.SchemaDescriptor),
commentCache: d.DescriptorMetadataFetcher(),
commentCache: d.DescriptorCommentCache(),
}
var err error
bs.hasAdmin, err = bs.auth.HasAdminRole(ctx)
Expand Down
60 changes: 58 additions & 2 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -687,6 +687,32 @@ 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(
prefix tree.ObjectNamePrefix, indexName tree.Name, p scbuildstmt.ResolveParams,
) scbuildstmt.ElementResultSet {
// Use public schema by default.
if !prefix.ExplicitSchema {
prefix.SchemaName = catconstants.PublicSchemaName
prefix.ExplicitSchema = true
}
// Use current database by default.
if !prefix.ExplicitCatalog {
prefix.CatalogName = tree.Name(b.cr.CurrentDatabase())
prefix.ExplicitCatalog = true
}

resolvedPrefix, tblDesc, _ := b.cr.MayResolveIndex(b.ctx, indexName, prefix)
tableName := tree.NewTableNameWithSchema(
tree.Name(resolvedPrefix.Database.GetName()),
tree.Name(resolvedPrefix.Schema.GetName()),
tree.Name(tblDesc.GetName()),
)
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 @@ -714,6 +740,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 Expand Up @@ -749,8 +802,11 @@ func (b *builderState) ensureDescriptor(id catid.DescID) {
})
}

if err := b.commentCache.LoadCommentsForObjects(b.ctx, c.desc.DescriptorType(), []descpb.ID{c.desc.GetID()}); err != nil {
panic(err)
switch c.desc.DescriptorType() {
case catalog.Database, catalog.Schema, catalog.Table:
if err := b.commentCache.LoadCommentsForObjects(b.ctx, c.desc.DescriptorType(), []descpb.ID{c.desc.GetID()}); err != nil {
panic(err)
}
}
c.backrefs = scdecomp.WalkDescriptor(b.ctx, c.desc, crossRefLookupFn, visitorFn, b.commentCache)
// Name prefix and namespace lookups.
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
8 changes: 6 additions & 2 deletions pkg/sql/schemachanger/scbuild/dependencies.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,9 @@ type Dependencies interface {
// partitioning descriptors for indexes.
IndexPartitioningCCLCallback() CreatePartitioningCCLCallback

// DescriptorMetadataFetcher returns a scdecomp.DescriptorCommentCache
// DescriptorCommentCache returns a scdecomp.DescriptorCommentCache
// Implementation.
DescriptorMetadataFetcher() scdecomp.DescriptorCommentCache
DescriptorCommentCache() scdecomp.DescriptorCommentCache
}

// CreatePartitioningCCLCallback is the type of the CCL callback for creating
Expand Down 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 prefix, index and the owner table are returned.
MayResolveIndex(ctx context.Context, indexName tree.Name, prefix tree.ObjectNamePrefix) (catalog.ResolvedObjectPrefix, catalog.TableDescriptor, catalog.Index)

// 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
Loading

0 comments on commit 22ba279

Please sign in to comment.