Skip to content

Commit

Permalink
multitenant: allow secondary tenants to split/scatter by default
Browse files Browse the repository at this point in the history
AdminSplit and AdminScatter requests are subject to capability checks.
Previously, these capabilities were codified in the "enabled" form. As
such, by default, secondary tenants did not have the ability to perform
these operations. This is in violation of what secondary tenants could
do prior to 23.1, at a time before capabilities existed. Moreover,
RESTORE/IMPORT rely on performing these operations for performance.
This made disallowing these operations by default a performance
regression.

This patch flips the phrasing of how these capabilities are stored on
the proto to use the "disable" verbiage. As such, secondary tenants are
able to perform splits and scatters by default.

As part of this change, we also clean up a testing knob that was used
by various backup, CDC, and logictests to override capability checks in
the authorizer. This isn't required with the new default behaviour. We
also add some missing E2E tests for the `CanAdminUnsplit` capability
which were missing when it was introduced.

Fixes #96736

Release note: None
  • Loading branch information
arulajmani committed Mar 13, 2023
1 parent c31c1ac commit 57dd19c
Show file tree
Hide file tree
Showing 36 changed files with 241 additions and 283 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,6 @@ go_test(
"//pkg/kv/kvserver/protectedts/ptpb",
"//pkg/kv/kvserver/protectedts/ptutil",
"//pkg/multitenant/mtinfopb",
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/scheduledjobs",
"//pkg/scheduledjobs/schedulebase",
Expand Down
15 changes: 0 additions & 15 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptutil"
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/username"
Expand Down Expand Up @@ -2949,13 +2948,6 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {
defer cleanupFn()
args := base.TestServerArgs{
ExternalIODir: dir,
Knobs: base.TestingKnobs{
TenantCapabilitiesTestingKnobs: &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
},
},
}

// Generate some testdata and back it up.
Expand Down Expand Up @@ -5296,13 +5288,6 @@ func TestBackupRestoreSequence(t *testing.T) {
defer cleanupFn()
args := base.TestServerArgs{
ExternalIODir: dir,
Knobs: base.TestingKnobs{
TenantCapabilitiesTestingKnobs: &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
},
},
}

backupLoc := localFoo
Expand Down
9 changes: 0 additions & 9 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -155,14 +154,6 @@ func (d *datadrivenTestState) addCluster(t *testing.T, cfg clusterCfg) error {
params.ServerArgs.Knobs = base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
}
// Backups issue splits underneath the hood, and as such, will fail capability
// checks for tests that run as secondary tenants. Skip these checks at a
// global level using a testing knob.
params.ServerArgs.Knobs.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
}

settings := cluster.MakeTestingClusterSettings()

Expand Down
28 changes: 0 additions & 28 deletions pkg/ccl/backupccl/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/keyvisualizer"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -96,11 +95,6 @@ func backupRestoreTestSetupWithParams(
}
params.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs).SmallEngineBlocks = true
}
params.ServerArgs.Knobs.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
}

params.ServerArgs.Knobs.KeyVisualizer = &keyvisualizer.TestingKnobs{
SkipJobBootstrap: true,
Expand Down Expand Up @@ -154,13 +148,6 @@ func backupRestoreTestSetup(
base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
DisableDefaultTestTenant: true,
Knobs: base.TestingKnobs{
TenantCapabilitiesTestingKnobs: &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
},
},
}})
}

Expand All @@ -173,11 +160,6 @@ func backupRestoreTestSetupEmpty(
) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, cleanup func()) {
// TODO (msbutler): this should be disabled by callers of this function
params.ServerArgs.DisableDefaultTestTenant = true
params.ServerArgs.Knobs.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
}
return backupRestoreTestSetupEmptyWithParams(t, clusterSize, tempDir, init, params)
}

Expand Down Expand Up @@ -205,11 +187,6 @@ func backupRestoreTestSetupEmptyWithParams(
}
params.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs).SmallEngineBlocks = true
}
params.ServerArgs.Knobs.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
}

tc = testcluster.StartTestCluster(t, clusterSize, params)
init(tc)
Expand All @@ -235,11 +212,6 @@ func createEmptyCluster(
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
}
params.ServerArgs.Knobs.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{
// TODO(arul): This can be removed once
// https://github.com/cockroachdb/cockroach/issues/96736 is fixed.
AuthorizerSkipAdminSplitCapabilityChecks: true,
}
tc := testcluster.StartTestCluster(t, clusterSize, params)

sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0])
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,5 @@
# tenant-cluster-setting-override-opt: sql.zone_configs.allow_for_secondary_tenant.enabled=true sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true
# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-no-los
# cluster-opt: can-admin-split

# Tests in this file assume no multi-region tenant setup as tenants have no
# access to nodelocal.
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
# tenant-cluster-setting-override-opt: sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true
# cluster-opt: can-admin-split
# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-tenant multiregion-9node-3region-3azs-no-los

statement ok
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
# cluster-opt: can-admin-split
# LogicTest: 5node !metamorphic-batch-sizes

statement ok
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
# tenant-cluster-setting-override-opt: sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true
# cluster-opt: can-admin-split
# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-no-los

statement ok
Expand Down
Original file line number Diff line number Diff line change
@@ -1,34 +1,25 @@
query-sql-system
SELECT * FROM [SHOW TENANT [10] WITH CAPABILITIES] WHERE capability_id = 'can_admin_split'
SELECT * FROM [SHOW TENANT [10] WITH CAPABILITIES] WHERE capability_id = 'disable_admin_split'
----
10 tenant-10 ready none can_admin_split false
10 tenant-10 ready none disable_admin_split false

exec-sql-tenant
CREATE TABLE t(a INT)
----
ok

exec-privileged-op-tenant
ALTER TABLE t SPLIT AT VALUES (0)
----
pq: ba: AdminSplit [/Tenant/10/Table/104/1/0,/Min) RPC error: rpc error: code = Unauthenticated desc = client tenant does not have capability "can_admin_split" (*kvpb.AdminSplitRequest)

update-capabilities
ALTER TENANT [10] GRANT CAPABILITY can_admin_split=true
----
ok

# By default, we should be able to split.
exec-privileged-op-tenant
ALTER TABLE t SPLIT AT VALUES (0)
----
ok

update-capabilities
ALTER TENANT [10] REVOKE CAPABILITY can_admin_split
ALTER TENANT [10] GRANT CAPABILITY disable_admin_split=true
----
ok

exec-privileged-op-tenant
ALTER TABLE t SPLIT AT VALUES (0)
----
pq: ba: AdminSplit [/Tenant/10/Table/104/1/0,/Min) RPC error: rpc error: code = Unauthenticated desc = client tenant does not have capability "can_admin_split" (*kvpb.AdminSplitRequest)
pq: ba: AdminSplit [/Tenant/10/Table/104/1/0,/Min) RPC error: rpc error: code = Unauthenticated desc = client tenant capability "disable_admin_split" prevents operation (*kvpb.AdminSplitRequest)
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
query-sql-system
SELECT * FROM [SHOW TENANT [10] WITH CAPABILITIES] WHERE capability_id = 'can_admin_unsplit'
----
10 tenant-10 ready none can_admin_unsplit false

exec-sql-tenant
CREATE TABLE t(a INT)
----
ok

exec-privileged-op-tenant
ALTER TABLE t UNSPLIT AT VALUES (0)
----
pq: could not UNSPLIT AT (0): ba: AdminUnsplit [/Tenant/10/Table/104/1/0,/Min) RPC error: grpc: client tenant does not have capability "can_admin_unsplit" (*kvpb.AdminUnsplitRequest) [code 16/Unauthenticated]

update-capabilities
ALTER TENANT [10] GRANT CAPABILITY can_admin_unsplit=true
----
ok

exec-privileged-op-tenant
ALTER TABLE t UNSPLIT AT VALUES (0)
----
pq: could not UNSPLIT AT (0): key /Tenant/10/Table/104/1/0 is not the start of a range

update-capabilities
ALTER TENANT [10] REVOKE CAPABILITY can_admin_unsplit
----
ok

exec-privileged-op-tenant
ALTER TABLE t UNSPLIT AT VALUES (0)
----
pq: could not UNSPLIT AT (0): ba: AdminUnsplit [/Tenant/10/Table/104/1/0,/Min) RPC error: grpc: client tenant does not have capability "can_admin_unsplit" (*kvpb.AdminUnsplitRequest) [code 16/Unauthenticated]
1 change: 0 additions & 1 deletion pkg/ccl/streamingccl/streamingest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,6 @@ go_test(
"//pkg/kv/kvpb",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/protectedts",
"//pkg/multitenant/tenantcapabilities",
"//pkg/repstream/streampb",
"//pkg/roachpb",
"//pkg/security/securityassets",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
Expand Down Expand Up @@ -685,9 +684,6 @@ func TestTenantStreamingMultipleNodes(t *testing.T) {
clientAddresses[addr] = struct{}{}
},
}
args.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{
AuthorizerSkipAdminSplitCapabilityChecks: true,
}

c, cleanup := replicationtestutils.CreateTenantStreamingClusters(ctx, t, args)
defer cleanup()
Expand Down
23 changes: 11 additions & 12 deletions pkg/multitenant/tenantcapabilities/capabilities.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,17 +162,16 @@ type TenantCapabilities interface {
const (
_ CapabilityID = iota

// CanAdminScatter describes the ability of a tenant to perform manual
// KV scatter requests. These operations need a capability
// because excessive KV range scatter can overwhelm the storage
// cluster.
CanAdminScatter // can_admin_scatter
// DisableAdminScatter disallows a secondary tenant from being able to scatter
// ranges using an AdminScatter request. By default, secondary tenants are
// allowed to scatter as doing so is integral to the performance of
// IMPORT/RESTORE.
DisableAdminScatter // disable_admin_scatter

// CanAdminSplit describes the ability of a tenant to perform manual
// KV range split requests. These operations need a capability
// because excessive KV range splits can overwhelm the storage
// cluster.
CanAdminSplit // can_admin_split
// DisableAdminSplit disallows a secondary tenant from being able to perform
// KV requests to split ranges. By default, secondary tenants are allowed to
// perform splits as doing so is integral to performance of IMPORT/RESTORE.
DisableAdminSplit // disable_admin_split

// CanAdminUnsplit describes the ability of a tenant to perform manual
// KV range unsplit requests. These operations need a capability
Expand Down Expand Up @@ -230,8 +229,8 @@ const (
func (c CapabilityID) CapabilityType() Type {
switch c {
case
CanAdminScatter,
CanAdminSplit,
DisableAdminScatter,
DisableAdminSplit,
CanAdminUnsplit,
CanViewNodeInfo,
CanViewTSDBMetrics:
Expand Down
8 changes: 4 additions & 4 deletions pkg/multitenant/tenantcapabilities/capabilityid_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -78,22 +78,39 @@ func (a *Authorizer) HasCapabilityForBatch(

for _, ru := range ba.Requests {
request := ru.GetInner()
requiredCap, hasCap := reqMethodToCap[request.Method()]
if requiredCap == noCapCheckNeeded {
capability, hasCap := reqMethodToCap[request.Method()]
if capability == noCapCheckNeeded {
continue
}
if !hasCap || requiredCap == onlySystemTenant || !found || !cp.GetBool(requiredCap) {
if (requiredCap == tenantcapabilities.CanAdminSplit || requiredCap == tenantcapabilities.CanAdminScatter) &&
a.knobs.AuthorizerSkipAdminSplitCapabilityChecks {
continue
switch request.Method() {
case kvpb.AdminSplit, kvpb.AdminScatter:
// AdminSplit and AdminScatter requests must are codified as
// "disable_admin_{split,scatter}" in the capabilities proto, as tenants
// have the ability to perform splits and scatters by default. This is
// because things like IMPORT/RESTORE rely on these operations for
// performance. As such, they must be explicitly revoked by the operator.
// This is in contrast to other capabilities that require authorization,
// which are explicitly granted.
if !found {
return nil // allowed by default
}
if disabled := cp.GetBool(capability); disabled {
return errors.Newf(
"client tenant capability %q prevents operation (%T)", capability, request,
)
}
default:
// All other requests that require capabilities are expressed in their
// "enabled" form.
if !hasCap || capability == onlySystemTenant || !found || !cp.GetBool(capability) {
// All allowable request types must be explicitly opted into the
// reqMethodToCap map. If a request type is missing from the map
// (!hasCap), we must be conservative and assume it is
// disallowed. This prevents accidents where someone adds a new
// sensitive request type in KV and forgets to add an explicit
// authorization rule for it here.
return errors.Newf("client tenant does not have capability %q (%T)", capability, request)
}
// All allowable request types must be explicitly opted into the
// reqMethodToCap map. If a request type is missing from the map
// (!hasCap), we must be conservative and assume it is
// disallowed. This prevents accidents where someone adds a new
// sensitive request type in KV and forgets to add an explicit
// authorization rule for it here.
return errors.Newf("client tenant does not have capability %q (%T)", requiredCap, request)
}
}
return nil
Expand Down Expand Up @@ -131,8 +148,8 @@ var reqMethodToCap = map[kvpb.Method]tenantcapabilities.CapabilityID{
kvpb.Scan: noCapCheckNeeded,

// The following are authorized via specific capabilities.
kvpb.AdminScatter: tenantcapabilities.CanAdminScatter,
kvpb.AdminSplit: tenantcapabilities.CanAdminSplit,
kvpb.AdminScatter: tenantcapabilities.DisableAdminScatter,
kvpb.AdminSplit: tenantcapabilities.DisableAdminSplit,
kvpb.AdminUnsplit: tenantcapabilities.CanAdminUnsplit,

// TODO(ecwall): The following should also be authorized via specific capabilities.
Expand Down
Loading

0 comments on commit 57dd19c

Please sign in to comment.