Skip to content

Commit

Permalink
sql/schemachanger: implement DROP OWNED BY
Browse files Browse the repository at this point in the history
Previously, we did not support the DROP OWNED BY statement (#55381).
This commit adds partial support for DROP OWNED BY in the declarative
schema changer. Followup work is needed to support the CASCADE modifier.

Release note (sql change): Support `DROP OWNED BY`.
  • Loading branch information
Jason Chan committed Jun 24, 2022
1 parent aadbaf9 commit c9edde5
Show file tree
Hide file tree
Showing 18 changed files with 2,335 additions and 4 deletions.
626 changes: 626 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/drop_owned_by

Large diffs are not rendered by default.

14 changes: 14 additions & 0 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"strings"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -155,6 +156,19 @@ func (b *builderState) checkPrivilege(id catid.DescID, priv privilege.Kind) {
}
}

// CurrentUserHasAdminOrIsMemberOf implements the scbuildstmt.PrivilegeChecker interface.
func (b *builderState) CurrentUserHasAdminOrIsMemberOf(role username.SQLUsername) bool {
if b.hasAdmin {
return true
}
memberships, err := b.auth.MemberOfWithAdminOption(b.ctx, role)
if err != nil {
panic(err)
}
_, ok := memberships[b.evalCtx.SessionData().User()]
return ok
}

var _ scbuildstmt.TableHelpers = (*builderState)(nil)

// NextTableColumnID implements the scbuildstmt.TableHelpers interface.
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 @@ -167,6 +167,10 @@ type AuthorizationAccessor interface {
CheckPrivilegeForUser(
ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind, user username.SQLUsername,
) error

// MemberOfWithAdminOption looks up all the roles 'member' belongs to (direct
// and indirect) and returns a map of "role" -> "isAdmin".
MemberOfWithAdminOption(ctx context.Context, member username.SQLUsername) (map[username.SQLUsername]bool, error)
}

// AstFormatter provides interfaces for formatting AST nodes.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
"create_index.go",
"dependencies.go",
"drop_database.go",
"drop_owned_by.go",
"drop_schema.go",
"drop_sequence.go",
"drop_table.go",
Expand All @@ -20,6 +21,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild/internal/scbuildstmt",
visibility = ["//pkg/sql/schemachanger/scbuild:__subpackages__"],
deps = [
"//pkg/security/username",
"//pkg/server/telemetry",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
Expand All @@ -29,6 +31,7 @@ go_library(
"//pkg/sql/catalog/schemaexpr",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/catalog/typedesc",
"//pkg/sql/decodeusername",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package scbuildstmt
import (
"context"

"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
Expand Down Expand Up @@ -148,6 +149,9 @@ type Telemetry interface {

// IncrementEnumCounter increments the selected enum telemetry counter.
IncrementEnumCounter(counterType sqltelemetry.EnumTelemetryType)

// IncrementDropOwnedByCounter increments the DROP OWNED BY telemetry counter.
IncrementDropOwnedByCounter()
}

// SchemaFeatureChecker checks if a schema change feature is allowed by the
Expand All @@ -167,6 +171,10 @@ type PrivilegeChecker interface {
// CheckPrivilege panics if the current user does not have the specified
// privilege for the element.
CheckPrivilege(e scpb.Element, privilege privilege.Kind)

// CurrentUserHasAdminOrIsMemberOf returns true iff the current user is (1)
// an admin or (2) has membership in the specified role.
CurrentUserHasAdminOrIsMemberOf(member username.SQLUsername) bool
}

// TableHelpers has methods useful for creating new table elements.
Expand Down
106 changes: 106 additions & 0 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_owned_by.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
// 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/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/decodeusername"
"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/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
)

// DropOwnedBy implements DROP OWNED BY.
func DropOwnedBy(b BuildCtx, n *tree.DropOwnedBy) {
normalizedRoles, err := decodeusername.FromRoleSpecList(
b.SessionData(), username.PurposeValidation, n.Roles,
)
if err != nil {
panic(err)
}
for _, role := range normalizedRoles {
if role.IsAdminRole() || role.IsRootUser() || role.IsNodeUser() {
panic(pgerror.Newf(pgcode.DependentObjectsStillExist,
"cannot drop objects owned by role %q because they are required by the database system", role))
}
if role != b.SessionData().User() && !b.CurrentUserHasAdminOrIsMemberOf(role) {
panic(pgerror.New(pgcode.InsufficientPrivilege, "permission denied to drop objects"))
}
}

var objects []descpb.ID
var toCheckBackrefs []descpb.ID

// Lookup all objects in the current database.
_, _, db := scpb.FindDatabase(b.ResolveDatabase(tree.Name(b.SessionData().Database), ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CONNECT,
}))
dbRefs := undroppedBackrefs(b, db.DatabaseID)
scpb.ForEachSchemaParent(dbRefs, func(_ scpb.Status, _ scpb.TargetStatus, sp *scpb.SchemaParent) {
schemaRefs := undroppedBackrefs(b, sp.SchemaID)
scpb.ForEachObjectParent(schemaRefs, func(_ scpb.Status, _ scpb.TargetStatus, op *scpb.ObjectParent) {
objects = append(objects, op.ObjectID)
})
objects = append(objects, sp.SchemaID)
})

// Drop owned objects and revoke user privileges for the specified roles.
for _, id := range objects {
elts := b.QueryByID(id)
_, _, owner := scpb.FindOwner(elts)
for _, role := range normalizedRoles {
if owner.Owner == role.Normalized() {
if n.DropBehavior == tree.DropCascade {
panic(unimplemented.NewWithIssue(55908, "DROP OWNED BY CASCADE is not yet supported"))
} else {
if dropRestrictDescriptor(b, id) {
toCheckBackrefs = append(toCheckBackrefs, id)
}
}
break
}
scpb.ForEachUserPrivileges(elts, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.UserPrivileges) {
if e.UserName == role.Normalized() {
b.Drop(e)
}
})
}
}

// Revoke privileges for the database. The current user shouldn't revoke
// their own database privileges.
dbElts := b.QueryByID(db.DatabaseID)
scpb.ForEachUserPrivileges(dbElts, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.UserPrivileges) {
for _, role := range normalizedRoles {
if e.UserName == role.Normalized() && e.UserName != b.SessionData().User().Normalized() {
b.Drop(e)
break
}
}
})

b.IncrementSubWorkID()
b.IncrementDropOwnedByCounter()

// Enforce RESTRICT semantics by checking for backreferences.
for _, id := range toCheckBackrefs {
backrefs := undroppedBackrefs(b, id)
if !backrefs.IsEmpty() {
panic(pgerror.New(pgcode.DependentObjectsStillExist,
"cannot drop desired object(s) because other objects depend on them"))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ var supportedStatements = map[reflect.Type]supportedStatement{
reflect.TypeOf((*tree.AlterTable)(nil)): {AlterTable, true},
reflect.TypeOf((*tree.CreateIndex)(nil)): {CreateIndex, false},
reflect.TypeOf((*tree.DropDatabase)(nil)): {DropDatabase, true},
reflect.TypeOf((*tree.DropOwnedBy)(nil)): {DropOwnedBy, true},
reflect.TypeOf((*tree.DropSchema)(nil)): {DropSchema, true},
reflect.TypeOf((*tree.DropSequence)(nil)): {DropSequence, true},
reflect.TypeOf((*tree.DropTable)(nil)): {DropTable, true},
Expand Down
Loading

0 comments on commit c9edde5

Please sign in to comment.