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. However, no change is
made to the public interface -- users above the `tenantcapabilitiespb`
package continue to interact with these capabilities as they were
before, oblivious to how these things are stored on disk.

As part of this patch, we also clean up a testing knob that was used by
various backup, CDC, and logictests to override capability checks in the
Authorizer. We no longer need this with the new defaults.

Fixes #96736

Release note: None
  • Loading branch information
arulajmani authored and knz committed Mar 14, 2023
1 parent f9dd5e0 commit 879bd1c
Show file tree
Hide file tree
Showing 35 changed files with 240 additions and 239 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
5 changes: 2 additions & 3 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -194,8 +194,7 @@ query ITT colnames,retry
SELECT * FROM crdb_internal.node_tenant_capabilities_cache WHERE capability_id = 'can_admin_split'
----
tenant_id capability_id capability_value
1 can_admin_split false
5 can_admin_split false
1 can_admin_split true

statement ok
ALTER TENANT [5] GRANT CAPABILITY can_admin_split
Expand All @@ -205,7 +204,7 @@ query ITT colnames,retry
SELECT * FROM crdb_internal.node_tenant_capabilities_cache WHERE capability_id = 'can_admin_split'
----
tenant_id capability_id capability_value
1 can_admin_split false
1 can_admin_split true
5 can_admin_split true

subtest end
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
16 changes: 8 additions & 8 deletions pkg/ccl/logictestccl/testdata/logic_test/tenant_capability
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,8 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "no-capabilities-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_split false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -51,7 +51,7 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "bool-capability-no-value-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_view_node_info false
Expand All @@ -64,7 +64,7 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "bool-capability-no-value-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_scatter true
can_admin_split false
can_admin_unsplit false
can_view_node_info false
Expand All @@ -84,7 +84,7 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "bool-capability-with-value-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_view_node_info false
Expand All @@ -104,7 +104,7 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "bool-capability-with-expression-value-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_view_node_info false
Expand All @@ -124,7 +124,7 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "multiple-capability-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_view_node_info true
Expand All @@ -137,7 +137,7 @@ query TT colnames
SELECT capability_id, capability_value FROM [SHOW TENANT "multiple-capability-tenant" WITH CAPABILITIES]
----
capability_id capability_value
can_admin_scatter false
can_admin_scatter true
can_admin_split false
can_admin_unsplit false
can_view_node_info false
Expand Down
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
14 changes: 6 additions & 8 deletions pkg/multitenant/tenantcapabilities/capabilities.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,16 +162,14 @@ 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 describes the ability of a tenant 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.
CanAdminScatter // can_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 describes the ability of a tenant 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.
CanAdminSplit // can_admin_split

// CanAdminUnsplit describes the ability of a tenant to perform manual
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,23 +82,46 @@ func (a *Authorizer) HasCapabilityForBatch(
if requiredCap == noCapCheckNeeded {
continue
}
if !hasCap || requiredCap == onlySystemTenant || !found || !cp.GetBool(requiredCap) {
if (requiredCap == tenantcapabilities.CanAdminSplit || requiredCap == tenantcapabilities.CanAdminScatter) &&
a.knobs.AuthorizerSkipAdminSplitCapabilityChecks {
if !found {
switch request.Method() {
case kvpb.AdminSplit, kvpb.AdminScatter:
// Secondary tenants are allowed to run AdminSplit and AdminScatter
// requests by default, as they're integral to the performance of IMPORT
// and RESTORE. If no entry is found in the capabilities map, we
// fallback to this default behavior. Note that this isn't expected to
// be the case during normal operation, as tenants that exist should
// always have an entry in this map. It does help for some tests,
// however.
continue
default:
// For all other requests we conservatively return an error if no entry
// is to be found for the tenant.
return newTenantDoesNotHaveCapabilityError(requiredCap, request)
}
}
if !hasCap || requiredCap == onlySystemTenant || !cp.GetBool(requiredCap) {
// 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)
//
// TODO(arul): This should be caught by a linter instead. Add a test that
// goes over all request types and ensures there's an entry in this map
// instead.
return newTenantDoesNotHaveCapabilityError(requiredCap, request)
}
}
return nil
}

func newTenantDoesNotHaveCapabilityError(
cap tenantcapabilities.CapabilityID, req kvpb.Request,
) error {
return errors.Newf("client tenant does not have capability %q (%T)", cap, req)
}

var reqMethodToCap = map[kvpb.Method]tenantcapabilities.CapabilityID{
// The following requests are authorized for all workloads.
kvpb.AddSSTable: noCapCheckNeeded,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,9 @@ has-capability-for-batch ten=11 cmds=(AdminUnsplit, Scan, ConditionalPut)
client tenant does not have capability "can_admin_unsplit" (*kvpb.AdminUnsplitRequest)

# Test that the order of the split request doesn't have any effect.
has-capability-for-batch ten=11 cmds=(Scan, ConditionalPut)
has-capability-for-batch ten=11 cmds=(Scan, ConditionalPut, AdminSplit)
----
ok
client tenant does not have capability "can_admin_split" (*kvpb.AdminSplitRequest)

# However, a batch request which doesn't include a split (by tenant 11) should
# work as you'd expect.
Expand All @@ -51,18 +51,19 @@ ok

# Lastly, flip tenant 10's capability for splits; ensure it can no longer issue
# splits as a result.
upsert ten=10 can_admin_scatter=false can_admin_split=false can_admin_unsplit=false can_view_node_info=true can_view_tsdb_metrics=true
upsert ten=10 can_admin_scatter=true can_admin_split=false can_admin_unsplit=false can_view_node_info=true can_view_tsdb_metrics=true
----
ok

has-capability-for-batch ten=10 cmds=(AdminScatter, Scan, ConditionalPut)
----
client tenant does not have capability "can_admin_scatter" (*kvpb.AdminScatterRequest)

has-capability-for-batch ten=10 cmds=(AdminSplit, Scan, ConditionalPut)
----
client tenant does not have capability "can_admin_split" (*kvpb.AdminSplitRequest)

# Does not affect admin scatters.
has-capability-for-batch ten=10 cmds=(AdminScatter, Scan, ConditionalPut)
----
ok

has-capability-for-batch ten=10 cmds=(AdminUnsplit, Scan, ConditionalPut)
----
client tenant does not have capability "can_admin_unsplit" (*kvpb.AdminUnsplitRequest)
Expand Down
Loading

0 comments on commit 879bd1c

Please sign in to comment.