Skip to content

Commit

Permalink
Merge #110084
Browse files Browse the repository at this point in the history
110084: privilege: add REPAIRCLUSTERMETADATA system prvilege and remove admin checks r=rafiss a=rafiss

As we move away from requiring the admin role to perform cluster debug/repair operations, we want to use a privilege instead. To facilitate that, the admin role now implicitly has ALL global privileges. The privilege for admins is not revokeable.

Some exceptions were made, since the focus of this change is to use this
privilege for debug/repair endpoints. The following are still gated by
the admin role:
- ALTER ROLE on an admin
- ALTER DEFAULT PRIVILEGES FOR ALL ROLES
- changing a backup schedule metric

Release note (sql change): SQL debug/repair commands that were previously only usable
by users with the admin role can now be used by users with the
VIEWCLUSTERMETADATA or REPAIRCLUSTERMETADATA system privilege, depending
on whether the operation is read-only or modifies state.

informs #109814
Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
craig[bot] and rafiss committed Sep 21, 2023
2 parents ccc2bad + 307136e commit b4bd6d7
Show file tree
Hide file tree
Showing 40 changed files with 149 additions and 145 deletions.
7 changes: 5 additions & 2 deletions pkg/ccl/backupccl/alter_backup_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,8 +299,11 @@ func processScheduleOptions(
// NB: as of 20.2, schedule creation requires admin so this is duplicative
// but in the future we might relax so you can schedule anything that you
// can backup, but then this cluster-wide metric should be admin-only.
if err := p.RequireAdminRole(ctx, optUpdatesLastBackupMetric); err != nil {
return pgerror.Wrap(err, pgcode.InsufficientPrivilege, "")
if hasAdmin, err := p.HasAdminRole(ctx); err != nil {
return err
} else if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to change %s", optUpdatesLastBackupMetric)
}

updatesLastBackupMetric, err := strconv.ParseBool(v)
Expand Down
7 changes: 6 additions & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/exprutil"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -286,8 +288,11 @@ func doCreateBackupSchedules(
// NB: as of 20.2, schedule creation requires admin so this is duplicative
// but in the future we might relax so you can schedule anything that you
// can backup, but then this cluster-wide metric should be admin-only.
if err := p.RequireAdminRole(ctx, optUpdatesLastBackupMetric); err != nil {
if hasAdmin, err := p.HasAdminRole(ctx); err != nil {
return err
} else if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to change %s", optUpdatesLastBackupMetric)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ let $fullID $incID
with schedules as (show schedules) select id from schedules where label='datatest3' order by command->>'backup_type' asc;
----

exec-sql user=testuser expect-error-regex=(only users with the admin role are allowed to updates_cluster_last_backup_time_metric)
exec-sql user=testuser expect-error-regex=(only users with the admin role are allowed to change updates_cluster_last_backup_time_metric)
alter backup schedule $fullID set schedule option updates_cluster_last_backup_time_metric = '1';
----
regex matches error
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ subtest node_tenant_capabilities_cache

user testuser

statement error only users with the admin role are allowed to node_tenant_capabilities_cache
statement error user testuser does not have VIEWCLUSTERMETADATA system privilege
SELECT * FROM crdb_internal.node_tenant_capabilities_cache

user root
Expand Down
16 changes: 8 additions & 8 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -461,28 +461,28 @@ select crdb_internal.set_vmodule('')
query error insufficient privilege
select crdb_internal.get_vmodule()

query error pq: only users with the admin role are allowed to access the node runtime information
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.node_runtime_info

query error pq: only users with the VIEWACTIVITY or VIEWACTIVITYREDACTED or ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases
select * from crdb_internal.ranges

query error pq: only users with the admin role are allowed to read crdb_internal.gossip_nodes
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.gossip_nodes

query error pq: only users with the admin role are allowed to read crdb_internal.gossip_liveness
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.gossip_liveness

query error pq: only users with the admin role are allowed to read crdb_internal.node_metrics
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.node_metrics

query error pq: only users with the admin role are allowed to read crdb_internal.kv_node_status
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.kv_node_status

query error pq: only users with the admin role are allowed to read crdb_internal.kv_store_status
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.kv_store_status

query error pq: only users with the admin role are allowed to read crdb_internal.gossip_alerts
query error user testuser does not have VIEWCLUSTERMETADATA system privilege
select * from crdb_internal.gossip_alerts

# Anyone can see the executable version.
Expand Down Expand Up @@ -638,7 +638,7 @@ subtest node_tenant_capabilities_cache

user testuser

statement error only users with the admin role are allowed to node_tenant_capabilities_cache
statement error user testuser does not have VIEWCLUSTERMETADATA system privilege
SELECT * FROM crdb_internal.node_tenant_capabilities_cache

user root
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/tenant_usage
Original file line number Diff line number Diff line change
Expand Up @@ -34,5 +34,5 @@ SELECT crdb_internal.update_tenant_resource_limits('apptenant', 1000, 100, 0, no

user testuser

statement error only users with the admin role are allowed to update tenant resource limits
statement error crdb_internal.update_tenant_resource_limits\(\): user testuser does not have REPAIRCLUSTERMETADATA system privilege
SELECT crdb_internal.update_tenant_resource_limits(5, 1000, 100, 0, now(), 0)
4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/zone
Original file line number Diff line number Diff line change
Expand Up @@ -786,13 +786,13 @@ CREATE INDEX x ON auth.t (x) PARTITION BY LIST (x) (
user testuser

# User should have no CONFIGURE ZONE abilities by default.
statement error only users with the admin role are allowed to alter system ranges
statement error user testuser does not have REPAIRCLUSTERMETADATA system privilege
ALTER RANGE default CONFIGURE ZONE USING num_replicas = 3

statement error pq: user testuser does not have ZONECONFIG or CREATE privilege on database auth
ALTER DATABASE auth CONFIGURE ZONE USING num_replicas = 3

statement error pq: only users with the admin role are allowed to alter system tables
statement error user testuser does not have REPAIRCLUSTERMETADATA system privilege
ALTER TABLE system.jobs CONFIGURE ZONE USING num_replicas = 3

statement error pq: user testuser does not have ZONECONFIG or CREATE privilege on relation t
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/privchecker/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ type CheckerForRPCHandlers interface {
// Its error return is a gRPC error.
RequireAdminUser(ctx context.Context) (userName username.SQLUsername, err error)

// RequireAdminRole validates the current user has the VIEWACTIVITY
// RequireViewActivityPermission validates the current user has the VIEWACTIVITY
// privilege or role option.
// Its error return is a gRPC error.
RequireViewActivityPermission(ctx context.Context) error
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/alter_default_privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,8 +140,11 @@ func (n *alterDefaultPrivilegesNode) startExec(params runParams) error {
}

if n.n.ForAllRoles {
if err := params.p.RequireAdminRole(params.ctx, "ALTER DEFAULT PRIVILEGES"); err != nil {
if hasAdmin, err := params.p.HasAdminRole(params.ctx); err != nil {
return err
} else if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to ALTER DEFAULT PRIVILEGES FOR ALL ROLES")
}
} else {
// You can change default privileges only for objects that will be created
Expand Down
23 changes: 20 additions & 3 deletions pkg/sql/alter_role.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessioninit"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
)
Expand Down Expand Up @@ -215,8 +216,11 @@ func (n *alterRoleNode) startExec(params runParams) error {
return err
}
if isAdmin {
if err := params.p.RequireAdminRole(params.ctx, "ALTER ROLE admin"); err != nil {
if hasAdmin, err := params.p.HasAdminRole(params.ctx); err != nil {
return err
} else if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to alter another admin")
}
}

Expand Down Expand Up @@ -287,8 +291,11 @@ func (p *planner) AlterRoleSet(ctx context.Context, n *tree.AlterRoleSet) (planN
// modifying their own defaults unless they have CREATEROLE. This is analogous
// to our restriction that prevents a user from modifying their own password.
if n.AllRoles {
if err := p.RequireAdminRole(ctx, "ALTER ROLE ALL"); err != nil {
if hasAdmin, err := p.HasAdminRole(ctx); err != nil {
return nil, err
} else if !hasAdmin {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to ALTER ROLE ALL ... SET")
}
} else {
canAlterRoleSet, err := p.HasGlobalPrivilegeOrRoleOption(ctx, privilege.CREATEROLE)
Expand Down Expand Up @@ -607,7 +614,17 @@ func (n *alterRoleSetNode) getRoleName(
return false, username.SQLUsername{}, err
}
if isAdmin {
if err := params.p.RequireAdminRole(params.ctx, "ALTER ROLE admin"); err != nil {
if hasAdmin, err := params.p.HasAdminRole(params.ctx); err != nil {
return false, username.SQLUsername{}, err
} else if !hasAdmin {
return false, username.SQLUsername{}, pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to alter another admin")
}

// Note that admins implicitly have the REPAIRCLUSTERMETADATA privilege.
if err := params.p.CheckPrivilege(
params.ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.REPAIRCLUSTERMETADATA,
); err != nil {
return false, username.SQLUsername{}, err
}
}
Expand Down
21 changes: 0 additions & 21 deletions pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,11 +107,6 @@ type AuthorizationAccessor interface {
// HasAdminRole checks if the current session's user has admin role.
HasAdminRole(ctx context.Context) (bool, error)

// RequireAdminRole is a wrapper on top of HasAdminRole.
// It errors if HasAdminRole errors or if the user isn't a super-user.
// Includes the named action in the error message.
RequireAdminRole(ctx context.Context, action string) error

// MemberOfWithAdminOption looks up all the roles (direct and indirect) that 'member' is a member
// of and returns a map of role -> isAdmin.
MemberOfWithAdminOption(ctx context.Context, member username.SQLUsername) (map[username.SQLUsername]bool, error)
Expand Down Expand Up @@ -508,22 +503,6 @@ func (p *planner) HasAdminRole(ctx context.Context) (bool, error) {
return p.UserHasAdminRole(ctx, p.User())
}

// RequireAdminRole implements the AuthorizationAccessor interface.
// Requires a valid transaction to be open.
func (p *planner) RequireAdminRole(ctx context.Context, action string) error {
ok, err := p.HasAdminRole(ctx)

if err != nil {
return err
}
if !ok {
// raise error if user is not a super-user
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to %s", action)
}
return nil
}

// MemberOfWithAdminOption is a wrapper around the MemberOfWithAdminOption
// method.
func (p *planner) MemberOfWithAdminOption(
Expand Down
7 changes: 6 additions & 1 deletion pkg/sql/copy_file_upload.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -108,8 +110,11 @@ func newFileUploadMachine(
c.parsingEvalCtx = c.p.EvalContext()

if n.Table.Table() == NodelocalFileUploadTable {
if err := c.p.RequireAdminRole(ctx, "upload to nodelocal"); err != nil {
if hasAdmin, err := p.HasAdminRole(ctx); err != nil {
return nil, err
} else if !hasAdmin {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to upload to nodelocal")
}
}

Expand Down
31 changes: 15 additions & 16 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/sql/vtable"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
Expand Down Expand Up @@ -265,7 +266,7 @@ CREATE TABLE crdb_internal.node_runtime_info (
value STRING NOT NULL
)`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "access the node runtime information"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -2122,7 +2123,7 @@ CREATE TABLE crdb_internal.node_txn_stats (
implicit_count INT NOT NULL
)`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "access application statistics"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -2275,14 +2276,12 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
operation STRING NULL -- The span's operation.
)`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
hasAdmin, err := p.HasAdminRole(ctx)
if err != nil {
if err := p.CheckPrivilege(
ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA,
); err != nil {
return err
}
if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to read crdb_internal.node_inflight_trace_spans")
}

return p.ExecCfg().AmbientCtx.Tracer.VisitSpans(func(span tracing.RegistrySpan) error {
trace := span.GetFullRecording(tracingpb.RecordingVerbose)
for _, rec := range trace.Flatten() {
Expand Down Expand Up @@ -3269,7 +3268,7 @@ var crdbInternalLocalMetricsTable = virtualSchemaTable{
value FLOAT NOT NULL -- value of the metric
)`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "read crdb_internal.node_metrics"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -4853,7 +4852,7 @@ CREATE TABLE crdb_internal.gossip_nodes (
)
`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "read crdb_internal.gossip_nodes"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -4972,7 +4971,7 @@ CREATE TABLE crdb_internal.kv_node_liveness (
)
`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "read crdb_internal.node_liveness"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -5038,7 +5037,7 @@ CREATE TABLE crdb_internal.gossip_liveness (
// which is highly available. DO NOT CALL functions which require the
// cluster to be healthy, such as NodesStatusServer.ListNodesInternal().

if err := p.RequireAdminRole(ctx, "read crdb_internal.gossip_liveness"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -5114,7 +5113,7 @@ CREATE TABLE crdb_internal.gossip_alerts (
)
`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "read crdb_internal.gossip_alerts"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}

Expand Down Expand Up @@ -5435,7 +5434,7 @@ CREATE TABLE crdb_internal.kv_node_status (
)
`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "read crdb_internal.kv_node_status"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}
ss, err := p.extendedEvalCtx.NodesStatusServer.OptionalNodesStatusServer(
Expand Down Expand Up @@ -5549,7 +5548,7 @@ CREATE TABLE crdb_internal.kv_store_status (
)
`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
if err := p.RequireAdminRole(ctx, "read crdb_internal.kv_store_status"); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}
ss, err := p.ExecCfg().NodesStatusServer.OptionalNodesStatusServer(
Expand Down Expand Up @@ -8457,7 +8456,7 @@ CREATE TABLE crdb_internal.node_tenant_capabilities_cache (
);`,
populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
const op = "node_tenant_capabilities_cache"
if err := p.RequireAdminRole(ctx, op); err != nil {
if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return err
}
tenantCapabilitiesReader, err := p.ExecCfg().TenantCapabilitiesReader.Get(op)
Expand Down
5 changes: 1 addition & 4 deletions pkg/sql/delegate/show_all_cluster_settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,10 +95,7 @@ func (d *delegator) delegateShowTenantClusterSettingList(
// privileged operation than viewing local cluster settings. So we
// shouldn't be allowing with just the role option
// VIEWCLUSTERSETTINGS.
//
// TODO(knz): Using admin authz for now; we may want to introduce a
// more specific role option later.
if err := d.catalog.RequireAdminRole(d.ctx, "show a tenant cluster setting"); err != nil {
if err := d.catalog.CheckPrivilege(d.ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil {
return nil, err
}

Expand Down
Loading

0 comments on commit b4bd6d7

Please sign in to comment.