Skip to content

Commit

Permalink
server: hook up tenant capabilities subsystem on startup
Browse files Browse the repository at this point in the history
This patch wires up the tenant capabilities subsystem during server
startup. This includes both starting the subsystem and adding a handle
to the Authorizer so that it can be used by GRPC interceptors to perform
authorization checks.

While working through this patch, I realized we can't instantiate an
Authorizer with a handle to the tenant capability state
(tenantcapabilities.Reader). This is because the afformentioned GRPC
setup happens early on during the Server startup process, and at that
point we do not have access to the dependencies required to setup the
capabilities Watcher (which is what provides the Reader interface to
the Authorizer). To break this dependency cycle, we end up with an
approach to lazily bind the Reader to the Authorizer.

With the Authorizer wired up, we can now start using it to perform
capability checks for incoming tenant requests. Currently, this is
limited to batch requests. Note that the Authorizer is only responsible
for performing capability checks -- other authorization checks, such as
bounds checks, continue to happen outside of Authorizer.

This patch also adds a datadriven framework to test tenant capabilities
end to end. The nice thing about it is it hides the asynchronous nature
of capability checks from test writers. The hope is that we'll be able
to extend this as we add more capabilities.

Informs #94643

Release note: None
  • Loading branch information
arulajmani committed Feb 9, 2023
1 parent 0ab83da commit 724598b
Show file tree
Hide file tree
Showing 59 changed files with 980 additions and 237 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ ALL_TESTS = [
"//pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-vec-off:multiregion-9node-3region-3azs-vec-off_test",
"//pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs:multiregion-9node-3region-3azs_test",
"//pkg/ccl/multiregionccl:multiregionccl_test",
"//pkg/ccl/multitenantccl/tenantcapabilitiesccl:tenantcapabilitiesccl_test",
"//pkg/ccl/multitenantccl/tenantcostclient:tenantcostclient_test",
"//pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket:tenanttokenbucket_test",
"//pkg/ccl/multitenantccl/tenantcostserver:tenantcostserver_test",
Expand Down Expand Up @@ -773,6 +774,7 @@ GO_TARGETS = [
"//pkg/ccl/multiregionccl/multiregionccltestutils:multiregionccltestutils",
"//pkg/ccl/multiregionccl:multiregionccl",
"//pkg/ccl/multiregionccl:multiregionccl_test",
"//pkg/ccl/multitenantccl/tenantcapabilitiesccl:tenantcapabilitiesccl_test",
"//pkg/ccl/multitenantccl/tenantcostclient:tenantcostclient",
"//pkg/ccl/multitenantccl/tenantcostclient:tenantcostclient_test",
"//pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket:tenanttokenbucket",
Expand Down Expand Up @@ -2359,6 +2361,7 @@ GET_X_DATA_TARGETS = [
"//pkg/ccl/multiregionccl:get_x_data",
"//pkg/ccl/multiregionccl/multiregionccltestutils:get_x_data",
"//pkg/ccl/multitenantccl:get_x_data",
"//pkg/ccl/multitenantccl/tenantcapabilitiesccl:get_x_data",
"//pkg/ccl/multitenantccl/tenantcostclient:get_x_data",
"//pkg/ccl/multitenantccl/tenantcostserver:get_x_data",
"//pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket:get_x_data",
Expand Down
73 changes: 37 additions & 36 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,40 +19,41 @@ type ModuleTestingKnobs interface {
// TestingKnobs contains facilities for controlling various parts of the
// system for testing.
type TestingKnobs struct {
Store ModuleTestingKnobs
KVClient ModuleTestingKnobs
RangeFeed ModuleTestingKnobs
SQLExecutor ModuleTestingKnobs
SQLLeaseManager ModuleTestingKnobs
SQLSchemaChanger ModuleTestingKnobs
SQLDeclarativeSchemaChanger ModuleTestingKnobs
SQLTypeSchemaChanger ModuleTestingKnobs
GCJob ModuleTestingKnobs
PGWireTestingKnobs ModuleTestingKnobs
DistSQL ModuleTestingKnobs
SQLEvalContext ModuleTestingKnobs
NodeLiveness ModuleTestingKnobs
Server ModuleTestingKnobs
TenantTestingKnobs ModuleTestingKnobs
JobsTestingKnobs ModuleTestingKnobs
BackupRestore ModuleTestingKnobs
TTL ModuleTestingKnobs
SchemaTelemetry ModuleTestingKnobs
Streaming ModuleTestingKnobs
UpgradeManager ModuleTestingKnobs
IndexUsageStatsKnobs ModuleTestingKnobs
SQLStatsKnobs ModuleTestingKnobs
SpanConfig ModuleTestingKnobs
SQLLivenessKnobs ModuleTestingKnobs
TelemetryLoggingKnobs ModuleTestingKnobs
DialerKnobs ModuleTestingKnobs
ProtectedTS ModuleTestingKnobs
CapturedIndexUsageStatsKnobs ModuleTestingKnobs
AdmissionControl ModuleTestingKnobs
UnusedIndexRecommendKnobs ModuleTestingKnobs
ExternalConnection ModuleTestingKnobs
EventExporter ModuleTestingKnobs
EventLog ModuleTestingKnobs
LOQRecovery ModuleTestingKnobs
KeyVisualizer ModuleTestingKnobs
Store ModuleTestingKnobs
KVClient ModuleTestingKnobs
RangeFeed ModuleTestingKnobs
SQLExecutor ModuleTestingKnobs
SQLLeaseManager ModuleTestingKnobs
SQLSchemaChanger ModuleTestingKnobs
SQLDeclarativeSchemaChanger ModuleTestingKnobs
SQLTypeSchemaChanger ModuleTestingKnobs
GCJob ModuleTestingKnobs
PGWireTestingKnobs ModuleTestingKnobs
DistSQL ModuleTestingKnobs
SQLEvalContext ModuleTestingKnobs
NodeLiveness ModuleTestingKnobs
Server ModuleTestingKnobs
TenantTestingKnobs ModuleTestingKnobs
JobsTestingKnobs ModuleTestingKnobs
BackupRestore ModuleTestingKnobs
TTL ModuleTestingKnobs
SchemaTelemetry ModuleTestingKnobs
Streaming ModuleTestingKnobs
UpgradeManager ModuleTestingKnobs
IndexUsageStatsKnobs ModuleTestingKnobs
SQLStatsKnobs ModuleTestingKnobs
SpanConfig ModuleTestingKnobs
SQLLivenessKnobs ModuleTestingKnobs
TelemetryLoggingKnobs ModuleTestingKnobs
DialerKnobs ModuleTestingKnobs
ProtectedTS ModuleTestingKnobs
CapturedIndexUsageStatsKnobs ModuleTestingKnobs
AdmissionControl ModuleTestingKnobs
UnusedIndexRecommendKnobs ModuleTestingKnobs
ExternalConnection ModuleTestingKnobs
EventExporter ModuleTestingKnobs
EventLog ModuleTestingKnobs
LOQRecovery ModuleTestingKnobs
KeyVisualizer ModuleTestingKnobs
TenantCapabilitiesTestingKnobs ModuleTestingKnobs
}
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,7 @@ 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
23 changes: 21 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ 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 @@ -2812,7 +2813,16 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {

_, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{ExternalIODir: dir}
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 @@ -5150,7 +5160,16 @@ func TestBackupRestoreSequence(t *testing.T) {
const numAccounts = 1
_, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{ExternalIODir: dir}
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: 9 additions & 0 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"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 @@ -152,6 +153,14 @@ 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
33 changes: 32 additions & 1 deletion pkg/ccl/backupccl/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ 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 @@ -95,6 +96,11 @@ 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 @@ -145,7 +151,17 @@ func backupRestoreTestSetup(
) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) {
// TODO (msbutler): DisableDefaultTestTenant should be disabled by the caller of this function
return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init,
base.TestClusterArgs{ServerArgs: base.TestServerArgs{DisableDefaultTestTenant: true}})
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,
},
},
}})
}

func backupRestoreTestSetupEmpty(
Expand All @@ -157,6 +173,11 @@ 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 @@ -184,6 +205,11 @@ 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 @@ -209,6 +235,11 @@ 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,5 +1,6 @@
# 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,4 +1,5 @@
# 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,3 +1,4 @@
# 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,4 +1,5 @@
# 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
39 changes: 39 additions & 0 deletions pkg/ccl/multitenantccl/tenantcapabilitiesccl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "tenantcapabilitiesccl_test",
srcs = [
"capabilities_test.go",
"main_test.go",
],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/kv/kvclient/rangefeed/rangefeedcache",
"//pkg/multitenant/tenantcapabilities",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/security/username",
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/randutil",
"//pkg/util/syncutil",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)

get_x_data(name = "get_x_data")
Loading

0 comments on commit 724598b

Please sign in to comment.