Skip to content

Commit

Permalink
sql: ensure user has correct privileges when adding/removing regions
Browse files Browse the repository at this point in the history
Previously we did not account for privileges on database objects when
adding the default locality ocnfig on first region add or removing the
locality config on last region drop properly. In particular, we weren't
adding/removing the locality config on any descriptor that wasn't
visible to the user. This is bad because our validation logic expects
only and all objects in multi-region databases to have a valid locality
config. This means future accesses to such descriptors would fail
validation.

The root of this problem was the API choice here, `ForEachTableDesc`,
which filters out invisible descriptors. This patch instead switches
to using `forEachTableInMultiRegionDatabase`. While here, instead of
issuing separate requests for every table, I refactored this thing to
issue a single batch request instead.

Now that we view all the descriptors inside the database, unfiltered,
we perform privilege checks on them before proceeding with the add/drop
operation. In particular, the semantics are:
- admin users are allowed to add/drop regions as they wish.
- non admin-users require the CREATE privilege or must have ownership
on all the objects inside the database.

Closes cockroachdb#61003

Release note (sql change): `ALTER DATABASE .. SET PRIMARY REGION` now
requires both CREATE and ZONECONFIG privilege on all  objects inside
the database when adding the first region to the database. Same for
dropping the last region using `ALTER DATABASE ... DROP REGION`.
  • Loading branch information
arulajmani committed Mar 19, 2021
1 parent f810c6c commit 0ea0538
Show file tree
Hide file tree
Showing 5 changed files with 271 additions and 216 deletions.
61 changes: 61 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/multi_region_privileges
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
# LogicTest: multiregion-9node-3region-3azs

user root

statement ok
CREATE DATABASE db;
CREATE TABLE db.t();
GRANT CREATE ON DATABASE db TO testuser;
CREATE TABLE db.t2();
ALTER USER testuser CREATEDB;

user testuser

statement error user testuser must be owner of t or have CREATE privilege on t
ALTER DATABASE db SET PRIMARY REGION "us-east-1"

user root

statement ok
GRANT CREATE ON TABLE db.t TO testuser

user testuser

statement ok
ALTER DATABASE db SET PRIMARY REGION "us-east-1"

user root

statement ok
REVOKE CREATE ON TABLE db.t FROM testuser

user testuser

statement error user testuser must be owner of t or have CREATE privilege on t
ALTER DATABASE db DROP REGION "us-east-1"

user root

statement ok
GRANT CREATE ON TABLE db.t TO testuser

user testuser

statement ok
ALTER DATABASE db DROP REGION "us-east-1"

# Same thing, but this time testuser is the owner of the table (and doesn't have
# CREATE privileges on it).
user root

statement ok
REVOKE CREATE ON TABLE db.t FROM testuser;
ALTER TABLE db.t OWNER TO testuser

user testuser

statement ok
ALTER DATABASE db SET PRIMARY REGION "us-east-1"

statement ok
ALTER DATABASE db DROP REGION "us-east-1"
158 changes: 92 additions & 66 deletions pkg/sql/alter_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"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/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
Expand Down Expand Up @@ -358,6 +360,38 @@ func (p *planner) AlterDatabaseDropRegion(
}, nil
}

// ensureCorrectMultiRegionPrivilegesForTable ensures the current user has the
// required privileges to alter the locality configuration of the given table
// descriptor. This entails that the user must:
// - either be part of an admin role.
// - or be an owner of the table.
// - or have the CREATE privilege on the table.
// privilege on the table descriptor.
func (p *planner) ensureCorrectMultiRegionPrivilegesForTable(
ctx context.Context, tableDesc catalog.TableDescriptor,
) error {
hasAdminRole, err := p.HasAdminRole(ctx)
if err != nil {
return err
}
if !hasAdminRole {
err := p.CheckPrivilege(ctx, tableDesc, privilege.CREATE)
// Wrap an insufficient privileges error a bit better to reflect the lack
// of ownership as well.
if pgerror.GetPGCode(err) == pgcode.InsufficientPrivilege {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s must be owner of %s or have %s privilege on %s",
p.SessionData().User(),
tableDesc.GetName(),
privilege.CREATE,
tableDesc.GetName(),
)
}
return err
}
return nil
}

// removeLocalityConfigFromAllTablesInDB removes the locality config from all
// tables under the supplied database.
func removeLocalityConfigFromAllTablesInDB(
Expand All @@ -370,51 +404,48 @@ func removeLocalityConfigFromAllTablesInDB(
)
}
b := p.Txn().NewBatch()
if err := forEachTableDesc(ctx, p, desc, hideVirtual,
func(immutable *dbdesc.Immutable, _ string, desc catalog.TableDescriptor) error {
mutDesc, err := p.Descriptors().GetMutableTableByID(ctx, p.txn, desc.GetID(), tree.ObjectLookupFlags{})
if err != nil {
if err := p.forEachTableInMultiRegionDatabase(ctx, desc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error {
// The user must either be an admin or have the requisite privileges.
if err := p.ensureCorrectMultiRegionPrivilegesForTable(ctx, tbDesc); err != nil {
return err
}

switch t := tbDesc.LocalityConfig.Locality.(type) {
case *descpb.TableDescriptor_LocalityConfig_Global_:
if err := ApplyZoneConfigForMultiRegionTable(
ctx,
p.txn,
p.ExecCfg(),
multiregion.RegionConfig{}, // pass dummy config as it is not used.
tbDesc,
applyZoneConfigForMultiRegionTableOptionRemoveGlobalZoneConfig,
); err != nil {
return err
}
switch t := mutDesc.LocalityConfig.Locality.(type) {
case *descpb.TableDescriptor_LocalityConfig_Global_:
if err := ApplyZoneConfigForMultiRegionTable(
ctx,
p.txn,
p.ExecCfg(),
multiregion.RegionConfig{}, // pass dummy config as it is not used.
mutDesc,
applyZoneConfigForMultiRegionTableOptionRemoveGlobalZoneConfig,
); err != nil {
return err
}
case *descpb.TableDescriptor_LocalityConfig_RegionalByTable_:
if t.RegionalByTable.Region != nil {
// This should error during the type descriptor changes.
return errors.AssertionFailedf(
"unexpected REGIONAL BY TABLE IN <region> on table %s during DROP REGION",
mutDesc.Name,
)
}
case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_:
case *descpb.TableDescriptor_LocalityConfig_RegionalByTable_:
if t.RegionalByTable.Region != nil {
// This should error during the type descriptor changes.
return errors.AssertionFailedf(
"unexpected REGIONAL BY ROW on table %s during DROP REGION",
mutDesc.Name,
"unexpected REGIONAL BY TABLE IN <region> on table %s during DROP REGION",
tbDesc.Name,
)
default:
return errors.AssertionFailedf(
"unexpected locality %T on table %s during DROP REGION",
t,
mutDesc.Name,
)
}
mutDesc.LocalityConfig = nil
if err := p.writeSchemaChangeToBatch(ctx, mutDesc, b); err != nil {
return err
}
return nil
}); err != nil {
case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_:
// This should error during the type descriptor changes.
return errors.AssertionFailedf(
"unexpected REGIONAL BY ROW on table %s during DROP REGION",
tbDesc.Name,
)
default:
return errors.AssertionFailedf(
"unexpected locality %T on table %s during DROP REGION",
t,
tbDesc.Name,
)
}
tbDesc.LocalityConfig = nil
return p.writeSchemaChangeToBatch(ctx, tbDesc, b)
}); err != nil {
return err
}
return p.Txn().Run(ctx, b)
Expand Down Expand Up @@ -608,38 +639,33 @@ func addDefaultLocalityConfigToAllTables(
)
}
b := p.Txn().NewBatch()
if err := forEachTableDesc(ctx, p, dbDesc, hideVirtual,
func(immutable *dbdesc.Immutable, _ string, desc catalog.TableDescriptor) error {
mutDesc, err := p.Descriptors().GetMutableTableByID(
ctx, p.txn, desc.GetID(), tree.ObjectLookupFlags{},
)
if err != nil {
return err
}
if err := p.forEachTableInMultiRegionDatabase(ctx, dbDesc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error {
if err := p.ensureCorrectMultiRegionPrivilegesForTable(ctx, tbDesc); err != nil {
return err
}

if err := checkCanConvertTableToMultiRegion(dbDesc, mutDesc); err != nil {
return err
}
if err := checkCanConvertTableToMultiRegion(dbDesc, tbDesc); err != nil {
return err
}

if mutDesc.MaterializedView() {
if err := p.alterTableDescLocalityToGlobal(
ctx, mutDesc, regionEnumID,
); err != nil {
return err
}
} else {
if err := p.alterTableDescLocalityToRegionalByTable(
ctx, tree.PrimaryRegionNotSpecifiedName, mutDesc, regionEnumID,
); err != nil {
return err
}
if tbDesc.MaterializedView() {
if err := p.alterTableDescLocalityToGlobal(
ctx, tbDesc, regionEnumID,
); err != nil {
return err
}

if err := p.writeSchemaChangeToBatch(ctx, mutDesc, b); err != nil {
} else {
if err := p.alterTableDescLocalityToRegionalByTable(
ctx, tree.PrimaryRegionNotSpecifiedName, tbDesc, regionEnumID,
); err != nil {
return err
}
return nil
}); err != nil {
}
if err := p.writeSchemaChangeToBatch(ctx, tbDesc, b); err != nil {
return err
}
return nil
}); err != nil {
return err
}
return p.Txn().Run(ctx, b)
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,9 @@ func (p *planner) CheckPrivilegeForUser(

// CheckPrivilege implements the AuthorizationAccessor interface.
// Requires a valid transaction to be open.
// TODO(arul): This CheckPrivileges method name is rather deceptive,
// 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,
) error {
Expand Down
84 changes: 28 additions & 56 deletions pkg/sql/region_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -57,13 +58,17 @@ func (s *LiveClusterRegions) toStrings() []string {
// A region name is deemed active if there is at least one alive node
// in the cluster in with locality set to a given region.
func (p *planner) getLiveClusterRegions(ctx context.Context) (LiveClusterRegions, error) {
// Non-admin users can't access the crdb_internal.kv_node_status table, which
// this query hits, so we must override the user here.
override := sessiondata.InternalExecutorOverride{
User: security.RootUserName(),
}

it, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryIteratorEx(
ctx,
"get_live_cluster_regions",
p.txn,
sessiondata.InternalExecutorOverride{
User: p.SessionData().User(),
},
override,
"SELECT region FROM [SHOW REGIONS FROM CLUSTER]",
)
if err != nil {
Expand Down Expand Up @@ -711,64 +716,31 @@ func applyZoneConfigForMultiRegionDatabase(
return nil
}

// forEachTableWithLocalityConfigInDatabase loops through each schema and table
// for a table with a LocalityConfig configured.
// NOTE: this function uses cached table and schema descriptors. As a result, it may
// not be safe to run within a schema change.
// TODO(arul): This looks like a remnant of when we could have tables inside an
// MR database without a locality config. Investigate if this can be cleaned up.
func (p *planner) forEachTableWithLocalityConfigInDatabase(
// forEachTableInMultiRegionDatabase calls the given function on every table
// descriptor inside the given multi-region database. Tables that have been
// dropped are skipped.
func (p *planner) forEachTableInMultiRegionDatabase(
ctx context.Context,
desc *dbdesc.Mutable,
f func(ctx context.Context, schema string, tbName tree.TableName, tbDesc *tabledesc.Mutable) error,
dbDesc *dbdesc.Immutable,
fn func(ctx context.Context, tbDesc *tabledesc.Mutable) error,
) error {
// No work to be done if the database isn't a multi-region database.
if !desc.IsMultiRegion() {
return nil
if !dbDesc.IsMultiRegion() {
return errors.AssertionFailedf("db %q is not multi-region", dbDesc.Name)
}
lookupFlags := p.CommonLookupFlags(true /*required*/)
lookupFlags.AvoidCached = false
schemas, err := p.Descriptors().GetSchemasForDatabase(ctx, p.txn, desc.GetID())
allDescs, err := p.Descriptors().GetAllDescriptors(ctx, p.txn)
if err != nil {
return err
}

tblLookupFlags := p.CommonLookupFlags(true /*required*/)
tblLookupFlags.AvoidCached = false
tblLookupFlags.Required = false

// Loop over all schemas, then loop over all tables.
for _, schema := range schemas {
tbNames, err := p.Descriptors().GetObjectNames(
ctx,
p.txn,
desc,
schema,
tree.DatabaseListFlags{
CommonLookupFlags: lookupFlags,
ExplicitPrefix: true,
},
)
if err != nil {
return err
lCtx := newInternalLookupCtx(ctx, allDescs, dbDesc, nil /* fallback */)
for _, tbID := range lCtx.tbIDs {
desc := lCtx.tbDescs[tbID]
if desc.Dropped() {
continue
}
for i := range tbNames {
found, tbDesc, err := p.Descriptors().GetMutableTableByName(
ctx, p.txn, &tbNames[i], tree.ObjectLookupFlags{CommonLookupFlags: tblLookupFlags},
)
if err != nil {
return err
}

// If we couldn't find the table, or it has no LocalityConfig, there's nothing
// to do here.
if !found || tbDesc.LocalityConfig == nil {
continue
}

if err := f(ctx, schema, tbNames[i], tbDesc); err != nil {
return err
}
mutable := tabledesc.NewBuilder(desc.TableDesc()).BuildExistingMutableTable()
if err := fn(ctx, mutable); err != nil {
return err
}
}
return nil
Expand All @@ -777,10 +749,10 @@ func (p *planner) forEachTableWithLocalityConfigInDatabase(
// updateZoneConfigsForAllTables loops through all of the tables in the
// specified database and refreshes the zone configs for all tables.
func (p *planner) updateZoneConfigsForAllTables(ctx context.Context, desc *dbdesc.Mutable) error {
return p.forEachTableWithLocalityConfigInDatabase(
return p.forEachTableInMultiRegionDatabase(
ctx,
desc,
func(ctx context.Context, schema string, tbName tree.TableName, tbDesc *tabledesc.Mutable) error {
&desc.Immutable,
func(ctx context.Context, tbDesc *tabledesc.Mutable) error {
regionConfig, err := SynthesizeRegionConfig(ctx, p.txn, desc.ID, p.Descriptors())
if err != nil {
return err
Expand Down
Loading

0 comments on commit 0ea0538

Please sign in to comment.