diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 4cbd339a21ab..55267ed7a05f 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -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", @@ -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", @@ -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", diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index e9ddf35ec2c8..5183a1f09522 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -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 } diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 19ef4484d626..b8f8bb737a60 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -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", diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index e2c648db2992..973e6597fd6d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -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" @@ -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. { @@ -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 diff --git a/pkg/ccl/backupccl/datadriven_test.go b/pkg/ccl/backupccl/datadriven_test.go index 1fafec6a20ae..0fc60ba41204 100644 --- a/pkg/ccl/backupccl/datadriven_test.go +++ b/pkg/ccl/backupccl/datadriven_test.go @@ -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" @@ -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() diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 0d28f6bc9499..bef1fe75f8d6 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -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" @@ -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, @@ -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( @@ -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) } @@ -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) @@ -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]) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_backup b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_backup index d5ba66b39d78..08d8e6176c48 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_backup +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_backup @@ -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. diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_mr b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_mr index 0854dcb80c46..837451598e0b 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_mr +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_mr @@ -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 diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_query_plan b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_query_plan index f518445e19e7..d9b6e191e789 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_query_plan +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index_query_plan @@ -1,3 +1,4 @@ +# cluster-opt: can-admin-split # LogicTest: 5node !metamorphic-batch-sizes statement ok diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_hash_sharded_index b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_hash_sharded_index index b4b6d2af579f..d9fd13ab6ae5 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_hash_sharded_index +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_hash_sharded_index @@ -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 diff --git a/pkg/ccl/multitenantccl/tenantcapabilitiesccl/BUILD.bazel b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/BUILD.bazel new file mode 100644 index 000000000000..6a7462bdbad6 --- /dev/null +++ b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/BUILD.bazel @@ -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") diff --git a/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go new file mode 100644 index 000000000000..0e1267a04088 --- /dev/null +++ b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go @@ -0,0 +1,161 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package tenantcapabilitiesccl + +import ( + "context" + gosql "database/sql" + "fmt" + "net/url" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +// TestDataDriven runs datadriven tests against the entire tenant capabilities +// subsystem, and in doing so, serves as an end-to-end integration test. +// +// Crucially, it keeps track of how up-to-date the in-memory Authorizer state +// is when capabilities are updated. This allows test authors to change +// capabilities and make assertions against those changes, without needing to +// worry about the asynchronous nature of capability changes applying. +// +// The test creates a secondary tenant, with tenant ID 10, that test authors can +// reference directly. +// +// The syntax is as follows: +// +// query-sql-system: runs a query against the system tenant. +// +// exec-sql-tenant: executes a query against a secondary tenant (with ID 10). +// +// exec-privileged-op-tenant: executes a privileged operation (one that requires +// capabilities) as a secondary tenant. +// +// update-capabilities: expects a SQL statement that updates capabilities for a +// tenant. Following statements are guaranteed to see the effects of the +// update. +func TestDataDriven(t *testing.T) { + defer leaktest.AfterTest(t)() + datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { + ctx := context.Background() + + // Setup both the system tenant and a secondary tenant. + mu := struct { + syncutil.Mutex + lastFrontierTS hlc.Timestamp // ensures watcher is caught up + }{} + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DisableDefaultTestTenant: true, // We'll create a tenant ourselves. + Knobs: base.TestingKnobs{ + TenantCapabilitiesTestingKnobs: &tenantcapabilities.TestingKnobs{ + WatcherTestingKnobs: &tenantcapabilitieswatcher.TestingKnobs{ + WatcherRangeFeedKnobs: &rangefeedcache.TestingKnobs{ + OnTimestampAdvance: func(ts hlc.Timestamp) { + mu.Lock() + defer mu.Unlock() + mu.lastFrontierTS = ts + }, + }, + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + systemSQLDB := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + // Create a tenant; we also want to allow test writers to issue + // ALTER TABLE ... SPLIT statements, so configure the settings as such. + // TODO(knz): Once https://github.com/cockroachdb/cockroach/issues/96512 is + // resolved, we could override this cluster setting for the secondary tenant + // using SQL instead of reaching in using this testing knob. One way to do + // so would be to perform the override for all tenants and only then + // initializing our test tenant; However, the linked issue above prevents + // us from being able to do so. + settings := cluster.MakeTestingClusterSettings() + sql.SecondaryTenantSplitAtEnabled.Override(ctx, &settings.SV, true) + tenantArgs := base.TestTenantArgs{ + TenantID: serverutils.TestTenantID(), + Settings: settings, + } + testTenantInterface, err := tc.Server(0).StartTenant(ctx, tenantArgs) + require.NoError(t, err) + + pgURL, cleanupPGUrl := sqlutils.PGUrl(t, testTenantInterface.SQLAddr(), "Tenant", url.User(username.RootUser)) + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + defer func() { + require.NoError(t, tenantSQLDB.Close()) + defer cleanupPGUrl() + }() + require.NoError(t, err) + + var lastUpdateTS hlc.Timestamp + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + + switch d.Cmd { + case "update-capabilities": + systemSQLDB.Exec(t, d.Input) + lastUpdateTS = tc.Server(0).Clock().Now() + + case "exec-privileged-op-tenant": + testutils.SucceedsSoon(t, func() error { + mu.Lock() + defer mu.Unlock() + + if lastUpdateTS.Less(mu.lastFrontierTS) { + return nil + } + + return errors.Newf("frontier timestamp (%s) lagging last update (%s)", + mu.lastFrontierTS.String(), lastUpdateTS.String()) + }) + _, err := tenantSQLDB.Exec(d.Input) + if err != nil { + return err.Error() + } + + case "exec-sql-tenant": + _, err := tenantSQLDB.Exec(d.Input) + if err != nil { + return err.Error() + } + + case "query-sql-system": + rows := systemSQLDB.Query(t, d.Input) + output, err := sqlutils.RowsToDataDrivenOutput(rows) + require.NoError(t, err) + return output + + default: + return fmt.Sprintf("unknown command %s", d.Cmd) + } + return "ok" + }) + }) +} diff --git a/pkg/ccl/multitenantccl/tenantcapabilitiesccl/main_test.go b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/main_test.go new file mode 100644 index 000000000000..f64cd611d7d2 --- /dev/null +++ b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package tenantcapabilitiesccl + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl" + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + defer ccl.TestingEnableEnterprise()() + securityassets.SetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/ccl/multitenantccl/tenantcapabilitiesccl/testdata/can_admin_split b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/testdata/can_admin_split new file mode 100644 index 000000000000..79a66b942973 --- /dev/null +++ b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/testdata/can_admin_split @@ -0,0 +1,35 @@ +query-sql-system +SHOW TENANT [10] WITH CAPABILITIES +---- +10 tenant-10 ready none can_admin_split false +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 SPLIT AT VALUES (0) +---- +pq: ba: AdminSplit [/Tenant/10/Table/104/1/0,/Min) RPC error: rpc error: code = Unauthenticated desc = tenant 10 does not have admin split capability + +update-capabilities +ALTER TENANT [10] GRANT CAPABILITY can_admin_split=true +---- +ok + +exec-privileged-op-tenant +ALTER TABLE t SPLIT AT VALUES (0) +---- +ok + +update-capabilities +ALTER TENANT [10] REVOKE CAPABILITY can_admin_split +---- +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 = tenant 10 does not have admin split capability diff --git a/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel b/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel index 91d526a8f9d2..6f99a031ff3c 100644 --- a/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel +++ b/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel @@ -51,6 +51,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/multitenant/mtinfopb", + "//pkg/multitenant/tenantcapabilities", "//pkg/repstream/streampb", "//pkg/roachpb", "//pkg/security/username", diff --git a/pkg/ccl/streamingccl/replicationtestutils/testutils.go b/pkg/ccl/streamingccl/replicationtestutils/testutils.go index 847bb513cbe5..63ed6752ae3b 100644 --- a/pkg/ccl/streamingccl/replicationtestutils/testutils.go +++ b/pkg/ccl/streamingccl/replicationtestutils/testutils.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "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/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -50,13 +51,14 @@ type TenantStreamingClustersArgs struct { SrcNumNodes int SrcClusterSettings map[string]string - DestTenantName roachpb.TenantName - DestTenantID roachpb.TenantID - DestInitFunc destInitExecFunc - DestNumNodes int - DestClusterSettings map[string]string - RetentionTTLSeconds int - TestingKnobs *sql.StreamingTestingKnobs + DestTenantName roachpb.TenantName + DestTenantID roachpb.TenantID + DestInitFunc destInitExecFunc + DestNumNodes int + DestClusterSettings map[string]string + RetentionTTLSeconds int + TestingKnobs *sql.StreamingTestingKnobs + TenantCapabilitiesTestingKnobs *tenantcapabilities.TestingKnobs } var DefaultTenantStreamingClustersArgs = TenantStreamingClustersArgs{ @@ -206,7 +208,8 @@ func CreateTenantStreamingClusters( DistSQL: &execinfra.TestingKnobs{ StreamingTestingKnobs: args.TestingKnobs, }, - Streaming: args.TestingKnobs, + Streaming: args.TestingKnobs, + TenantCapabilitiesTestingKnobs: args.TenantCapabilitiesTestingKnobs, }, } diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel index 2c8674d789bc..3eb017a671cb 100644 --- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel @@ -106,6 +106,7 @@ go_test( "//pkg/keys", "//pkg/kv/kvserver", "//pkg/kv/kvserver/protectedts", + "//pkg/multitenant/tenantcapabilities", "//pkg/repstream/streampb", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go index c8d6b9aa9fb5..45e01509a5ab 100644 --- a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go +++ b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go @@ -22,6 +22,7 @@ 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" @@ -684,6 +685,9 @@ func TestTenantStreamingMultipleNodes(t *testing.T) { clientAddresses[addr] = struct{}{} }, } + args.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{ + AuthorizerSkipAdminSplitCapabilityChecks: true, + } c, cleanup := replicationtestutils.CreateTenantStreamingClusters(ctx, t, args) defer cleanup() diff --git a/pkg/multitenant/tenantcapabilities/BUILD.bazel b/pkg/multitenant/tenantcapabilities/BUILD.bazel index c1e2c3ad50d5..ef9c025ee0b8 100644 --- a/pkg/multitenant/tenantcapabilities/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/BUILD.bazel @@ -11,7 +11,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/kv/kvclient/rangefeed/rangefeedcache", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", ], diff --git a/pkg/multitenant/tenantcapabilities/capabilities.go b/pkg/multitenant/tenantcapabilities/capabilities.go index a47a528c4288..3722209bf032 100644 --- a/pkg/multitenant/tenantcapabilities/capabilities.go +++ b/pkg/multitenant/tenantcapabilities/capabilities.go @@ -41,10 +41,24 @@ type Reader interface { // signals other than just the tenant capability state. For example, request // usage pattern over a timespan. type Authorizer interface { - // HasCapabilityForBatch returns whether a tenant, referenced by its ID, is - // allowed to execute the supplied batch request given the capabilities it - // possesses. - HasCapabilityForBatch(context.Context, roachpb.TenantID, *roachpb.BatchRequest) bool + // HasCapabilityForBatch returns an error if a tenant, referenced by its ID, + // is not allowed to execute the supplied batch request given the capabilities + // it possesses. + HasCapabilityForBatch(context.Context, roachpb.TenantID, *roachpb.BatchRequest) error + + // BindReader is a mechanism by which the caller can bind a Reader[1] to the + // Authorizer post-creation. The Authorizer uses the Reader to consult the + // global tenant capability state to authorize incoming requests. This + // function cannot be used to update the Reader. + // + // + // [1] The canonical implementation of the Authorizer lives on GRPC + // interceptors, and as such, must be instantiated before the GRPC Server is + // created. However, the GRPC server is created very early on during Server + // startup and serves as a dependency for the canonical Reader's + // implementation. Binding the Reader late allows us to break this dependency + // cycle. + BindReader(reader Reader) } // Entry ties together a tenantID with its capabilities. diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel index 9392f4771dc5..b029594cada5 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel @@ -3,13 +3,19 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "tenantcapabilitiesauthorizer", - srcs = ["authorizer.go"], + srcs = [ + "authorizer.go", + "noop.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer", visibility = ["//visibility:public"], deps = [ "//pkg/multitenant/tenantcapabilities", "//pkg/roachpb", + "//pkg/settings/cluster", "//pkg/util/log", + "//pkg/util/log/logcrash", + "@com_github_cockroachdb_errors//:errors", ], ) @@ -24,6 +30,7 @@ go_test( "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils", "//pkg/roachpb", + "//pkg/settings/cluster", "//pkg/testutils/datapathutils", "//pkg/util/leaktest", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go index 27a36856d21d..90a8c89bbaa7 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go @@ -15,21 +15,33 @@ import ( "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" + "github.com/cockroachdb/errors" ) // Authorizer is a concrete implementation of the tenantcapabilities.Authorizer // interface. It's safe for concurrent use. type Authorizer struct { capabilitiesReader tenantcapabilities.Reader + settings *cluster.Settings + + knobs tenantcapabilities.TestingKnobs } var _ tenantcapabilities.Authorizer = &Authorizer{} // New constructs a new tenantcapabilities.Authorizer. -func New(reader tenantcapabilities.Reader) *Authorizer { +func New(settings *cluster.Settings, knobs *tenantcapabilities.TestingKnobs) *Authorizer { + var testingKnobs tenantcapabilities.TestingKnobs + if knobs != nil { + testingKnobs = *knobs + } a := &Authorizer{ - capabilitiesReader: reader, + settings: settings, + knobs: testingKnobs, + // capabilitiesReader is set post construction, using BindReader. } return a } @@ -37,14 +49,20 @@ func New(reader tenantcapabilities.Reader) *Authorizer { // HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface. func (a *Authorizer) HasCapabilityForBatch( ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest, -) bool { +) error { if tenID.IsSystem() { - return true // the system tenant is allowed to do as it pleases + return nil // the system tenant is allowed to do as it pleases + } + if a.capabilitiesReader == nil { + err := errors.AssertionFailedf("trying to perform capability check when no reader exists") + logcrash.ReportOrPanic(ctx, &a.settings.SV, "%v", err) + return err } cp, found := a.capabilitiesReader.GetCapabilities(tenID) if !found { - log.Infof( + log.VInfof( ctx, + 3, "no capability information for tenant %s; requests that require capabilities may be denied", tenID, ) @@ -53,12 +71,17 @@ func (a *Authorizer) HasCapabilityForBatch( for _, ru := range ba.Requests { switch ru.GetInner().(type) { case *roachpb.AdminSplitRequest: - if !cp.CanAdminSplit { - return false + if !cp.CanAdminSplit && !a.knobs.AuthorizerSkipAdminSplitCapabilityChecks { + return errors.Newf("tenant %s does not have admin split capability", tenID) } default: - // No capability checks for other types of requests. + // No capability checks for any other type of request. } } - return true + return nil +} + +// BindReader implements the tenantcapabilities.Authorizer interface. +func (a *Authorizer) BindReader(reader tenantcapabilities.Reader) { + a.capabilitiesReader = reader } diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer_test.go index a939885d3399..2059ad323e6d 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" @@ -47,7 +48,8 @@ func TestDataDriven(t *testing.T) { datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { mockReader := mockReader(make(map[roachpb.TenantID]tenantcapabilitiespb.TenantCapabilities)) - authorizer := New(mockReader) + authorizer := New(cluster.MakeTestingClusterSettings(), nil /* TestingKnobs */) + authorizer.BindReader(mockReader) datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { @@ -57,8 +59,11 @@ func TestDataDriven(t *testing.T) { case "has-capability-for-batch": tenID, ba := tenantcapabilitiestestutils.ParseBatchRequestString(t, d.Input) - hasCapability := authorizer.HasCapabilityForBatch(context.Background(), tenID, &ba) - return fmt.Sprintf("%t", hasCapability) + err := authorizer.HasCapabilityForBatch(context.Background(), tenID, &ba) + if err == nil { + return "ok" + } + return err.Error() default: return fmt.Sprintf("unknown command %s", d.Cmd) diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go new file mode 100644 index 000000000000..0957aaf1528a --- /dev/null +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go @@ -0,0 +1,39 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tenantcapabilitiesauthorizer + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// NoopAuthorizer is a tenantcapabilities.Authorizer that simply no-ops. +type NoopAuthorizer struct{} + +var _ tenantcapabilities.Authorizer = &NoopAuthorizer{} + +// NewNoopAuthorizer constructs and returns a NoopAuthorizer. +func NewNoopAuthorizer() *NoopAuthorizer { + return &NoopAuthorizer{} +} + +// HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface. +func (n *NoopAuthorizer) HasCapabilityForBatch( + context.Context, roachpb.TenantID, *roachpb.BatchRequest, +) error { + return nil +} + +// BindReader implements the tenantcapabilities.Authorizer interface. +func (n *NoopAuthorizer) BindReader(tenantcapabilities.Reader) { +} diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/basic b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/basic index 3d256192b847..a92d061409bc 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/basic +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/basic @@ -12,7 +12,7 @@ split scan cput ---- -true +ok # Tenant 11 shouldn't be able to issue splits. has-capability-for-batch @@ -21,7 +21,7 @@ split scan cput ---- -false +tenant 11 does not have admin split capability # Test that the order of the split request doesn't have any effect. has-capability-for-batch @@ -29,7 +29,7 @@ has-capability-for-batch scan cput ---- -true +ok # However, a batch request which doesn't include a split (by tenant 11) should # work as you'd expect. @@ -38,7 +38,7 @@ has-capability-for-batch scan cput ---- -true +ok # Ditto for tenant 10. has-capability-for-batch @@ -46,7 +46,7 @@ has-capability-for-batch scan cput ---- -true +ok # Lastly, flip tenant 10's capability for splits; ensure it can no longer issue # splits as a result. @@ -61,7 +61,7 @@ split scan cput ---- -false +tenant 10 does not have admin split capability # However, this has no effect on batch requests that don't contain splits. has-capability-for-batch @@ -69,4 +69,4 @@ has-capability-for-batch scan cput ---- -true +ok diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/no_capabilities b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/no_capabilities index b8c8f90cfc84..fe775b70b10a 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/no_capabilities +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/no_capabilities @@ -7,7 +7,7 @@ split scan cput ---- -false +tenant 10 does not have admin split capability # However, if there was no split in the batch, the batch should be allowed to # go through. @@ -16,7 +16,7 @@ has-capability-for-batch scan cput ---- -true +ok # Update the capability state to give tenant 10 the capability to run splits. update-state @@ -30,7 +30,7 @@ split scan cput ---- -true +ok # Remove the capability. update-state @@ -44,4 +44,4 @@ split scan cput ---- -false +tenant 10 does not have admin split capability diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/system_tenant b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/system_tenant index d9a1b55f754f..1a93d81c0241 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/system_tenant +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/system_tenant @@ -6,20 +6,20 @@ split scan cput ---- -true +ok has-capability-for-batch {ten=system} scan cput ---- -true +ok has-capability-for-batch {ten=system} split ---- -true +ok update-state upsert {ten=system}:{CanAdminSplit=false} @@ -35,4 +35,4 @@ has-capability-for-batch {ten=system} split ---- -true +ok diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel index a5e734e48bba..016bf17cab35 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel @@ -5,11 +5,13 @@ go_library( name = "tenantcapabilitieswatcher", srcs = [ "decoder.go", + "testingknobs.go", "watcher.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/keys", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", @@ -42,8 +44,8 @@ go_test( ], args = ["-test.timeout=295s"], data = glob(["testdata/**"]), - embed = [":tenantcapabilitieswatcher"], deps = [ + ":tenantcapabilitieswatcher", "//pkg/base", "//pkg/keys", "//pkg/kv/kvclient/rangefeed", diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go index 3f0d778793f1..705cccd0b45e 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go @@ -130,3 +130,8 @@ func (d *decoder) translateEvent( ts: ev.Value.Timestamp, } } + +// TestingDecoderFn exports the decoding routine for testing purposes. +func TestingDecoderFn() func(roachpb.KeyValue) (tenantcapabilities.Entry, error) { + return newDecoder().decode +} diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go index 7ff568385649..205dfd8d3119 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tenantcapabilitieswatcher +package tenantcapabilitieswatcher_test import ( "context" @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -72,8 +73,7 @@ func TestDecodeCapabilities(t *testing.T) { // Decode and verify. row := rows[0] - decoder := newDecoder() - got, err := decoder.decode(roachpb.KeyValue{ + got, err := tenantcapabilitieswatcher.TestingDecoderFn()(roachpb.KeyValue{ Key: row.Key, Value: *row.Value, }) diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/main_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/main_test.go index 01a884582130..3ca1f2b5948a 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/main_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/main_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tenantcapabilitieswatcher +package tenantcapabilitieswatcher_test import ( "os" diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/testingknobs.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/testingknobs.go new file mode 100644 index 000000000000..f63691d70f84 --- /dev/null +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/testingknobs.go @@ -0,0 +1,35 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tenantcapabilitieswatcher + +import ( + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" +) + +// TestingKnobs contain testing helpers which are used by the Watcher. +type TestingKnobs struct { + // WatcherRangeFeedKnobs control the lifecycle events for the underlying + // rangefeed of the tenant capabilities Watcher. + WatcherRangeFeedKnobs base.ModuleTestingKnobs + + // WatcherUpdatesInterceptor, if set, is called each time the Watcher + // receives a set of updates. + WatcherUpdatesInterceptor func( + updateType rangefeedcache.UpdateType, updates []tenantcapabilities.Update, + ) +} + +// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. +func (t *TestingKnobs) ModuleTestingKnobs() {} + +var _ base.ModuleTestingKnobs = (*TestingKnobs)(nil) diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go index 5e68919a8961..f52188ffb0d9 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go @@ -37,7 +37,7 @@ type Watcher struct { bufferMemLimit int64 tenantsTableID uint32 // overriden for tests - knobs tenantcapabilities.TestingKnobs + knobs TestingKnobs mu struct { syncutil.RWMutex @@ -57,8 +57,9 @@ func New( bufferMemLimit int64, knobs *tenantcapabilities.TestingKnobs, ) *Watcher { - if knobs == nil { - knobs = &tenantcapabilities.TestingKnobs{} + watcherKnobs := TestingKnobs{} + if knobs != nil && knobs.WatcherTestingKnobs != nil { + watcherKnobs = *knobs.WatcherTestingKnobs.(*TestingKnobs) } w := &Watcher{ clock: clock, @@ -67,7 +68,7 @@ func New( decoder: newDecoder(), tenantsTableID: tenantsTableID, bufferMemLimit: bufferMemLimit, - knobs: *knobs, + knobs: watcherKnobs, } w.mu.store = make(map[roachpb.TenantID]tenantcapabilitiespb.TenantCapabilities) return w @@ -102,6 +103,10 @@ func (w *Watcher) Start(ctx context.Context) error { EndKey: tenantsTableStart.PrefixEnd(), } + var rfcTestingKnobs *rangefeedcache.TestingKnobs + if w.knobs.WatcherRangeFeedKnobs != nil { + rfcTestingKnobs = w.knobs.WatcherRangeFeedKnobs.(*rangefeedcache.TestingKnobs) + } rfc := rangefeedcache.NewWatcher( "tenant-capability-watcher", w.clock, @@ -111,7 +116,7 @@ func (w *Watcher) Start(ctx context.Context) error { true, /* withPrevValue */ w.decoder.translateEvent, w.handleUpdate, - w.knobs.WatcherRangeFeedKnobs.(*rangefeedcache.TestingKnobs), + rfcTestingKnobs, ) return rangefeedcache.Start(ctx, w.stopper, rfc, nil /* onError */) } @@ -167,9 +172,9 @@ func (w *Watcher) handleIncrementalUpdate(updates []tenantcapabilities.Update) { } } -// testFlushCapabilitiesState flushes the underlying global tenant capability +// TestingFlushCapabilitiesState flushes the underlying global tenant capability // state for testing purposes. The returned entries are sorted by tenant ID. -func (w *Watcher) testingFlushCapabilitiesState() (entries []tenantcapabilities.Entry) { +func (w *Watcher) TestingFlushCapabilitiesState() (entries []tenantcapabilities.Entry) { w.mu.Lock() defer w.mu.Unlock() diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go index f8c2377bb4ed..443223184770 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tenantcapabilitieswatcher +package tenantcapabilitieswatcher_test import ( "context" @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -99,40 +100,42 @@ func TestDataDriven(t *testing.T) { close(restartAfterErrCh) }() - watcher := New( + watcher := tenantcapabilitieswatcher.New( ts.Clock(), ts.RangeFeedFactory().(*rangefeed.Factory), dummyTableID, ts.Stopper(), 1<<20, /* 1 MB */ &tenantcapabilities.TestingKnobs{ - WatcherRangeFeedKnobs: &rangefeedcache.TestingKnobs{ - PostRangeFeedStart: func() { - mu.Lock() - defer mu.Unlock() + WatcherTestingKnobs: &tenantcapabilitieswatcher.TestingKnobs{ + WatcherRangeFeedKnobs: &rangefeedcache.TestingKnobs{ + PostRangeFeedStart: func() { + mu.Lock() + defer mu.Unlock() - mu.rangeFeedRunning = true + mu.rangeFeedRunning = true + }, + OnTimestampAdvance: func(ts hlc.Timestamp) { + mu.Lock() + defer mu.Unlock() + mu.lastFrontierTS = ts + }, + ErrorInjectionCh: errorInjectionCh, + PreExit: func() { + mu.Lock() + mu.rangeFeedRunning = false + mu.Unlock() + // Block until the test directives indicate otherwise. + <-restartAfterErrCh + }, }, - OnTimestampAdvance: func(ts hlc.Timestamp) { + WatcherUpdatesInterceptor: func(UpdateType rangefeedcache.UpdateType, updates []tenantcapabilities.Update) { mu.Lock() defer mu.Unlock() - mu.lastFrontierTS = ts - }, - ErrorInjectionCh: errorInjectionCh, - PreExit: func() { - mu.Lock() - mu.rangeFeedRunning = false - mu.Unlock() - // Block until the test directives indicate otherwise. - <-restartAfterErrCh + mu.receivedUpdates = append(mu.receivedUpdates, updates...) + mu.receivedUpdateType = UpdateType }, }, - WatcherUpdatesInterceptor: func(UpdateType rangefeedcache.UpdateType, updates []tenantcapabilities.Update) { - mu.Lock() - defer mu.Unlock() - mu.receivedUpdates = append(mu.receivedUpdates, updates...) - mu.receivedUpdateType = UpdateType - }, }) var lastUpdateTS hlc.Timestamp @@ -234,7 +237,7 @@ func TestDataDriven(t *testing.T) { case "flush-state": var output strings.Builder - entries := watcher.testingFlushCapabilitiesState() + entries := watcher.TestingFlushCapabilitiesState() for _, entry := range entries { output.WriteString(fmt.Sprintf("%s\n", tenantcapabilitiestestutils.PrintTenantCapabilityEntry(entry))) } diff --git a/pkg/multitenant/tenantcapabilities/testingknobs.go b/pkg/multitenant/tenantcapabilities/testingknobs.go index c6b53afd067c..eeae79e90950 100644 --- a/pkg/multitenant/tenantcapabilities/testingknobs.go +++ b/pkg/multitenant/tenantcapabilities/testingknobs.go @@ -10,19 +10,20 @@ package tenantcapabilities -import ( - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" -) +import "github.com/cockroachdb/cockroach/pkg/base" // TestingKnobs contain testing helpers which are used by various components // that enable tenant capabilities. type TestingKnobs struct { - // WatcherRangeFeedKnobs control the lifecycle events for the underlying - // rangefeed of the tenant capabilities Watcher. - WatcherRangeFeedKnobs base.ModuleTestingKnobs + // WatcherTestingKnobs can be used to test the tenant capabilities Watcher. + WatcherTestingKnobs base.ModuleTestingKnobs - // WatcherUpdatesInterceptor, if set, is called each time the Watcher - // receives a set of updates. - WatcherUpdatesInterceptor func(updateType rangefeedcache.UpdateType, updates []Update) + // AuthorizerSkipAdminSplitCapabilityChecks, if set, skips capability checks + // for AdminSplit requests in the Authorizer for secondary tenants. + AuthorizerSkipAdminSplitCapabilityChecks bool } + +// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. +func (t *TestingKnobs) ModuleTestingKnobs() {} + +var _ base.ModuleTestingKnobs = (*TestingKnobs)(nil) diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index 93616e331149..884006042a71 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//pkg/base", "//pkg/clusterversion", "//pkg/keys", + "//pkg/multitenant/tenantcapabilities", "//pkg/roachpb", "//pkg/security", "//pkg/security/certnames", @@ -99,6 +100,8 @@ go_test( "//pkg/base", "//pkg/clusterversion", "//pkg/keys", + "//pkg/multitenant/tenantcapabilities", + "//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer", "//pkg/roachpb", "//pkg/security", "//pkg/security/certnames", diff --git a/pkg/rpc/auth.go b/pkg/rpc/auth.go index 1045c5e9a9dc..f1f8d0b96ace 100644 --- a/pkg/rpc/auth.go +++ b/pkg/rpc/auth.go @@ -79,7 +79,7 @@ func (a kvAuth) unaryInterceptor( // Handle authorization according to the selected authz method. switch ar := authz.(type) { case authzTenantServerToKVServer: - if err := a.tenant.authorize(roachpb.TenantID(ar), info.FullMethod, req); err != nil { + if err := a.tenant.authorize(ctx, roachpb.TenantID(ar), info.FullMethod, req); err != nil { return nil, err } case authzTenantServerToTenantServer: @@ -119,7 +119,7 @@ func (a kvAuth) streamInterceptor( return err } // 'm' is now populated and contains the request from the client. - return a.tenant.authorize(roachpb.TenantID(ar), info.FullMethod, m) + return a.tenant.authorize(ctx, roachpb.TenantID(ar), info.FullMethod, m) }, } case authzTenantServerToTenantServer: diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index abc74495e3f5..6b046b3a2b18 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -15,6 +15,7 @@ import ( "strconv" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" @@ -28,6 +29,10 @@ type tenantAuthorizer struct { // tenantID is the tenant ID for the current node. // Equals SystemTenantID when running a KV node. tenantID roachpb.TenantID + // capabilitiesAuthorizer is used to perform capability checks for incoming + // tenant requests. Capability checks are only performed when running on a KV + // node; the TenantRPCAuthorizer no-ops on secondary tenants. + capabilitiesAuthorizer tenantcapabilities.Authorizer } func tenantIDFromString(commonName, field string) (roachpb.TenantID, error) { @@ -44,11 +49,11 @@ func tenantIDFromString(commonName, field string) (roachpb.TenantID, error) { // authorize enforces a security boundary around endpoints that tenants // request from the host KV node or other tenant SQL pod. func (a tenantAuthorizer) authorize( - tenID roachpb.TenantID, fullMethod string, req interface{}, + ctx context.Context, tenID roachpb.TenantID, fullMethod string, req interface{}, ) error { switch fullMethod { case "/cockroach.roachpb.Internal/Batch": - return a.authBatch(tenID, req.(*roachpb.BatchRequest)) + return a.authBatch(ctx, tenID, req.(*roachpb.BatchRequest)) case "/cockroach.roachpb.Internal/RangeLookup": return a.authRangeLookup(tenID, req.(*roachpb.RangeLookupRequest)) @@ -156,9 +161,17 @@ func checkSpanBounds(rSpan, tenSpan roachpb.RSpan) error { // authBatch authorizes the provided tenant to invoke the Batch RPC with the // provided args. -func (a tenantAuthorizer) authBatch(tenID roachpb.TenantID, args *roachpb.BatchRequest) error { - // Consult reqAllowed to determine whether each request in the batch - // is permitted. If not, reject the entire batch. +func (a tenantAuthorizer) authBatch( + ctx context.Context, tenID roachpb.TenantID, args *roachpb.BatchRequest, +) error { + if err := a.capabilitiesAuthorizer.HasCapabilityForBatch(ctx, tenID, args); err != nil { + return authError(err.Error()) + } + + // TODO(ecwall): This list isn't exhaustive. For any request that isn't + // contained in here, there should be a corresponding capability. Once that's + // done, we can get rid of this loop entirely and perform all checks inside + // the capabilities Authorizer above. for _, ru := range args.Requests { switch ru.GetInner().(type) { case diff --git a/pkg/rpc/auth_test.go b/pkg/rpc/auth_test.go index 20aff433fd35..6914704ba912 100644 --- a/pkg/rpc/auth_test.go +++ b/pkg/rpc/auth_test.go @@ -20,12 +20,15 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" @@ -207,29 +210,46 @@ func TestAuthenticateTenant(t *testing.T) { } } +func prefix(tenID uint64, key string) string { + tenPrefix := keys.MakeTenantPrefix(roachpb.MustMakeTenantID(tenID)) + return string(append(tenPrefix, []byte(key)...)) +} + +func makeSpanShared(t *testing.T, key string, endKey ...string) roachpb.Span { + s := roachpb.Span{Key: roachpb.Key(key)} + if len(endKey) > 1 { + t.Fatalf("unexpected endKey vararg %v", endKey) + } else if len(endKey) == 1 { + s.EndKey = roachpb.Key(endKey[0]) + } + return s +} + +func makeReqShared(t *testing.T, key string, endKey ...string) roachpb.Request { + s := makeSpanShared(t, key, endKey...) + h := roachpb.RequestHeaderFromSpan(s) + return &roachpb.ScanRequest{RequestHeader: h} +} + +func makeReqs(reqs ...roachpb.Request) []roachpb.RequestUnion { + ru := make([]roachpb.RequestUnion, len(reqs)) + for i, r := range reqs { + ru[i].MustSetInner(r) + } + return ru +} + func TestTenantAuthRequest(t *testing.T) { defer leaktest.AfterTest(t)() tenID := roachpb.MustMakeTenantID(10) - prefix := func(tenID uint64, key string) string { - tenPrefix := keys.MakeTenantPrefix(roachpb.MustMakeTenantID(tenID)) - return string(append(tenPrefix, []byte(key)...)) - } makeSpan := func(key string, endKey ...string) roachpb.Span { - s := roachpb.Span{Key: roachpb.Key(key)} - if len(endKey) > 1 { - t.Fatalf("unexpected endKey vararg %v", endKey) - } else if len(endKey) == 1 { - s.EndKey = roachpb.Key(endKey[0]) - } - return s + return makeSpanShared(t, key, endKey...) } makeReq := func(key string, endKey ...string) roachpb.Request { - s := makeSpan(key, endKey...) - h := roachpb.RequestHeaderFromSpan(s) - return &roachpb.ScanRequest{RequestHeader: h} + return makeReqShared(t, key, endKey...) } makeDisallowedAdminReq := func(key string) roachpb.Request { - s := makeSpan(key) + s := makeSpanShared(t, key) h := roachpb.RequestHeader{Key: s.Key} return &roachpb.AdminMergeRequest{RequestHeader: h} } @@ -243,13 +263,6 @@ func TestTenantAuthRequest(t *testing.T) { h := roachpb.RequestHeaderFromSpan(s) return &roachpb.AdminScatterRequest{RequestHeader: h} } - makeReqs := func(reqs ...roachpb.Request) []roachpb.RequestUnion { - ru := make([]roachpb.RequestUnion, len(reqs)) - for i, r := range reqs { - ru[i].MustSetInner(r) - } - return ru - } makeSystemSpanConfigTarget := func(source, target uint64) roachpb.SpanConfigTarget { return roachpb.SpanConfigTarget{ Union: &roachpb.SpanConfigTarget_SystemSpanConfigTarget{ @@ -857,9 +870,12 @@ func TestTenantAuthRequest(t *testing.T) { }, } { t.Run(method, func(t *testing.T) { + ctx := context.Background() for _, tc := range tests { t.Run("", func(t *testing.T) { - err := rpc.TestingAuthorizeTenantRequest(tenID, method, tc.req) + err := rpc.TestingAuthorizeTenantRequest( + ctx, tenID, method, tc.req, tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) if tc.expErr == noError { require.NoError(t, err) } else { @@ -872,3 +888,74 @@ func TestTenantAuthRequest(t *testing.T) { }) } } + +// TestTenantAuthCapabilityChecks ensures capability checks are performed +// correctly by the tenant authorizer. +func TestTenantAuthCapabilityChecks(t *testing.T) { + defer leaktest.AfterTest(t)() + + tenID := roachpb.MustMakeTenantID(10) + for method, tests := range map[string][]struct { + req interface{} + configureAuthorizer func(authorizer *mockAuthorizer) + expErr string + }{ + "/cockroach.roachpb.Internal/Batch": { + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReqShared(t, prefix(10, "a"), prefix(10, "b")), + )}, + configureAuthorizer: func(authorizer *mockAuthorizer) { + authorizer.hasCapabilityForBatch = true + }, + expErr: "", + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReqShared(t, prefix(10, "a"), prefix(10, "b")), + )}, + configureAuthorizer: func(authorizer *mockAuthorizer) { + authorizer.hasCapabilityForBatch = false + }, + expErr: "tenant does not have capability", + }, + }, + } { + ctx := context.Background() + for _, tc := range tests { + authorizer := mockAuthorizer{} + tc.configureAuthorizer(&authorizer) + err := rpc.TestingAuthorizeTenantRequest( + ctx, tenID, method, tc.req, authorizer, + ) + if tc.expErr == "" { + require.NoError(t, err) + } else { + require.Error(t, err) + require.Equal(t, codes.Unauthenticated, status.Code(err)) + require.Regexp(t, tc.expErr, err) + } + } + } +} + +type mockAuthorizer struct { + hasCapabilityForBatch bool +} + +var _ tenantcapabilities.Authorizer = &mockAuthorizer{} + +// HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface. +func (m mockAuthorizer) HasCapabilityForBatch( + context.Context, roachpb.TenantID, *roachpb.BatchRequest, +) error { + if m.hasCapabilityForBatch { + return nil + } + return errors.New("tenant does not have capability") +} + +// BindReader implements the tenantcapabilities.Authorizer interface. +func (m mockAuthorizer) BindReader(tenantcapabilities.Reader) { + panic("unimplemented") +} diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 0d81a39e8299..0d3a42ab26e0 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -25,6 +25,7 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -245,7 +246,8 @@ func NewServerEx( a := kvAuth{ sv: &rpcCtx.Settings.SV, tenant: tenantAuthorizer{ - tenantID: rpcCtx.tenID, + tenantID: rpcCtx.tenID, + capabilitiesAuthorizer: rpcCtx.capabilitiesAuthorizer, }, } @@ -505,6 +507,11 @@ type ContextOptions struct { // the same process as a KV layer and thus should feel empowered // to use its node cert to perform outgoing RPC dials. UseNodeAuth bool + + // TenantRPCAuthorizer provides a handle into the tenantcapabilities + // subsystem. It allows KV nodes to perform capability checks for incoming + // tenant requests. + TenantRPCAuthorizer tenantcapabilities.Authorizer } func (c ContextOptions) validate() error { @@ -614,6 +621,7 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { opts.Config, security.ClusterTLSSettings(opts.Settings), opts.TenantID, + opts.TenantRPCAuthorizer, ) secCtx.useNodeAuth = opts.UseNodeAuth diff --git a/pkg/rpc/helpers_test.go b/pkg/rpc/helpers_test.go index e7b79e7a4ae9..acb01fa35977 100644 --- a/pkg/rpc/helpers_test.go +++ b/pkg/rpc/helpers_test.go @@ -13,6 +13,7 @@ package rpc import ( "context" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "google.golang.org/grpc" ) @@ -54,7 +55,14 @@ func TestingAuthenticateTenant( // TestingAuthorizeTenantRequest performs authorization of a tenant request // for testing. func TestingAuthorizeTenantRequest( - tenID roachpb.TenantID, method string, request interface{}, + ctx context.Context, + tenID roachpb.TenantID, + method string, + request interface{}, + authorizer tenantcapabilities.Authorizer, ) error { - return tenantAuthorizer{}.authorize(tenID, method, request) + return tenantAuthorizer{ + tenantID: tenID, + capabilitiesAuthorizer: authorizer, + }.authorize(ctx, tenID, method, request) } diff --git a/pkg/rpc/tls.go b/pkg/rpc/tls.go index e3fd28dd0fc9..052bc31c8dc1 100644 --- a/pkg/rpc/tls.go +++ b/pkg/rpc/tls.go @@ -23,6 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "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/certnames" @@ -58,9 +59,10 @@ func wrapError(err error) error { type SecurityContext struct { certnames.Locator security.TLSSettings - config *base.Config - tenID roachpb.TenantID - lazy struct { + config *base.Config + tenID roachpb.TenantID + capabilitiesAuthorizer tenantcapabilities.Authorizer + lazy struct { // The certificate manager. Must be accessed through GetCertificateManager. certificateManager lazyCertificateManager // httpClient uses the client TLS config. It is initialized lazily. @@ -73,16 +75,20 @@ type SecurityContext struct { // // TODO(tbg): don't take a whole Config. This can be trimmed down significantly. func NewSecurityContext( - cfg *base.Config, tlsSettings security.TLSSettings, tenID roachpb.TenantID, + cfg *base.Config, + tlsSettings security.TLSSettings, + tenID roachpb.TenantID, + capabilitiesAuthorizer tenantcapabilities.Authorizer, ) *SecurityContext { if tenID.ToUint64() == 0 { panic(errors.AssertionFailedf("programming error: tenant ID not defined")) } return &SecurityContext{ - Locator: certnames.MakeLocator(cfg.SSLCertsDir), - TLSSettings: tlsSettings, - config: cfg, - tenID: tenID, + Locator: certnames.MakeLocator(cfg.SSLCertsDir), + TLSSettings: tlsSettings, + config: cfg, + tenID: tenID, + capabilitiesAuthorizer: capabilitiesAuthorizer, } } diff --git a/pkg/security/BUILD.bazel b/pkg/security/BUILD.bazel index b7f8d712ad63..234dc5ae8b5b 100644 --- a/pkg/security/BUILD.bazel +++ b/pkg/security/BUILD.bazel @@ -74,6 +74,7 @@ go_test( embed = [":security"], deps = [ "//pkg/base", + "//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer", "//pkg/roachpb", "//pkg/rpc", "//pkg/security/certnames", diff --git a/pkg/security/certs_rotation_test.go b/pkg/security/certs_rotation_test.go index 1487313fac90..48508400dda4 100644 --- a/pkg/security/certs_rotation_test.go +++ b/pkg/security/certs_rotation_test.go @@ -27,6 +27,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" @@ -105,7 +106,12 @@ func TestRotateCerts(t *testing.T) { // Test client with the same certs. clientContext := testutils.NewNodeTestBaseContext() clientContext.SSLCertsDir = certsDir - firstSCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + firstSCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) firstClient, err := firstSCtx.GetHTTPClient() if err != nil { t.Fatalf("could not create http client: %v", err) @@ -137,7 +143,12 @@ func TestRotateCerts(t *testing.T) { clientContext = testutils.NewNodeTestBaseContext() clientContext.SSLCertsDir = certsDir - secondSCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + secondSCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) secondClient, err := secondSCtx.GetHTTPClient() if err != nil { t.Fatalf("could not create http client: %v", err) @@ -246,7 +257,12 @@ func TestRotateCerts(t *testing.T) { // This is HTTP and succeeds because we do not ask for or verify client certificates. clientContext = testutils.NewNodeTestBaseContext() clientContext.SSLCertsDir = certsDir - thirdSCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + thirdSCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) thirdClient, err := thirdSCtx.GetHTTPClient() if err != nil { t.Fatalf("could not create http client: %v", err) diff --git a/pkg/security/certs_test.go b/pkg/security/certs_test.go index 2da9a9390918..1a1dbb7d91a4 100644 --- a/pkg/security/certs_test.go +++ b/pkg/security/certs_test.go @@ -24,6 +24,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" @@ -399,7 +400,12 @@ func TestUseCerts(t *testing.T) { // Insecure mode. clientContext := testutils.NewNodeTestBaseContext() clientContext.Insecure = true - sCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + sCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) httpClient, err := sCtx.GetHTTPClient() if err != nil { t.Fatal(err) @@ -419,7 +425,12 @@ func TestUseCerts(t *testing.T) { clientContext = testutils.NewNodeTestBaseContext() clientContext.SSLCertsDir = certsDir { - secondSCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + secondSCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) httpClient, err = secondSCtx.GetHTTPClient() } if err != nil { @@ -481,7 +492,12 @@ func TestUseSplitCACerts(t *testing.T) { // Insecure mode. clientContext := testutils.NewNodeTestBaseContext() clientContext.Insecure = true - sCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + sCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) httpClient, err := sCtx.GetHTTPClient() if err != nil { t.Fatal(err) @@ -501,7 +517,12 @@ func TestUseSplitCACerts(t *testing.T) { clientContext = testutils.NewNodeTestBaseContext() clientContext.SSLCertsDir = certsDir { - secondSCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + secondSCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) httpClient, err = secondSCtx.GetHTTPClient() } if err != nil { @@ -599,7 +620,12 @@ func TestUseWrongSplitCACerts(t *testing.T) { // Insecure mode. clientContext := testutils.NewNodeTestBaseContext() clientContext.Insecure = true - sCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + sCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) httpClient, err := sCtx.GetHTTPClient() if err != nil { t.Fatal(err) @@ -619,7 +645,12 @@ func TestUseWrongSplitCACerts(t *testing.T) { clientContext = testutils.NewNodeTestBaseContext() clientContext.SSLCertsDir = certsDir { - secondCtx := rpc.NewSecurityContext(clientContext, security.CommandTLSSettings{}, roachpb.SystemTenantID) + secondCtx := rpc.NewSecurityContext( + clientContext, + security.CommandTLSSettings{}, + roachpb.SystemTenantID, + tenantcapabilitiesauthorizer.NewNoopAuthorizer(), + ) httpClient, err = secondCtx.GetHTTPClient() } if err != nil { diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 3bf5af275e90..4d07989f7478 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -137,6 +137,9 @@ go_library( "//pkg/multitenant/mtinfopb", "//pkg/multitenant/multitenantcpu", "//pkg/multitenant/multitenantio", + "//pkg/multitenant/tenantcapabilities", + "//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer", + "//pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher", "//pkg/multitenant/tenantcostmodel", "//pkg/obs", "//pkg/roachpb", diff --git a/pkg/server/server.go b/pkg/server/server.go index ea46d5810d41..a2be72378e47 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -50,6 +50,9 @@ import ( serverrangefeed "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangelog" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/reports" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher" "github.com/cockroachdb/cockroach/pkg/obs" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -171,6 +174,8 @@ type Server struct { spanConfigSubscriber spanconfig.KVSubscriber spanConfigReporter spanconfig.Reporter + tenantCapabilitiesWatcher tenantcapabilities.Watcher + // pgL is the SQL listener. pgL net.Listener // loopbackPgL is the SQL listener for internal pgwire connections. @@ -287,6 +292,8 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { &cfg.DefaultZoneConfig, ) + tenantCapabilitiesTestingKnobs, _ := cfg.TestingKnobs.TenantCapabilitiesTestingKnobs.(*tenantcapabilities.TestingKnobs) + authorizer := tenantcapabilitiesauthorizer.New(cfg.Settings, tenantCapabilitiesTestingKnobs) rpcCtxOpts := rpc.ContextOptions{ TenantID: roachpb.SystemTenantID, UseNodeAuth: true, @@ -314,6 +321,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // operations should fail immediately. return checkPingFor(ctx, req.OriginNodeID, codes.PermissionDenied) }, + TenantRPCAuthorizer: authorizer, } if knobs := cfg.TestingKnobs.Server; knobs != nil { serverKnobs := knobs.(*TestingKnobs) @@ -778,6 +786,15 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { clock, rangeFeedFactory, stopper, st, ) + tenantCapabilitiesWatcher := tenantcapabilitieswatcher.New( + clock, + rangeFeedFactory, + keys.TenantsTableID, + stopper, + 1<<20, /* 1 MB */ + tenantCapabilitiesTestingKnobs, + ) + node := NewNode( storeCfg, recorder, @@ -1091,55 +1108,56 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { ) *lateBoundServer = Server{ - nodeIDContainer: nodeIDContainer, - cfg: cfg, - st: st, - clock: clock, - rpcContext: rpcContext, - engines: engines, - grpc: grpcServer, - gossip: g, - nodeDialer: nodeDialer, - nodeLiveness: nodeLiveness, - storePool: storePool, - tcsFactory: tcsFactory, - distSender: distSender, - db: db, - node: node, - registry: registry, - recorder: recorder, - ruleRegistry: ruleRegistry, - promRuleExporter: promRuleExporter, - updates: updates, - ctSender: ctSender, - runtime: runtimeSampler, - http: sHTTP, - adminAuthzCheck: adminAuthzCheck, - admin: sAdmin, - status: sStatus, - drain: drain, - decomNodeMap: decomNodeMap, - authentication: sAuth, - tsDB: tsDB, - tsServer: &sTS, - eventsExporter: eventsExporter, - recoveryServer: recoveryServer, - raftTransport: raftTransport, - stopper: stopper, - stopTrigger: stopTrigger, - debug: debugServer, - kvProber: kvProber, - replicationReporter: replicationReporter, - protectedtsProvider: protectedtsProvider, - spanConfigSubscriber: spanConfig.subscriber, - spanConfigReporter: spanConfig.reporter, - pgPreServer: pgPreServer, - sqlServer: sqlServer, - serverController: sc, - externalStorageBuilder: externalStorageBuilder, - storeGrantCoords: gcoords.Stores, - kvMemoryMonitor: kvMemoryMonitor, - spanStatsServer: spanStatsServer, + nodeIDContainer: nodeIDContainer, + cfg: cfg, + st: st, + clock: clock, + rpcContext: rpcContext, + engines: engines, + grpc: grpcServer, + gossip: g, + nodeDialer: nodeDialer, + nodeLiveness: nodeLiveness, + storePool: storePool, + tcsFactory: tcsFactory, + distSender: distSender, + db: db, + node: node, + registry: registry, + recorder: recorder, + ruleRegistry: ruleRegistry, + promRuleExporter: promRuleExporter, + updates: updates, + ctSender: ctSender, + runtime: runtimeSampler, + http: sHTTP, + adminAuthzCheck: adminAuthzCheck, + admin: sAdmin, + status: sStatus, + drain: drain, + decomNodeMap: decomNodeMap, + authentication: sAuth, + tsDB: tsDB, + tsServer: &sTS, + eventsExporter: eventsExporter, + recoveryServer: recoveryServer, + raftTransport: raftTransport, + stopper: stopper, + stopTrigger: stopTrigger, + debug: debugServer, + kvProber: kvProber, + replicationReporter: replicationReporter, + protectedtsProvider: protectedtsProvider, + spanConfigSubscriber: spanConfig.subscriber, + spanConfigReporter: spanConfig.reporter, + tenantCapabilitiesWatcher: tenantCapabilitiesWatcher, + pgPreServer: pgPreServer, + sqlServer: sqlServer, + serverController: sc, + externalStorageBuilder: externalStorageBuilder, + storeGrantCoords: gcoords.Stores, + kvMemoryMonitor: kvMemoryMonitor, + spanStatsServer: spanStatsServer, } return lateBoundServer, err @@ -1926,6 +1944,13 @@ func (s *Server) PreStart(ctx context.Context) error { if err := s.node.tenantSettingsWatcher.Start(workersCtx, s.sqlServer.execCfg.SystemTableIDResolver); err != nil { return errors.Wrap(err, "failed to initialize the tenant settings watcher") } + if err := s.tenantCapabilitiesWatcher.Start(ctx); err != nil { + return errors.Wrap(err, "initializing tenant capabilities") + } + // Now that we've got the tenant capabilities subsystem all started, we bind + // the Reader to the TenantRPCAuthorizer, so that it has a handle into the + // global tenant capabilities state. + s.rpcContext.TenantRPCAuthorizer.BindReader(s.tenantCapabilitiesWatcher) if err := s.kvProber.Start(workersCtx, s.stopper); err != nil { return errors.Wrapf(err, "failed to start KV prober") diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index ae8db7073c4c..e607ddeefff8 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -344,7 +344,7 @@ type sqlServerArgs struct { // the node/cluster and cancel sessions/queries. sqlStatusServer serverpb.SQLStatusServer - // Used to watch settings and descriptor changes. + // Used to construct rangefeeds. rangeFeedFactory *rangefeed.Factory // Used to query status information useful for debugging on the server. diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 3d14d56132c7..5b9c6844ad68 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptreconcile" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/multitenantcpu" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" "github.com/cockroachdb/cockroach/pkg/obs" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -856,17 +857,20 @@ func makeTenantSQLServerArgs( if p, ok := baseCfg.TestingKnobs.Server.(*TestingKnobs); ok { rpcTestingKnobs = p.ContextTestingKnobs } + + authorizer := tenantcapabilitiesauthorizer.NewNoopAuthorizer() rpcContext := rpc.NewContext(startupCtx, rpc.ContextOptions{ - TenantID: sqlCfg.TenantID, - UseNodeAuth: sqlCfg.LocalKVServerInfo != nil, - NodeID: baseCfg.IDContainer, - StorageClusterID: baseCfg.ClusterIDContainer, - Config: baseCfg.Config, - Clock: clock.WallClock(), - MaxOffset: clock.MaxOffset(), - Stopper: stopper, - Settings: st, - Knobs: rpcTestingKnobs, + TenantID: sqlCfg.TenantID, + UseNodeAuth: sqlCfg.LocalKVServerInfo != nil, + NodeID: baseCfg.IDContainer, + StorageClusterID: baseCfg.ClusterIDContainer, + Config: baseCfg.Config, + Clock: clock.WallClock(), + MaxOffset: clock.MaxOffset(), + Stopper: stopper, + Settings: st, + Knobs: rpcTestingKnobs, + TenantRPCAuthorizer: authorizer, }) if !baseCfg.Insecure { diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 8c00bf7abb55..5184d945740b 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -709,6 +709,7 @@ go_test( "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/protectedts", + "//pkg/multitenant/tenantcapabilities", "//pkg/roachpb", "//pkg/rpc", "//pkg/rpc/nodedialer", diff --git a/pkg/sql/logictest/BUILD.bazel b/pkg/sql/logictest/BUILD.bazel index d1974b9be6aa..4db5f1c9c926 100644 --- a/pkg/sql/logictest/BUILD.bazel +++ b/pkg/sql/logictest/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", + "//pkg/multitenant/tenantcapabilities", "//pkg/roachpb", "//pkg/security/username", "//pkg/server", diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 5dae16b33f46..43805e32ce6d 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -47,6 +47,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server" @@ -214,6 +215,16 @@ import ( // - tracing-off: If specified, tracing defaults to being turned off. This is // used to override the environment, which may ask for tracing to be on by // default. +// TODO(ecwall): We already have a tenant-capability-override-opt directive, +// and ideally, this option would be better suited in that category. However, +// that thing doesn't actually work. In particular, tenant capabilities are +// checked against an eventually consistent, in-memory state. However, that +// directive makes no attempt to ensure the in-memory state is sufficiently +// caught up. We should probably get rid of this cluster option once that +// directive is fixed. +// - can-admin-split: If specified, allows secondary tenants to perform +// AdminSplit operations regardless of the underlying tenant capabilities +// state. // // // ########################################### @@ -1924,6 +1935,22 @@ func (c clusterOptTracingOff) apply(args *base.TestServerArgs) { args.TracingDefault = tracing.TracingModeOnDemand } +// clusterOptAllowAdminSplitsForSecondaryTenants overrides can_admin_split capability +// checks using the AuthorizerOverrideCapabilities testing knob. +type clusterOptAllowAdminSplitsForSecondaryTenants struct{} + +// apply implements the knobOpt interface. +func (a clusterOptAllowAdminSplitsForSecondaryTenants) apply(args *base.TestServerArgs) { + _, ok := args.Knobs.TenantCapabilitiesTestingKnobs.(*tenantcapabilities.TestingKnobs) + if !ok { + args.Knobs.TenantCapabilitiesTestingKnobs = &tenantcapabilities.TestingKnobs{} + } + args.Knobs.TenantCapabilitiesTestingKnobs.(*tenantcapabilities.TestingKnobs). + AuthorizerSkipAdminSplitCapabilityChecks = true +} + +var _ clusterOpt = clusterOptAllowAdminSplitsForSecondaryTenants{} + // knobOpt is implemented by options for configuring the testing knobs // for the cluster under which a test will run. type knobOpt interface { @@ -2106,6 +2133,8 @@ func readClusterOptions(t *testing.T, path string) []clusterOpt { res = append(res, clusterOptTracingOff{}) case "ignore-tenant-strict-gc-enforcement": res = append(res, clusterOptIgnoreStrictGCForTenants{}) + case "can-admin-split": + res = append(res, clusterOptAllowAdminSplitsForSecondaryTenants{}) default: t.Fatalf("unrecognized cluster option: %s", opt) } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index aa55f110b85a..05c4cbea0f87 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -1,3 +1,5 @@ +# cluster-opt: can-admin-split + statement ok CREATE TABLE t (x INT PRIMARY KEY, y INT NOT NULL, z INT NOT NULL, w INT, INDEX i (x), INDEX i2 (z)) diff --git a/pkg/sql/logictest/testdata/logic_test/create_index b/pkg/sql/logictest/testdata/logic_test/create_index index bca10b313251..5f4e197f2596 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_index +++ b/pkg/sql/logictest/testdata/logic_test/create_index @@ -1,4 +1,5 @@ # LogicTest: default-configs local-mixed-22.2-23.1 +# cluster-opt: can-admin-split statement ok CREATE TABLE t ( diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality b/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality index 4f21408ab4df..552f90d6b688 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality +++ b/pkg/sql/logictest/testdata/logic_test/distsql_tenant_locality @@ -1,3 +1,4 @@ +# cluster-opt: can-admin-split # LogicTest: 3node-tenant-multiregion # tenant-cluster-setting-override-opt: sql.split_at.allow_for_secondary_tenant.enabled=true sql.zone_configs.allow_for_secondary_tenant.enabled=true sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true diff --git a/pkg/sql/logictest/testdata/logic_test/hash_sharded_index b/pkg/sql/logictest/testdata/logic_test/hash_sharded_index index 11e47b3cd31c..7419a82ef55e 100644 --- a/pkg/sql/logictest/testdata/logic_test/hash_sharded_index +++ b/pkg/sql/logictest/testdata/logic_test/hash_sharded_index @@ -1,3 +1,5 @@ +# cluster-opt: can-admin-split + # Tests for creating a hash sharded primary key statement ok CREATE TABLE sharded_primary (a INT PRIMARY KEY USING HASH WITH (bucket_count=10)) diff --git a/pkg/sql/logictest/testdata/logic_test/sql_keys b/pkg/sql/logictest/testdata/logic_test/sql_keys index 0d5a1ed6ccfc..0bed4efcf91f 100644 --- a/pkg/sql/logictest/testdata/logic_test/sql_keys +++ b/pkg/sql/logictest/testdata/logic_test/sql_keys @@ -1,5 +1,6 @@ # LogicTest: local 3node-tenant # tenant-cluster-setting-override-opt: sql.split_at.allow_for_secondary_tenant.enabled=true +# cluster-opt: can-admin-split # This test depends on table ID's being stable, so add new tests at the bottom # of the file. diff --git a/pkg/sql/multitenant_admin_function_test.go b/pkg/sql/multitenant_admin_function_test.go index 3cf7f28abe42..c0544fc66ce5 100644 --- a/pkg/sql/multitenant_admin_function_test.go +++ b/pkg/sql/multitenant_admin_function_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "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/server" "github.com/cockroachdb/cockroach/pkg/settings" @@ -30,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -353,6 +355,10 @@ func TestMultiTenantAdminFunction(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + // TODO(ecwall): This test doesn't work now that capability checks exist. + // We should migrate what is being tested to a logictest instead. + skip.WithIssue(t, 96739) + testCases := []testCase{ { desc: "ALTER RANGE x RELOCATE LEASE", @@ -549,7 +555,8 @@ func TestMultiTenantAdminFunction(t *testing.T) { } } -// TestTruncateTable tests that range splits are retained after a table is truncated. +// TestTruncateTable tests that range splits are retained after a table is +// truncated. func TestTruncateTable(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -579,6 +586,15 @@ func TestTruncateTable(t *testing.T) { t, func() testClusterCfg { return testClusterCfg{ + TestClusterArgs: base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + TenantCapabilitiesTestingKnobs: &tenantcapabilities.TestingKnobs{ + AuthorizerSkipAdminSplitCapabilityChecks: true, + }, + }, + }, + }, setupClusterSetting: sql.SecondaryTenantSplitAtEnabled, } }, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_tenant_locality b/pkg/sql/opt/exec/execbuilder/testdata/distsql_tenant_locality index 45a60377b6ff..28d54a6991ee 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_tenant_locality +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_tenant_locality @@ -1,5 +1,6 @@ # LogicTest: 3node-tenant-multiregion # tenant-cluster-setting-override-opt: sql.split_at.allow_for_secondary_tenant.enabled=true sql.zone_configs.allow_for_secondary_tenant.enabled=true sql.multi_region.allow_abstractions_for_secondary_tenants.enabled=true +# cluster-opt: can-admin-split # Create a table on the secondary tenant. statement ok