Skip to content

Commit

Permalink
Merge #110220
Browse files Browse the repository at this point in the history
110220: sql: add CREATELOGIN, CREATEDB, and CONTROLJOB system privileges r=rafiss a=andyyang890

Fixes #103237

Co-authored-by: Andy Yang <[email protected]>
  • Loading branch information
craig[bot] and andyyang890 committed Sep 8, 2023
2 parents 397dc0b + 81eb17e commit 4922a93
Show file tree
Hide file tree
Showing 13 changed files with 280 additions and 24 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,6 @@ go_library(
"//pkg/sql/physicalplan",
"//pkg/sql/privilege",
"//pkg/sql/protoreflect",
"//pkg/sql/roleoption",
"//pkg/sql/rowenc",
"//pkg/sql/rowexec",
"//pkg/sql/schemachanger/scbackup",
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -1556,7 +1555,7 @@ func checkPrivilegesForRestore(
"RESTORE system privilege.", deprecatedPrivilegesRestorePreamble, p.User().Normalized())
p.BufferClientNotice(ctx, pgnotice.Newf("%s", notice))

hasCreateDB, err := p.HasRoleOption(ctx, roleoption.CREATEDB)
hasCreateDB, err := p.HasGlobalPrivilegeOrRoleOption(ctx, privilege.CREATEDB)
if err != nil {
return err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/jobs/jobsauth/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ go_test(
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/privilege",
"//pkg/sql/roleoption",
"//pkg/sql/syntheticprivilege",
"//pkg/util/randutil",
"@com_github_stretchr_testify//assert",
],
Expand Down
23 changes: 14 additions & 9 deletions pkg/jobs/jobsauth/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,9 @@ type AuthorizationAccessor interface {
// HasAdminRole mirrors sql.AuthorizationAccessor.
HasAdminRole(ctx context.Context) (bool, error)

// HasGlobalPrivilegeOrRoleOption mirrors sql.AuthorizationAccessor.
HasGlobalPrivilegeOrRoleOption(ctx context.Context, privilege privilege.Kind) (bool, error)

// User mirrors sql.PlanHookState.
User() username.SQLUsername
}
Expand All @@ -81,7 +84,7 @@ type AuthorizationAccessor interface {
// TODO(#96432): sort out internal job owners and rules for accessing them
// Authorize checks these rules in order:
// 1. If the user is an admin, grant access.
// 2. If the AccessLevel is ViewAccess, grant access if the user has CONTROLJOB
// 2. If the AccessLevel is ViewAccess, grant access if the user has CONTROLJOB, VIEWJOB,
// or if the user owns the job.
// 3. If the AccessLevel is ControlAccess, grant access if the user has CONTROLJOB
// and the job owner is not an admin.
Expand All @@ -101,20 +104,22 @@ func Authorize(
return nil
}

hasControlJob, err := a.HasRoleOption(ctx, roleoption.CONTROLJOB)
if err != nil {
return err
}

hasViewJob, err := a.HasPrivilege(ctx, &syntheticprivilege.GlobalPrivilege{}, privilege.VIEWJOB, a.User())
hasControlJob, err := a.HasGlobalPrivilegeOrRoleOption(ctx, privilege.CONTROLJOB)
if err != nil {
return err
}

jobOwnerUser := payload.UsernameProto.Decode()

if accessLevel == ViewAccess {
if a.User() == jobOwnerUser || hasControlJob || hasViewJob {
if a.User() == jobOwnerUser || hasControlJob {
return nil
}
hasViewJob, err := a.HasPrivilege(ctx, &syntheticprivilege.GlobalPrivilege{}, privilege.VIEWJOB, a.User())
if err != nil {
return err
}
if hasViewJob {
return nil
}
}
Expand All @@ -136,6 +141,6 @@ func Authorize(
return check(ctx, a, jobID, payload)
}
return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s does not have privileges for job $d",
"user %s does not have privileges for job %d",
a.User(), jobID)
}
44 changes: 42 additions & 2 deletions pkg/jobs/jobsauth/authorization_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/assert"
)
Expand All @@ -37,6 +38,8 @@ type userPrivilege struct {

var viewJobGlobalPrivilege = userPrivilege{privilege.VIEWJOB, privilege.Global}

var controlJobGlobalPrivilege = userPrivilege{privilege.CONTROLJOB, privilege.Global}

type testAuthAccessor struct {
user username.SQLUsername

Expand Down Expand Up @@ -112,6 +115,22 @@ func (a *testAuthAccessor) HasAdminRole(ctx context.Context) (bool, error) {
return ok, nil
}

func (a *testAuthAccessor) HasGlobalPrivilegeOrRoleOption(
ctx context.Context, privilege privilege.Kind,
) (bool, error) {
ok, err := a.HasPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege, a.User())
if err != nil {
return false, err
}
if ok {
return true, nil
}
if roleOption, ok := roleoption.ByName[privilege.String()]; ok {
return a.HasRoleOption(ctx, roleOption)
}
return false, nil
}

func (a *testAuthAccessor) User() username.SQLUsername {
return a.user
}
Expand Down Expand Up @@ -157,7 +176,7 @@ func TestAuthorization(t *testing.T) {
userErr error
}{
{
name: "controljob-sufficient-for-non-admin-jobs",
name: "controljob-role-option-sufficient-for-non-admin-jobs",

user: username.MakeSQLUsernameFromPreNormalizedString("user1"),
roleOptions: map[roleoption.Option]struct{}{
Expand All @@ -168,7 +187,7 @@ func TestAuthorization(t *testing.T) {
accessLevel: jobsauth.ControlAccess,
},
{
name: "controljob-sufficient-to-view-admin-jobs",
name: "controljob-role-option-sufficient-to-view-admin-jobs",
user: username.MakeSQLUsernameFromPreNormalizedString("user1"),
roleOptions: map[roleoption.Option]struct{}{
roleoption.CONTROLJOB: {},
Expand Down Expand Up @@ -262,6 +281,27 @@ func TestAuthorization(t *testing.T) {
accessLevel: jobsauth.ControlAccess,
userErr: pgerror.New(pgcode.InsufficientPrivilege, "foo"),
},
{
name: "controljob-system-privilege-sufficient-for-non-admin-jobs",

user: username.MakeSQLUsernameFromPreNormalizedString("user1"),
userPrivileges: map[userPrivilege]struct{}{
controlJobGlobalPrivilege: {},
},
admins: map[string]struct{}{},
payload: makeBackupPayload("user2"),
accessLevel: jobsauth.ControlAccess,
},
{
name: "controljob-system-privilege-sufficient-to-view-admin-jobs",
user: username.MakeSQLUsernameFromPreNormalizedString("user1"),
userPrivileges: map[userPrivilege]struct{}{
controlJobGlobalPrivilege: {},
},
admins: map[string]struct{}{"user2": {}},
payload: makeBackupPayload("user2"),
accessLevel: jobsauth.ViewAccess,
},
} {
t.Run(tc.name, func(t *testing.T) {
testAuth := &testAuthAccessor{
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/alter_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
Expand Down Expand Up @@ -84,7 +83,7 @@ func (n *alterDatabaseOwnerNode) startExec(params runParams) error {
}

// To alter the owner, the user also has to have CREATEDB privilege.
if err := params.p.CheckRoleOption(params.ctx, roleoption.CREATEDB); err != nil {
if err := params.p.CheckGlobalPrivilegeOrRoleOption(params.ctx, privilege.CREATEDB); err != nil {
return err
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_role.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ func (p *planner) checkPasswordOptionConstraints(
// NOCREATELOGIN to another role, or set up a password for
// authentication, or set up password validity, or enable/disable
// LOGIN privilege; even if they have CREATEROLE privilege.
if err := p.CheckRoleOption(ctx, roleoption.CREATELOGIN); err != nil {
if err := p.CheckGlobalPrivilegeOrRoleOption(ctx, privilege.CREATELOGIN); err != nil {
return err
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/create_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
Expand Down Expand Up @@ -148,7 +148,7 @@ func (p *planner) CreateDatabase(ctx context.Context, n *tree.CreateDatabase) (p
// CanCreateDatabase verifies that the current user has the CREATEDB
// role option.
func (p *planner) CanCreateDatabase(ctx context.Context) error {
hasCreateDB, err := p.HasRoleOption(ctx, roleoption.CREATEDB)
hasCreateDB, err := p.HasGlobalPrivilegeOrRoleOption(ctx, privilege.CREATEDB)
if err != nil {
return err
}
Expand Down
74 changes: 74 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/jobs
Original file line number Diff line number Diff line change
Expand Up @@ -280,3 +280,77 @@ query I
SELECT count(*) FROM [SHOW AUTOMATIC JOBS] WHERE job_type = 'AUTO CONFIG RUNNER' AND status = 'running'
----
1

subtest control_job_priv

user testuser

statement ok
CREATE TABLE t_control_job_priv(x INT);
DROP TABLE t_control_job_priv

let $job_id
SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser' AND job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for DROP TABLE test.public.t_control_job_priv'

statement error user testuser does not have privileges for job
PAUSE JOB (SELECT $job_id)

user root

statement ok
GRANT SYSTEM CONTROLJOB TO testuser

user testuser

statement ok
PAUSE JOB (SELECT $job_id)

user root

statement ok
REVOKE SYSTEM CONTROLJOB FROM testuser

subtest end

subtest control_job_priv_inherited

user testuser

statement ok
CREATE TABLE t_control_job_priv_inherited(x INT);
DROP TABLE t_control_job_priv_inherited

let $job_id
SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser' AND job_type = 'SCHEMA CHANGE GC' AND description LIKE 'GC for DROP TABLE test.public.t_control_job_priv_inherited'

statement error user testuser does not have privileges for job
PAUSE JOB (SELECT $job_id)

user root

statement ok
CREATE ROLE jobcontroller

statement ok
GRANT SYSTEM CONTROLJOB TO jobcontroller

statement ok
GRANT jobcontroller TO testuser

user testuser

statement ok
PAUSE JOB (SELECT $job_id)

user root

statement ok
REVOKE SYSTEM CONTROLJOB FROM jobcontroller

statement ok
REVOKE jobcontroller FROM testuser

statement ok
DROP ROLE jobcontroller

subtest end
Loading

0 comments on commit 4922a93

Please sign in to comment.