Skip to content

Commit

Permalink
sql: remove usages of UserHasAdminRole
Browse files Browse the repository at this point in the history
This commit covers a few cases that were missed by an earlier commit:
#110084. No release note is
included since it would be redundant with the release note from that PR.

Release note: None
  • Loading branch information
rafiss authored and yuzefovich committed Sep 25, 2023
1 parent f4197e7 commit 548cd02
Show file tree
Hide file tree
Showing 8 changed files with 39 additions and 37 deletions.
12 changes: 7 additions & 5 deletions pkg/ccl/backupccl/alter_backup_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"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/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/errors"
pbtypes "github.com/gogo/protobuf/types"
)
Expand Down Expand Up @@ -127,16 +129,16 @@ func doAlterBackupSchedules(
s.incJob.ScheduleID())
}

// Check that the user is admin or the owner of the schedules being altered.
isAdmin, err := p.UserHasAdminRole(ctx, p.User())
// Check that the user has privileges or is the owner of the schedules being altered.
hasPriv, err := p.HasPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.REPAIRCLUSTERMETADATA, p.User())
if err != nil {
return err
}
isOwnerOfFullJob := s.fullJob == nil || s.fullJob.Owner() == p.User()
isOwnerOfIncJob := s.incJob == nil || s.incJob.Owner() == p.User()
if !isAdmin && !(isOwnerOfFullJob && isOwnerOfIncJob) {
return pgerror.New(pgcode.InsufficientPrivilege, "must be admin or owner of the "+
"schedules being altered.")
if !hasPriv && !(isOwnerOfFullJob && isOwnerOfIncJob) {
return pgerror.Newf(pgcode.InsufficientPrivilege, "must be admin or the owner of the "+
"schedules being altered, or have %s privilege", privilege.REPAIRCLUSTERMETADATA)
}

if s, err = processFullBackupRecurrence(
Expand Down
10 changes: 5 additions & 5 deletions pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ foocluster_admin BACKUP INTO LATEST IN 'external://foo/cluster' WITH OPTIONS (de
foocluster_admin BACKUP INTO 'external://foo/cluster' WITH OPTIONS (detached)

# nonadmin testuser is not allowed to drop a schedule they do not own.
exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser
exec-sql expect-error-regex=(must have REPAIRCLUSTERMETADATA privilege or be owner of the schedule [0-9]+ to DROP it) user=testuser
DROP SCHEDULE $fullID
----
regex matches error
Expand Down Expand Up @@ -141,7 +141,7 @@ let $otherFullID $otherIncID
with schedules as (show schedules) select id from schedules where label='foocluster_admin_revoke' order by command->>'backup_type' asc;
----

exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser
exec-sql expect-error-regex=(must have REPAIRCLUSTERMETADATA privilege or be owner of the schedule [0-9]+ to DROP it) user=testuser
DROP SCHEDULE $otherFullID
----
regex matches error
Expand Down Expand Up @@ -180,17 +180,17 @@ DROP SCHEDULE $testuserIncID;
----

# But testuser can't drop, alter, resume or pause the root owned schedules.
exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to PAUSE it) user=testuser
exec-sql expect-error-regex=(must have REPAIRCLUSTERMETADATA privilege or be owner of the schedule [0-9]+ to PAUSE it) user=testuser
PAUSE SCHEDULE $otherFullID
----
regex matches error

exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to RESUME it) user=testuser
exec-sql expect-error-regex=(must have REPAIRCLUSTERMETADATA privilege or be owner of the schedule [0-9]+ to RESUME it) user=testuser
RESUME SCHEDULE $otherFullID
----
regex matches error

exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser
exec-sql expect-error-regex=(must have REPAIRCLUSTERMETADATA privilege or be owner of the schedule [0-9]+ to DROP it) user=testuser
DROP SCHEDULE $otherFullID;
----
regex matches error
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,12 +80,12 @@ GRANT SYSTEM EXTERNALCONNECTION TO testuser
query-sql user=testuser
SHOW CREATE ALL EXTERNAL CONNECTIONS
----
pq: must be admin to run `SHOW CREATE ALL EXTERNAL CONNECTIONS
pq: must have VIEWCLUSTERMETADATA privilege to run `SHOW CREATE ALL EXTERNAL CONNECTIONS`

query-sql user=testuser
SHOW CREATE EXTERNAL CONNECTION foo
----
pq: must be admin or owner of the External Connection "foo"
pq: must have VIEWCLUSTERMETADATA privilege or be owner of the External Connection "foo"

# Create External Connection where testuser is the owner, they should be able to SHOW this object.
exec-sql user=testuser
Expand All @@ -95,7 +95,7 @@ CREATE EXTERNAL CONNECTION bar AS 'nodelocal://1/foo'
query-sql user=testuser
SHOW CREATE ALL EXTERNAL CONNECTIONS
----
pq: must be admin to run `SHOW CREATE ALL EXTERNAL CONNECTIONS
pq: must have VIEWCLUSTERMETADATA privilege to run `SHOW CREATE ALL EXTERNAL CONNECTIONS`

# TODO(aditymaru): Synthetic privileges do not have a concept of owners. Once they do, testuser will
# be able to run this query successfully since they are the owner of the External Connection object.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,12 +76,12 @@ GRANT SYSTEM EXTERNALCONNECTION TO testuser
query-sql user=testuser
SHOW CREATE ALL EXTERNAL CONNECTIONS
----
pq: must be admin to run `SHOW CREATE ALL EXTERNAL CONNECTIONS
pq: must have VIEWCLUSTERMETADATA privilege to run `SHOW CREATE ALL EXTERNAL CONNECTIONS`

query-sql user=testuser
SHOW CREATE EXTERNAL CONNECTION foo
----
pq: must be admin or owner of the External Connection "foo"
pq: must have VIEWCLUSTERMETADATA privilege or be owner of the External Connection "foo"

# Create External Connection where testuser is the owner, they should be able to SHOW this object.
exec-sql user=testuser
Expand All @@ -91,7 +91,7 @@ CREATE EXTERNAL CONNECTION bar AS 'nodelocal://1/foo'
query-sql user=testuser
SHOW CREATE ALL EXTERNAL CONNECTIONS
----
pq: must be admin to run `SHOW CREATE ALL EXTERNAL CONNECTIONS
pq: must have VIEWCLUSTERMETADATA privilege to run `SHOW CREATE ALL EXTERNAL CONNECTIONS`

# TODO(aditymaru): Synthetic privileges do not have a concept of owners. Once they do, testuser will
# be able to run this query successfully since they are the owner of the External Connection object.
Expand Down
13 changes: 9 additions & 4 deletions pkg/sql/control_schedules.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"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/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -129,14 +131,17 @@ func (n *controlSchedulesNode) startExec(params runParams) error {
continue // not an error if schedule does not exist
}

isAdmin, err := params.p.UserHasAdminRole(params.ctx, params.p.User())
// Check that the user has privileges or is the owner of the schedules being altered.
hasPriv, err := params.p.HasPrivilege(
params.ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.REPAIRCLUSTERMETADATA, params.p.User(),
)
if err != nil {
return err
}
isOwner := schedule.Owner() == params.p.User()
if !isAdmin && !isOwner {
return pgerror.Newf(pgcode.InsufficientPrivilege, "must be admin or owner of the "+
"schedule %d to %s it", schedule.ScheduleID(), n.command.String())
if !hasPriv && !isOwner {
return pgerror.Newf(pgcode.InsufficientPrivilege, "must have %s privilege or be owner of the "+
"schedule %d to %s it", privilege.REPAIRCLUSTERMETADATA, schedule.ScheduleID(), n.command.String())
}

switch n.command {
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/repair.go
Original file line number Diff line number Diff line change
Expand Up @@ -712,13 +712,9 @@ func checkPlannerStateForRepairFunctions(ctx context.Context, p *planner, method
if p.extendedEvalCtx.TxnReadOnly {
return readOnlyError(method)
}
hasAdmin, err := p.UserHasAdminRole(ctx, p.User())
if err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.REPAIRCLUSTERMETADATA); err != nil {
return err
}
if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege, "admin role required for %s", method)
}
return nil
}

Expand Down
12 changes: 7 additions & 5 deletions pkg/sql/show_create_external_connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"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/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
Expand Down Expand Up @@ -74,12 +75,13 @@ func (p *planner) ShowCreateExternalConnection(
) (planNode, error) {
var hasPrivileges bool
var err error
if hasPrivileges, err = p.UserHasAdminRole(ctx, p.User()); err != nil {
if hasPrivileges, err = p.HasPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA, p.User()); err != nil {
return nil, err
}

// If the user is not admin, and is running a `SHOW CREATE EXTERNAL CONNECTION foo`
// check if the user is the owner of the object.
// If the user does not have VIEWCLUSTERMETADATA, and is running a `SHOW
// CREATE EXTERNAL CONNECTION foo` check if the user is the owner of the
// object.
exprEval := p.ExprEvaluator(externalConnectionOp)
if !hasPrivileges && n.ConnectionLabel != nil {
name, err := exprEval.String(ctx, n.ConnectionLabel)
Expand All @@ -94,10 +96,10 @@ func (p *planner) ShowCreateExternalConnection(
return nil, err
}
if !isOwner {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "must be admin or owner of the External Connection %q", name)
return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "must have %s privilege or be owner of the External Connection %q", privilege.VIEWCLUSTERMETADATA, name)
}
} else if !hasPrivileges {
return nil, pgerror.New(pgcode.InsufficientPrivilege, "must be admin to run `SHOW CREATE ALL EXTERNAL CONNECTIONS")
return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "must have %s privilege to run `SHOW CREATE ALL EXTERNAL CONNECTIONS`", privilege.VIEWCLUSTERMETADATA)
}

sqltelemetry.IncrementShowCounter(sqltelemetry.CreateExternalConnection)
Expand Down
11 changes: 4 additions & 7 deletions pkg/sql/show_create_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"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/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

Expand Down Expand Up @@ -85,12 +85,9 @@ func loadSchedules(params runParams, n *tree.ShowCreateSchedules) ([]*jobs.Sched
func (p *planner) ShowCreateSchedule(
ctx context.Context, n *tree.ShowCreateSchedules,
) (planNode, error) {
// Only admin users can execute SHOW CREATE SCHEDULE
if userIsAdmin, err := p.UserHasAdminRole(ctx, p.User()); err != nil {
// Only privileged users can execute SHOW CREATE SCHEDULE
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return nil, err
} else if !userIsAdmin {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s does not have admin role", p.User())
}

sqltelemetry.IncrementShowCounter(sqltelemetry.CreateSchedule)
Expand Down

0 comments on commit 548cd02

Please sign in to comment.