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 checking privileges/roles in the
declarative schema changer and to support the CASCADE modifier.

Release note: None
  • Loading branch information
Jason Chan committed Jun 15, 2022
1 parent aadbaf9 commit eb377e6
Show file tree
Hide file tree
Showing 9 changed files with 352 additions and 1 deletion.
212 changes: 212 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/drop_owned_by
Original file line number Diff line number Diff line change
@@ -0,0 +1,212 @@
# LogicTest: !local-legacy-schema-changer

statement ok
CREATE USER testuser2

user testuser2

statement ok
CREATE TABLE u()

user testuser

statement ok
CREATE TABLE t(a INT)

statement ok
CREATE VIEW v AS SELECT 1

query TTTTIT
SHOW TABLES FROM public
----
public t table testuser 0 NULL
public u table testuser2 0 NULL
public v view testuser 0 NULL

statement ok
DROP OWNED BY testuser

query TTTTIT
SHOW TABLES FROM public
----
public u table testuser2 0 NULL

query error pgcode 42P01 relation "t" does not exist
SELECT * FROM t

query error pgcode 42P01 relation "v" does not exist
SELECT * FROM v

statement ok
CREATE TABLE t1(a INT)

statement ok
CREATE TABLE t2(b INT)

statement ok
GRANT SELECT ON t1 TO testuser2 WITH GRANT OPTION

user testuser2

statement ok
CREATE VIEW v1 AS SELECT a FROM t1

user testuser

statement error pq: cannot drop desired object\(s\) because other objects depend on them
DROP OWNED BY testuser

statement error pq: cannot drop desired object\(s\) because other objects depend on them
DROP OWNED BY testuser RESTRICT

query TTTTIT
SHOW TABLES FROM public
----
public t1 table testuser 0 NULL
public t2 table testuser 0 NULL
public u table testuser2 0 NULL
public v1 view testuser2 0 NULL

user testuser2

statement ok
DROP OWNED BY testuser2

user testuser

statement ok
DROP OWNED BY testuser

query TTTTIT
SHOW TABLES FROM public
----

user root

statement ok
GRANT ALL ON DATABASE test TO testuser WITH GRANT OPTION

user testuser

statement ok
CREATE SCHEMA s

statement ok
CREATE TABLE s.t1()

statement ok
CREATE TABLE s.t2()

statement ok
GRANT CREATE ON SCHEMA s TO testuser2 WITH GRANT OPTION

user testuser2

statement ok
CREATE TABLE s.t3()

user testuser

statement error pq: cannot drop desired object\(s\) because other objects depend on them
DROP OWNED BY testuser

user root

statement ok
GRANT testuser2 TO testuser

user testuser

statement ok
DROP OWNED BY testuser, testuser2

statement error pq: target database or schema does not exist
SHOW TABLES FROM s

user root

statement ok
REVOKE testuser2 FROM testuser

query TTTB
SHOW GRANTS ON DATABASE test
----
test admin ALL true
test public CONNECT false
test root ALL true
test testuser ALL true

statement ok
DROP OWNED BY testuser

query TTTB
SHOW GRANTS ON DATABASE test
----
test admin ALL true
test public CONNECT false
test root ALL true

statement ok
CREATE SCHEMA s

statement ok
GRANT CREATE ON SCHEMA s TO testuser WITH GRANT OPTION

user testuser

statement ok
CREATE TABLE s.t()

statement ok
DROP OWNED BY testuser

query TTTTB
SHOW GRANTS ON SCHEMA s
----
test s admin ALL true
test s root ALL true

query TTTTIT
SHOW TABLES FROM s
----

user root

statement ok
DROP SCHEMA s

statement ok
CREATE TABLE t()

statement ok
GRANT ALL ON t TO testuser WITH GRANT OPTION

user testuser

query TTTTTB
SHOW GRANTS ON t
----
test public t admin ALL true
test public t root ALL true
test public t testuser ALL true

statement ok
DROP OWNED BY testuser

query TTTTTB
SHOW GRANTS ON t
----
test public t admin ALL true
test public t root ALL true

user root

statement ok
DROP TABLE t

user root
statement error pq: cannot drop objects owned by role "root" because they are required by the database system
DROP OWNED BY root

statement error pq: cannot drop objects owned by role "admin" because they are required by the database system
DROP OWNED BY admin
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
105 changes: 105 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,105 @@
// 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/scerrors"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

// 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() {
panic(pgerror.Newf(pgcode.DependentObjectsStillExist,
"cannot drop objects owned by role %q because they are required by the database system", role))
}
// TODO(jasonmchan): check that current user is part of all specified roles
}

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 {
// TODO(jasonmchan): implement for #55908
panic(scerrors.NotImplementedError(n))
} else {
if dropRestrictDescriptor(b, id) {
toCheckBackrefs = append(toCheckBackrefs, id)
}
}
continue
}
scpb.ForEachUserPrivileges(elts, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.UserPrivileges) {
if e.UserName == role.Normalized() {
dropElement(b, 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() {
dropElement(b, e)
break
}
}
})

b.IncrementSubWorkID()
b.IncrementSchemaChangeDropCounter("owned_by")

// 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
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_library(
deps = [
"//pkg/jobs/jobspb",
"//pkg/keys",
"//pkg/security/username",
"//pkg/sql/catalog",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descpb",
Expand Down
14 changes: 14 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/drop.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"

"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/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
Expand Down Expand Up @@ -132,6 +133,19 @@ func (m *visitor) RemoveDatabaseRoleSettings(
return m.s.DeleteDatabaseRoleSettings(ctx, op.DatabaseID)
}

func (m *visitor) RemoveUserPrivileges(ctx context.Context, op scop.RemoveUserPrivileges) error {
desc, err := m.s.CheckOutDescriptor(ctx, op.DescID)
if err != nil {
return err
}
user, err := username.MakeSQLUsernameFromUserInput(op.User, username.PurposeValidation)
if err != nil {
return err
}
desc.GetPrivileges().RemoveUser(user)
return nil
}

func (m *visitor) DeleteSchedule(_ context.Context, op scop.DeleteSchedule) error {
if op.ScheduleID != 0 {
m.s.DeleteSchedule(op.ScheduleID)
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/schemachanger/scop/mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,6 +530,12 @@ type RemoveDatabaseRoleSettings struct {
DatabaseID descpb.ID
}

type RemoveUserPrivileges struct {
mutationOp
DescID descpb.ID
User string
}

// DeleteSchedule is used to delete a schedule ID from the database.
type DeleteSchedule struct {
mutationOp
Expand Down
Loading

0 comments on commit eb377e6

Please sign in to comment.