Skip to content

Commit

Permalink
sql: Introduce the concept of system cluster privileges
Browse files Browse the repository at this point in the history
System Cluster Privileges (global) privileges are the first
type of privileges that use the system.privileges table.

More functionality will be gated behind having `MODIFYCLUSTERSETTING` privilege
in the future. This is not the same as the `MODIFYCLUSTERSETTING` role option.

Release note (sql change): Introduce SYSTEM CLUSTER PRIVILEGES.
These are "global" privileges that live above the database level.

Example: `GRANT SYSTEM MODIFYCLUSTERSETTING TO foo`

Currently `MODIFYCLUSTERSETTING` is the only system cluster privilege,
it allows users to query the `crdb_internal.cluster_settings` table.
  • Loading branch information
RichardJCai committed Jun 24, 2022
1 parent b17caa8 commit 5df8cbd
Show file tree
Hide file tree
Showing 50 changed files with 1,085 additions and 227 deletions.
6 changes: 6 additions & 0 deletions docs/generated/sql/bnf/grant_stmt.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -31,3 +31,9 @@ grant_stmt ::=
| 'GRANT' 'ALL' 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
| 'GRANT' privilege_list 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
| 'GRANT' privilege_list 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list
| 'GRANT' 'SYSTEM' 'ALL' 'PRIVILEGES' 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
| 'GRANT' 'SYSTEM' 'ALL' 'PRIVILEGES' 'TO' role_spec_list
| 'GRANT' 'SYSTEM' 'ALL' 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
| 'GRANT' 'SYSTEM' 'ALL' 'TO' role_spec_list
| 'GRANT' 'SYSTEM' privilege_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION'
| 'GRANT' 'SYSTEM' privilege_list 'TO' role_spec_list
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/revoke_stmt.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -28,3 +28,6 @@ revoke_stmt ::=
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privilege_list 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
| 'REVOKE' 'SYSTEM' 'ALL' 'PRIVILEGES' 'FROM' role_spec_list
| 'REVOKE' 'SYSTEM' 'ALL' 'FROM' role_spec_list
| 'REVOKE' 'SYSTEM' privilege_list 'FROM' role_spec_list
2 changes: 2 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ grant_stmt ::=
| 'GRANT' privileges 'ON' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
| 'GRANT' privileges 'ON' 'ALL' 'SEQUENCES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
| 'GRANT' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option
| 'GRANT' 'SYSTEM' privileges 'TO' role_spec_list opt_with_grant_option

prepare_stmt ::=
'PREPARE' table_alias_name prep_type_clause 'AS' preparable_stmt
Expand All @@ -100,6 +101,7 @@ revoke_stmt ::=
| 'REVOKE' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
| 'REVOKE' privileges 'ON' 'ALL' 'SEQUENCES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
| 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list
| 'REVOKE' 'SYSTEM' privileges 'FROM' role_spec_list

savepoint_stmt ::=
'SAVEPOINT' name
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -418,6 +418,7 @@ ALL_TESTS = [
"//pkg/sql/sqltestutils:sqltestutils_test",
"//pkg/sql/stats:stats_test",
"//pkg/sql/stmtdiagnostics:stmtdiagnostics_test",
"//pkg/sql/systemprivilege:systemprivilege_test",
"//pkg/sql/tests:tests_test",
"//pkg/sql/ttl/ttljob:ttljob_test",
"//pkg/sql/types:types_disallowed_imports_test",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,7 @@ go_library(
"export.go",
"filter.go",
"grant_revoke.go",
"grant_revoke_system.go",
"grant_role.go",
"group.go",
"index_backfiller.go",
Expand Down
56 changes: 35 additions & 21 deletions pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
Expand Down Expand Up @@ -72,16 +73,14 @@ type userRoleMembership map[username.SQLUsername]bool
type AuthorizationAccessor interface {
// CheckPrivilege verifies that the user has `privilege` on `descriptor`.
CheckPrivilegeForUser(
ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind, user username.SQLUsername,
ctx context.Context, privilegeObject catalog.PrivilegeObject, privilege privilege.Kind, user username.SQLUsername,
) error

// CheckPrivilege verifies that the current user has `privilege` on `descriptor`.
CheckPrivilege(
ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind,
) error
CheckPrivilege(ctx context.Context, privilegeObject catalog.PrivilegeObject, privilege privilege.Kind) error

// CheckAnyPrivilege returns nil if user has any privileges at all.
CheckAnyPrivilege(ctx context.Context, descriptor catalog.Descriptor) error
CheckAnyPrivilege(ctx context.Context, descriptor catalog.PrivilegeObject) error

// UserHasAdminRole returns tuple of bool and error:
// (true, nil) means that the user has an admin role (i.e. root or node)
Expand Down Expand Up @@ -118,7 +117,7 @@ var _ AuthorizationAccessor = &planner{}
// Requires a valid transaction to be open.
func (p *planner) CheckPrivilegeForUser(
ctx context.Context,
descriptor catalog.Descriptor,
privilegeObject catalog.PrivilegeObject,
privilege privilege.Kind,
user username.SQLUsername,
) error {
Expand All @@ -134,17 +133,20 @@ func (p *planner) CheckPrivilegeForUser(
// it will not be forgotten if features are added that access
// descriptors (since every use of descriptors presumably need a
// permission check).
p.maybeAudit(descriptor, privilege)
p.maybeAudit(privilegeObject, privilege)

privs := descriptor.GetPrivileges()
privs, err := privilegeObject.GetPrivilegeDescriptor(ctx, p)
if err != nil {
return err
}

// Check if the 'public' pseudo-role has privileges.
if privs.CheckPrivilege(username.PublicRoleName(), privilege) {
return nil
}

hasPriv, err := p.checkRolePredicate(ctx, user, func(role username.SQLUsername) bool {
return IsOwner(descriptor, role) || privs.CheckPrivilege(role, privilege)
return IsOwner(privilegeObject, role) || privs.CheckPrivilege(role, privilege)
})
if err != nil {
return err
Expand All @@ -154,7 +156,7 @@ func (p *planner) CheckPrivilegeForUser(
}
return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s does not have %s privilege on %s %s",
user, privilege, descriptor.DescriptorType(), descriptor.GetName())
user, privilege, privilegeObject.GetObjectType(), privilegeObject.GetName())
}

// CheckPrivilege implements the AuthorizationAccessor interface.
Expand All @@ -163,9 +165,9 @@ func (p *planner) CheckPrivilegeForUser(
// it should be probably be called CheckPrivilegesOrOwnership and return
// a better error.
func (p *planner) CheckPrivilege(
ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind,
ctx context.Context, object catalog.PrivilegeObject, privilege privilege.Kind,
) error {
return p.CheckPrivilegeForUser(ctx, descriptor, privilege, p.User())
return p.CheckPrivilegeForUser(ctx, object, privilege, p.User())
}

// CheckGrantOptionsForUser calls PrivilegeDescriptor.CheckGrantOptions, which
Expand All @@ -174,7 +176,8 @@ func (p *planner) CheckPrivilege(
// options are inherited from parent roles.
func (p *planner) CheckGrantOptionsForUser(
ctx context.Context,
descriptor catalog.Descriptor,
privs *catpb.PrivilegeDescriptor,
privilegeObject catalog.PrivilegeObject,
privList privilege.List,
user username.SQLUsername,
isGrant bool,
Expand All @@ -187,9 +190,8 @@ func (p *planner) CheckGrantOptionsForUser(
return nil
}

privs := descriptor.GetPrivileges()
hasPriv, err := p.checkRolePredicate(ctx, user, func(role username.SQLUsername) bool {
return IsOwner(descriptor, role) || privs.CheckGrantOptions(role, privList)
return privs.CheckGrantOptions(role, privList) || IsOwner(privilegeObject, role)
})
if err != nil {
return err
Expand Down Expand Up @@ -231,9 +233,16 @@ func getOwnerOfDesc(desc catalog.Descriptor) username.SQLUsername {
return owner
}

// IsOwner returns if the role has ownership on the descriptor.
func IsOwner(desc catalog.Descriptor, role username.SQLUsername) bool {
return role == getOwnerOfDesc(desc)
// IsOwner returns if the role has ownership on the privilege object.
func IsOwner(privilegeObject catalog.PrivilegeObject, role username.SQLUsername) bool {
switch p := privilegeObject.(type) {
case catalog.Descriptor:
return role == getOwnerOfDesc(p)
case catalog.SystemPrivilegeObject:
return role.IsAdminRole()
default:
panic(errors.AssertionFailedf("unknown privilege object type %v", privilegeObject))
}
}

// HasOwnership returns if the role or any role the role is a member of
Expand Down Expand Up @@ -270,7 +279,9 @@ func (p *planner) checkRolePredicate(

// CheckAnyPrivilege implements the AuthorizationAccessor interface.
// Requires a valid transaction to be open.
func (p *planner) CheckAnyPrivilege(ctx context.Context, descriptor catalog.Descriptor) error {
func (p *planner) CheckAnyPrivilege(
ctx context.Context, privilegeObject catalog.PrivilegeObject,
) error {
// Verify that the txn is valid in any case, so that
// we don't get the risk to say "OK" to root requests
// with an invalid API usage.
Expand All @@ -285,7 +296,10 @@ func (p *planner) CheckAnyPrivilege(ctx context.Context, descriptor catalog.Desc
return nil
}

privs := descriptor.GetPrivileges()
privs, err := privilegeObject.GetPrivilegeDescriptor(ctx, p)
if err != nil {
return err
}

// Check if 'user' itself has privileges.
if privs.AnyPrivilege(user) {
Expand All @@ -312,7 +326,7 @@ func (p *planner) CheckAnyPrivilege(ctx context.Context, descriptor catalog.Desc

return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s has no privileges on %s %s",
p.SessionData().User(), descriptor.DescriptorType(), descriptor.GetName())
p.SessionData().User(), privilegeObject.GetObjectType(), privilegeObject.GetName())
}

// UserHasAdminRole implements the AuthorizationAccessor interface.
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,9 @@ go_library(
"descriptor_id_set.go",
"errors.go",
"post_derserialization_changes.go",
"privilege_object.go",
"schema.go",
"system_privilege.go",
"system_table.go",
"table_col_map.go",
"table_col_set.go",
Expand All @@ -25,13 +27,18 @@ go_library(
"//pkg/keys",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/security/username",
"//pkg/server/telemetry",
"//pkg/sql/catalog/catpb",
"//pkg/sql/catalog/descpb",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/privilege",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/catconstants",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/hlc",
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/catalog/catprivilege/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ go_library(
"//pkg/sql/catalog/descpb",
"//pkg/sql/privilege",
"//pkg/sql/sem/catconstants",
"//pkg/sql/systemprivilege",
"@com_github_cockroachdb_errors//:errors",
],
)

Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/catalog/catprivilege/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,13 @@
package catprivilege

import (
"reflect"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/systemprivilege"
"github.com/cockroachdb/errors"
)

// Validate validates a privilege descriptor.
Expand Down Expand Up @@ -52,3 +56,18 @@ func allowedSuperuserPrivileges(objectNameKey catalog.NameKey) privilege.List {
}
return catpb.DefaultSuperuserPrivileges
}

// ValidateSystemPrivilegeObject validates a SystemPrivilegeObject.
// TODO(richardjcai): Make sure the actual privilege descriptor is validated.
func ValidateSystemPrivilegeObject(systemPrivilegeObject catalog.SystemPrivilegeObject) error {
out, err := systemprivilege.Parse(systemPrivilegeObject.ToString())
if err != nil {
return err
}

if !reflect.DeepEqual(out, systemPrivilegeObject) {
return errors.Newf("system privilege object is invalid, expected %v, got %v", out, systemPrivilegeObject)
}

return nil
}
1 change: 1 addition & 0 deletions pkg/sql/catalog/dbdesc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/sql/catalog/multiregion",
"//pkg/sql/privilege",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/util/hlc",
"//pkg/util/iterutil",
Expand Down
14 changes: 14 additions & 0 deletions pkg/sql/catalog/dbdesc/database_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
package dbdesc

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/keys"
Expand All @@ -23,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
Expand Down Expand Up @@ -535,6 +537,18 @@ func (desc *Mutable) SetDeclarativeSchemaChangerState(state *scpb.DescriptorStat
desc.DeclarativeSchemaChangerState = state
}

// GetObjectType implements the PrivilegeObject interface.
func (desc *immutable) GetObjectType() string {
return string(desc.DescriptorType())
}

// GetPrivilegeDescriptor implements the PrivilegeObject interface.
func (desc *immutable) GetPrivilegeDescriptor(
ctx context.Context, planner eval.Planner,
) (*catpb.PrivilegeDescriptor, error) {
return desc.GetPrivileges(), nil
}

// maybeRemoveDroppedSelfEntryFromSchemas removes an entry in the Schemas map corresponding to the
// database itself which was added due to a bug in prior versions when dropping any user-defined schema.
// The bug inserted an entry for the database rather than the schema being dropped. This function fixes the
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,7 @@ type LeasableDescriptor interface {
type Descriptor interface {
NameEntry
LeasableDescriptor
PrivilegeObject

// GetPrivileges returns this descriptor's PrivilegeDescriptor, which
// describes the set of privileges that users have to use, modify, or delete
Expand Down
34 changes: 34 additions & 0 deletions pkg/sql/catalog/privilege_object.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// 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 catalog

import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
)

// PrivilegeObject represents an object that can have privileges. The privileges
// can either live on the descriptor or in the system.privileges table.
type PrivilegeObject interface {
// GetPrivilegeDescriptor returns the privilege descriptor for the
// object. Note that for non-descriptor backed objects, we query the
// system.privileges table to synthesize a PrivilegeDescriptor.
GetPrivilegeDescriptor(ctx context.Context, planner eval.Planner) (*catpb.PrivilegeDescriptor, error)
// GetObjectType returns the object type of the PrivilegeObject.
// For descriptor backed objects this is the "DescriptorType" for
// non-descriptor backed objects, this is a string constant.
GetObjectType() string
// GetName returns the name of the object. For example, the name of a
// table, schema or database.
GetName() string
}
1 change: 1 addition & 0 deletions pkg/sql/catalog/schemadesc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
"//pkg/sql/privilege",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/catconstants",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/util/hlc",
"//pkg/util/log",
Expand Down
Loading

0 comments on commit 5df8cbd

Please sign in to comment.