From cfa437560c93dd07fae2ac804c757842eca5d369 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Mon, 20 Mar 2023 14:10:06 +0000 Subject: [PATCH] serverutils: prevent start tenant when probabilistic behaviour is active The probabilistic behaviour that maybe starts a test tenant currently present in test server and test cluster is not widely known or apparent. This change seeks to alleviate that by warning, via an error, new test authors that if they start a tenant in a test there is the possibility that a default test tenant could also be started. This is so that the author of a test can make an informed decision whether to control the starting of tenants or allow probabilistic behaviour to take effect. Previously there was a `DisableDefaultTestTenant` test arg for tests that wanted to opt out of the probabilistic behaviour. This argument has been augmented to now support multiple modes for more flexibility. Tests can now choose to rely on the probabilistic test tenant, or create its own tenants, or both. The main purpose of the default mode which disallows the starting of tenants is to enlighten test authors of the possible presence of the default test tenant. It is still possible to have the probabilistic behaviour and start tenants in the test as well, but it has to be explicitly set in the arguments if the test author seeks this behaviour. Numerous tests that currently start the test tenant in the test itself, had this behaviour. Those tests have been updated to disable the default test tenant from probabilistically starting. As these tests would now have raised an error (warning) without changes. Epic: CRDB-18499 --- pkg/base/test_server_args.go | 36 +++++++++-- pkg/bench/foreachdb.go | 12 ++-- .../backupccl/alter_backup_schedule_test.go | 2 +- pkg/ccl/backupccl/backup_tenant_test.go | 2 +- pkg/ccl/backupccl/backup_test.go | 64 ++++++++++--------- .../backupccl/backuprand/backup_rand_test.go | 6 +- .../backupccl/backuptestutils/testutils.go | 4 +- .../backupccl/create_scheduled_backup_test.go | 2 +- pkg/ccl/backupccl/datadriven_test.go | 45 ++++++------- .../full_cluster_backup_restore_test.go | 14 ++-- .../restore_mid_schema_change_test.go | 4 +- .../backupccl/restore_old_versions_test.go | 6 +- pkg/ccl/backupccl/system_schema_test.go | 2 +- .../backupccl/tenant_backup_nemesis_test.go | 4 +- pkg/ccl/backupccl/utils_test.go | 6 +- .../changefeedccl/alter_changefeed_test.go | 2 +- pkg/ccl/changefeedccl/changefeed_test.go | 18 +++--- pkg/ccl/changefeedccl/helpers_test.go | 8 +-- .../scheduled_changefeed_test.go | 2 +- .../table_event_filter_datadriven_test.go | 2 +- .../cloudccl/externalconn/datadriven_test.go | 1 + pkg/ccl/importerccl/ccl_test.go | 14 +++- .../jobs_protected_ts_test.go | 4 +- .../boundedstaleness_test.go | 2 +- .../kvfollowerreadsccl/followerreads_test.go | 12 ++-- pkg/ccl/kvccl/kvtenantccl/tenant_kv_test.go | 2 +- .../kvtenantccl/tenant_range_lookup_test.go | 2 +- .../tenant_scan_range_descriptors_test.go | 2 +- .../kvccl/kvtenantccl/tenant_trace_test.go | 2 +- .../kvccl/kvtenantccl/tenant_upgrade_test.go | 12 ++-- .../multiregionccl/cold_start_latency_test.go | 4 +- pkg/ccl/multiregionccl/datadriven_test.go | 2 +- .../multiregionccltestutils/testutils.go | 2 +- .../multiregionccl/regional_by_row_test.go | 2 +- pkg/ccl/multiregionccl/unique_test.go | 2 +- .../capabilities_test.go | 2 +- .../query_ru_estimate_test.go | 4 +- .../tenantcostclient/tenant_side_test.go | 14 ++-- pkg/ccl/partitionccl/drop_test.go | 8 +-- pkg/ccl/partitionccl/partition_test.go | 4 +- pkg/ccl/partitionccl/scrub_test.go | 4 +- pkg/ccl/partitionccl/zone_test.go | 4 +- pkg/ccl/serverccl/admin_test.go | 14 ++-- .../serverccl/diagnosticsccl/reporter_test.go | 10 ++- pkg/ccl/serverccl/role_authentication_test.go | 2 +- pkg/ccl/serverccl/server_controller_test.go | 22 +++---- pkg/ccl/serverccl/server_sql_test.go | 30 ++++++--- .../server_startup_guardrails_test.go | 4 +- .../serverccl/shared_process_tenant_test.go | 2 +- .../serverccl/statusccl/tenant_status_test.go | 2 +- .../tenant_decommissioned_host_test.go | 5 +- pkg/ccl/serverccl/tenant_migration_test.go | 8 +-- pkg/ccl/serverccl/tenant_test_utils.go | 2 +- .../kvaccessor_test.go | 2 +- .../spanconfiglimiterccl/datadriven_test.go | 1 + .../spanconfiglimiterccl/drop_table_test.go | 4 +- .../datadriven_test.go | 2 +- .../spanconfigsplitterccl/datadriven_test.go | 2 +- .../datadriven_test.go | 2 +- .../sqlwatcher_test.go | 10 +-- pkg/ccl/sqlproxyccl/backend_dialer_test.go | 2 +- pkg/ccl/sqlproxyccl/proxy_handler_test.go | 18 +++--- .../tenant/directory_cache_test.go | 2 +- .../replicationtestutils/testutils.go | 2 +- .../partitioned_stream_client_test.go | 2 +- .../replication_random_client_test.go | 2 +- .../streamingest/stream_ingestion_job_test.go | 12 ++-- .../streamproducer/producer_job_test.go | 2 +- .../streamproducer/replication_stream_test.go | 10 +-- pkg/ccl/testccl/sqlccl/run_control_test.go | 2 +- .../testccl/sqlccl/temp_table_clean_test.go | 4 +- pkg/ccl/testccl/sqlccl/tenant_gc_test.go | 3 +- pkg/ccl/testccl/sqlstatsccl/sql_stats_test.go | 2 +- pkg/ccl/testutilsccl/alter_primary_key.go | 2 +- pkg/ccl/workloadccl/allccl/all_test.go | 10 +-- pkg/cli/democluster/demo_cluster.go | 2 +- pkg/cli/democluster/demo_cluster_test.go | 4 +- .../rangefeed/rangefeedcache/watcher_test.go | 2 +- pkg/kv/kvserver/client_tenant_test.go | 4 ++ pkg/kv/kvserver/gc/gc_int_test.go | 2 +- pkg/kv/kvserver/replicate_queue_test.go | 10 +-- .../tenantcapabilitieswatcher/decoder_test.go | 2 +- pkg/server/admin_test.go | 62 +++++++++--------- pkg/server/server_controller_test.go | 4 +- .../test_system_config_watcher.go | 2 +- pkg/server/testserver.go | 29 ++++++++- .../spanconfigkvaccessor/kvaccessor_test.go | 4 +- pkg/sql/backfill_protected_timestamp_test.go | 4 +- .../internal/catkv/catalog_reader_test.go | 2 +- pkg/sql/colenc/encode_test.go | 2 +- pkg/sql/conn_executor_test.go | 3 +- pkg/sql/crdb_internal_test.go | 2 +- pkg/sql/generate_objects_test.go | 2 +- pkg/sql/importer/exportcsv_test.go | 8 +-- pkg/sql/importer/exportparquet_test.go | 12 ++-- pkg/sql/importer/import_into_test.go | 2 +- pkg/sql/importer/import_processor_test.go | 4 +- pkg/sql/importer/import_stmt_test.go | 38 +++++------ pkg/sql/importer/read_import_mysql_test.go | 4 +- pkg/sql/logictest/logic.go | 18 ++++-- pkg/sql/logictest/logictestbase/BUILD.bazel | 1 + .../logictest/logictestbase/logictestbase.go | 25 ++++---- pkg/sql/multitenant_admin_function_test.go | 1 + pkg/sql/run_control_test.go | 1 + pkg/sql/schemachanger/scbuild/builder_test.go | 2 +- pkg/sql/schemachanger/scdecomp/decomp_test.go | 2 +- pkg/sql/schemachanger/sctest/end_to_end.go | 4 +- .../instancestorage/instancecache_test.go | 2 +- pkg/sql/sqltestutils/telemetry.go | 1 + pkg/sql/tests/server_params.go | 2 +- pkg/sql/ttl/ttljob/ttljob_test.go | 8 +++ pkg/testutils/serverutils/test_server_shim.go | 40 +++++++++--- pkg/testutils/testcluster/testcluster.go | 12 +++- pkg/ts/server_test.go | 2 +- pkg/util/tracing/collector/collector_test.go | 2 +- 115 files changed, 508 insertions(+), 370 deletions(-) diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 2b2edebadfab..863b57bb7868 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -151,13 +151,13 @@ type TestServerArgs struct { // config span. DisableSpanConfigs bool - // TestServer will probabilistically start a single test tenant on each - // node for multi-tenant testing, and default all connections through that - // tenant. Use this flag to disable that behavior. You might want/need to - // disable this behavior if your test case is already leveraging tenants, - // or if some of the functionality being tested is not accessible from - // within tenants. - DisableDefaultTestTenant bool + // TestServer will probabilistically start a single test tenant on each node + // for multi-tenant testing, and default all connections through that tenant. + // Use this flag to change this behavior. You might want/need to alter this + // behavior if your test case is already leveraging tenants, or if some of the + // functionality being tested is not accessible from within tenants. See + // DefaultTestTenantOptions for alternative options that suits your test case. + DefaultTestTenant DefaultTestTenantOptions // StartDiagnosticsReporting checks cluster.TelemetryOptOut(), and // if not disabled starts the asynchronous goroutine that checks for @@ -207,6 +207,28 @@ type TestClusterArgs struct { ReusableListeners bool } +// DefaultTestTenantOptions specifies the conditions under which the default +// test tenant will be started. +type DefaultTestTenantOptions int + +const ( + // TestTenantProbabilisticOnly will start the default test tenant on a + // probabilistic basis. It will also prevent the starting of additional + // tenants by raising an error if it is attempted. + // This is the default behavior. + TestTenantProbabilisticOnly DefaultTestTenantOptions = iota + // TestTenantProbabilistic will start the default test tenant on a + // probabilistic basis. It allows the starting of additional tenants. + TestTenantProbabilistic + // TestTenantEnabled will always start the default test tenant. This is useful + // for quickly verifying that a test works with tenants enabled. + TestTenantEnabled + // TestTenantDisabled will disable the implicit starting of the default test + // tenant. This is useful for tests that want to explicitly control the + // starting of tenants, or currently don't work with tenants. + TestTenantDisabled +) + var ( // DefaultTestStoreSpec is just a single in memory store of 512 MiB // with no special attributes. diff --git a/pkg/bench/foreachdb.go b/pkg/bench/foreachdb.go index 931a950f10d0..378efec1fa0a 100644 --- a/pkg/bench/foreachdb.go +++ b/pkg/bench/foreachdb.go @@ -46,8 +46,8 @@ type BenchmarkFn func(b *testing.B, db *sqlutils.SQLRunner) func benchmarkCockroach(b *testing.B, f BenchmarkFn) { s, db, _ := serverutils.StartServer( b, base.TestServerArgs{ - UseDatabase: "bench", - DisableDefaultTestTenant: true, + UseDatabase: "bench", + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.TODO()) @@ -65,7 +65,7 @@ func benchmarkSharedProcessTenantCockroach(b *testing.B, f BenchmarkFn) { ctx := context.Background() s, db, _ := serverutils.StartServer( b, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) @@ -114,7 +114,7 @@ func benchmarkSepProcessTenantCockroach(b *testing.B, f BenchmarkFn) { ctx := context.Background() s, db, _ := serverutils.StartServer( b, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) @@ -142,8 +142,8 @@ func benchmarkMultinodeCockroach(b *testing.B, f BenchmarkFn) { base.TestClusterArgs{ ReplicationMode: base.ReplicationAuto, ServerArgs: base.TestServerArgs{ - UseDatabase: "bench", - DisableDefaultTestTenant: true, + UseDatabase: "bench", + DefaultTestTenant: base.TestTenantDisabled, }, }) if _, err := tc.Conns[0].Exec(`CREATE DATABASE bench`); err != nil { diff --git a/pkg/ccl/backupccl/alter_backup_schedule_test.go b/pkg/ccl/backupccl/alter_backup_schedule_test.go index 605a1caa959c..e2238f62deb0 100644 --- a/pkg/ccl/backupccl/alter_backup_schedule_test.go +++ b/pkg/ccl/backupccl/alter_backup_schedule_test.go @@ -71,7 +71,7 @@ func newAlterSchedulesTestHelper(t *testing.T) (*alterSchedulesTestHelper, func( ExternalIODir: dir, // Some scheduled backup tests fail when run within a tenant. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: knobs, }, diff --git a/pkg/ccl/backupccl/backup_tenant_test.go b/pkg/ccl/backupccl/backup_tenant_test.go index 3556bc7df150..22390c5fa045 100644 --- a/pkg/ccl/backupccl/backup_tenant_test.go +++ b/pkg/ccl/backupccl/backup_tenant_test.go @@ -43,7 +43,7 @@ func TestBackupTenantImportingTable(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test is designed to run with explicit tenants. No need to // implicitly create a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 37132ca7fa37..9c1260cbba69 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -247,7 +247,7 @@ func TestBackupRestoreJobTagAndLabel(t *testing.T) { tc, _, _, cleanupFn := backupRestoreTestSetupWithParams(t, numNodes, numAccounts, InitManualReplication, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ SetupFlowCb: func(ctx context.Context, _ base.SQLInstanceID, _ *execinfrapb.SetupFlowRequest) error { @@ -339,7 +339,7 @@ func TestBackupRestorePartitioned(t *testing.T) { args := base.TestClusterArgs{ ServerArgsPerNode: map[int]base.TestServerArgs{ 0: { - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "region", Value: "west"}, // NB: This has the same value as an az in the east region @@ -349,7 +349,7 @@ func TestBackupRestorePartitioned(t *testing.T) { }}, }, 1: { - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "region", Value: "east"}, // NB: This has the same value as an az in the west region @@ -359,7 +359,7 @@ func TestBackupRestorePartitioned(t *testing.T) { }}, }, 2: { - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "region", Value: "east"}, {Key: "az", Value: "az2"}, @@ -492,32 +492,32 @@ func TestBackupRestoreExecLocality(t *testing.T) { args := base.TestClusterArgs{ ServerArgsPerNode: map[int]base.TestServerArgs{ 0: { - ExternalIODir: "/west0", - DisableDefaultTestTenant: true, + ExternalIODir: "/west0", + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "0"}, {Key: "region", Value: "west"}, }}, }, 1: { - ExternalIODir: "/west1", - DisableDefaultTestTenant: true, + ExternalIODir: "/west1", + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "1"}, {Key: "region", Value: "west"}, }}, }, 2: { - ExternalIODir: "/east0", - DisableDefaultTestTenant: true, + ExternalIODir: "/east0", + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "0"}, {Key: "region", Value: "east"}, }}, }, 3: { - ExternalIODir: "/east1", - DisableDefaultTestTenant: true, + ExternalIODir: "/east1", + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "1"}, {Key: "region", Value: "east"}, @@ -1639,7 +1639,7 @@ func TestRestoreCheckpointing(t *testing.T) { }, JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), } - testServerArgs := base.TestServerArgs{DisableDefaultTestTenant: true} + testServerArgs := base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled} params.ServerArgs = testServerArgs params.ServerArgs.Knobs = knobs @@ -5409,7 +5409,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer cleanupFn() // Test fails when run within a tenant. More investigation is // required. Tracked with #76378. - args := base.TestServerArgs{ExternalIODir: dir, DisableDefaultTestTenant: true} + args := base.TestServerArgs{ExternalIODir: dir, DefaultTestTenant: base.TestTenantDisabled} // Setup for sequence ownership backup/restore tests in the same database. backupLoc := localFoo + `/d` @@ -5973,7 +5973,7 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) { params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() // This test instantiates its own secondary tenants below. No need to run // it probabilistically under a test tenant. - params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.DefaultTestTenant = base.TestTenantDisabled tc := testcluster.StartTestCluster(t, 1, params) defer tc.Stopper().Stop(ctx) @@ -6269,7 +6269,7 @@ func TestRestoreErrorPropagates(t *testing.T) { // below which is expected to fail, doesn't. This may be a problem with the // testing knobs being incorrectly applied to the cluster. More // investigation is required. Tracked with #76378. - params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.DefaultTestTenant = base.TestTenantDisabled jobsTableKey := keys.SystemSQLCodec.TablePrefix(uint32(systemschema.JobsTable.GetID())) var shouldFail, failures int64 params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ @@ -6345,7 +6345,7 @@ func TestProtectedTimestampsFailDueToLimits(t *testing.T) { params.ServerArgs.Knobs.ProtectedTS = &protectedts.TestingKnobs{ DisableProtectedTimestampForMultiTenant: true} // Test fails within a tenant. Tracked with #76378. - params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.DefaultTestTenant = base.TestTenantDisabled tc := testcluster.StartTestCluster(t, 1, params) defer tc.Stopper().Stop(ctx) db := tc.ServerConn(0) @@ -6378,7 +6378,7 @@ func TestPaginatedBackupTenant(t *testing.T) { Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, // Disabled to probabilistically spin up a tenant in each server because the // test explicitly sets up tenants to test on. - DisableDefaultTestTenant: true} + DefaultTestTenant: base.TestTenantDisabled} params := base.TestClusterArgs{ServerArgs: serverArgs} var numExportRequests int @@ -6856,7 +6856,7 @@ func TestBackupRestoreTenant(t *testing.T) { // Disabled to probabilistically spin up a tenant in each server because the // test explicitly sets up tenants to test on. - DisableDefaultTestTenant: true}, + DefaultTestTenant: base.TestTenantDisabled}, } const numAccounts = 1 @@ -6946,8 +6946,8 @@ func TestBackupRestoreTenant(t *testing.T) { // by creating a tenant. Furthermore, the test requires that // it run from the system tenant because it restores tenants. // Disable the default test tenant because it's not necessary. - DisableDefaultTestTenant: true, - Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, + DefaultTestTenant: base.TestTenantDisabled, + Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, }}, ) defer restoreTC.Stopper().Stop(ctx) @@ -7110,7 +7110,7 @@ func TestBackupRestoreTenant(t *testing.T) { // by creating a tenant. Furthermore, the test requires that // it run from the system tenant because it restores tenants. // Disable the default test tenant because it's not necessary. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}, ) defer restoreTC.Stopper().Stop(ctx) @@ -7164,7 +7164,7 @@ func TestBackupRestoreTenant(t *testing.T) { // it run from the system tenant because it queries the // system.tenants table. Disable the default test tenant because // it's not necessary. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}, ) @@ -7266,7 +7266,7 @@ func TestBackupRestoreTenant(t *testing.T) { // by creating a tenant. Furthermore, the test requires that // it run from the system tenant because it restores tenants. // Disable the default test tenant because it's not necessary. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}, ) defer restoreTC.Stopper().Stop(ctx) @@ -7291,7 +7291,7 @@ func TestBackupRestoreTenant(t *testing.T) { // by creating a tenant. Furthermore, the test requires that // it run from the system tenant because it restores tenants. // Disable the default test tenant because it's not necessary. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}, ) defer restoreTC.Stopper().Stop(ctx) @@ -8587,7 +8587,7 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { // Disable running within a tenant because expected index span is not received. // More investigation is necessary. // https://github.com/cockroachdb/cockroach/issues/88633F - serverArgs := base.TestServerArgs{DisableDefaultTestTenant: true} + serverArgs := base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled} serverArgs.Knobs = base.TestingKnobs{ // Configure knobs to block the index backfills. SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ @@ -9150,7 +9150,7 @@ func TestGCDropIndexSpanExpansion(t *testing.T) { // the cause of the hang is the fact that we're waiting on the GC to // complete, and we don't have visibility into the GC completing from // the tenant. More investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ GCJob: &sql.GCJobTestingKnobs{ RunBeforePerformGC: func(id jobspb.JobID) error { @@ -9323,7 +9323,7 @@ func TestExcludeDataFromBackupAndRestore(t *testing.T) { ServerArgs: base.TestServerArgs{ // Disabled to run within tenants because the function that sets up the restoring cluster // has not been configured yet to run within tenants. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test SpanConfig: &spanconfig.TestingKnobs{ @@ -9393,7 +9393,7 @@ func TestExportRequestBelowGCThresholdOnDataExcludedFromBackup(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails when run within a tenant as zone config // updates are not allowed by default. Tracked with 73768. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, } args.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ @@ -9488,7 +9488,7 @@ func TestExcludeDataFromBackupDoesNotHoldupGC(t *testing.T) { params.ServerArgs.ExternalIODir = dir // Test fails when run within a tenant. More investigation is // required. Tracked with #76378. - params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.DefaultTestTenant = base.TestTenantDisabled params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ DisableGCQueue: true, DisableLastProcessedCheck: true, @@ -9581,7 +9581,9 @@ func TestProtectRestoreTargets(t *testing.T) { numAccounts := 100 params := base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, + // Allow creating a different tenant if the default test tenant is not started. + DefaultTestTenant: base.TestTenantProbabilistic, + Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, }, } tc, sqlDB, tempDir, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, diff --git a/pkg/ccl/backupccl/backuprand/backup_rand_test.go b/pkg/ccl/backupccl/backuprand/backup_rand_test.go index dff8004e1f64..16864def8e05 100644 --- a/pkg/ccl/backupccl/backuprand/backup_rand_test.go +++ b/pkg/ccl/backupccl/backuprand/backup_rand_test.go @@ -49,9 +49,9 @@ func TestBackupRestoreRandomDataRoundtrips(t *testing.T) { ServerArgs: base.TestServerArgs{ // Fails with the default test tenant due to span limits. Tracked // with #76378. - DisableDefaultTestTenant: true, - UseDatabase: "rand", - ExternalIODir: dir, + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: "rand", + ExternalIODir: dir, }, } const localFoo = "nodelocal://1/foo/" diff --git a/pkg/ccl/backupccl/backuptestutils/testutils.go b/pkg/ccl/backupccl/backuptestutils/testutils.go index 0f8a8562b40c..21098b54716f 100644 --- a/pkg/ccl/backupccl/backuptestutils/testutils.go +++ b/pkg/ccl/backupccl/backuptestutils/testutils.go @@ -66,13 +66,13 @@ func backupRestoreTestSetupWithParams( dir, dirCleanupFn := testutils.TempDir(t) params.ServerArgs.ExternalIODir = dir params.ServerArgs.UseDatabase = "data" - params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.DefaultTestTenant = base.TestTenantDisabled if len(params.ServerArgsPerNode) > 0 { for i := range params.ServerArgsPerNode { param := params.ServerArgsPerNode[i] param.ExternalIODir = dir param.UseDatabase = "data" - param.DisableDefaultTestTenant = true + param.DefaultTestTenant = base.TestTenantDisabled params.ServerArgsPerNode[i] = param } } diff --git a/pkg/ccl/backupccl/create_scheduled_backup_test.go b/pkg/ccl/backupccl/create_scheduled_backup_test.go index d0ff22e51097..8b61d81ffdb0 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup_test.go +++ b/pkg/ccl/backupccl/create_scheduled_backup_test.go @@ -103,7 +103,7 @@ func newTestHelper(t *testing.T) (*testHelper, func()) { ExternalIODir: dir, // Some scheduled backup tests fail when run within a tenant. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: knobs, }, diff --git a/pkg/ccl/backupccl/datadriven_test.go b/pkg/ccl/backupccl/datadriven_test.go index 279ecc00da75..29ccbbc6a90f 100644 --- a/pkg/ccl/backupccl/datadriven_test.go +++ b/pkg/ccl/backupccl/datadriven_test.go @@ -134,15 +134,15 @@ func (d *datadrivenTestState) cleanup(ctx context.Context, t *testing.T) { } type clusterCfg struct { - name string - iodir string - nodes int - splits int - ioConf base.ExternalIODirConfig - localities string - beforeVersion string - testingKnobCfg string - disableTenant bool + name string + iodir string + nodes int + splits int + ioConf base.ExternalIODirConfig + localities string + beforeVersion string + testingKnobCfg string + defaultTestTenant base.DefaultTestTenantOptions } func (d *datadrivenTestState) addCluster(t *testing.T, cfg clusterCfg) error { @@ -150,7 +150,8 @@ func (d *datadrivenTestState) addCluster(t *testing.T, cfg clusterCfg) error { var cleanup func() params := base.TestClusterArgs{} params.ServerArgs.ExternalIODirConfig = cfg.ioConf - params.ServerArgs.DisableDefaultTestTenant = cfg.disableTenant + + params.ServerArgs.DefaultTestTenant = cfg.defaultTestTenant params.ServerArgs.Knobs = base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), } @@ -285,7 +286,7 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, name string, user string) * // - testingKnobCfg: specifies a key to a hardcoded testingKnob configuration // // - disable-tenant : ensures the test is never run in a multitenant environment by -// setting testserverargs.DisableDefaultTestTenant to true. +// setting testserverargs.DefaultTestTenant to base.TestTenantDisabled. // // - "upgrade-cluster version=" // Upgrade the cluster version of the active cluster to the passed in @@ -473,7 +474,7 @@ func TestDataDriven(t *testing.T) { case "new-cluster": var name, shareDirWith, iodir, localities, beforeVersion, testingKnobCfg string var splits int - var disableTenant bool + var defaultTestTenant base.DefaultTestTenantOptions nodes := singleNode var io base.ExternalIODirConfig d.ScanArgs(t, "name", &name) @@ -509,20 +510,20 @@ func TestDataDriven(t *testing.T) { d.ScanArgs(t, "testingKnobCfg", &testingKnobCfg) } if d.HasArg("disable-tenant") { - disableTenant = true + defaultTestTenant = base.TestTenantDisabled } lastCreatedCluster = name cfg := clusterCfg{ - name: name, - iodir: iodir, - nodes: nodes, - splits: splits, - ioConf: io, - localities: localities, - beforeVersion: beforeVersion, - testingKnobCfg: testingKnobCfg, - disableTenant: disableTenant, + name: name, + iodir: iodir, + nodes: nodes, + splits: splits, + ioConf: io, + localities: localities, + beforeVersion: beforeVersion, + testingKnobCfg: testingKnobCfg, + defaultTestTenant: defaultTestTenant, } err := ds.addCluster(t, cfg) if err != nil { diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 2326eceb2023..c75b7ab7ebdc 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -58,8 +58,8 @@ func TestFullClusterBackup(t *testing.T) { // helper function, that is not yet enabled to set up tenants within // clusters by default. Tracking issue // https://github.com/cockroachdb/cockroach/issues/76378 - DisableDefaultTestTenant: true, - Settings: settings, + DefaultTestTenant: base.TestTenantDisabled, + Settings: settings, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ // We compare job progress before and after a restore. Disable @@ -364,7 +364,7 @@ func TestSingletonSpanConfigJobPostRestore(t *testing.T) { // helper function, is not yet enabled to set up tenants within // clusters by default. Tracking issue // https://github.com/cockroachdb/cockroach/issues/76378 - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, @@ -966,7 +966,7 @@ func TestReintroduceOfflineSpans(t *testing.T) { // helper function, is not yet enabled to set up tenants within // clusters by default. Tracking issue // https://github.com/cockroachdb/cockroach/issues/76378 - params.ServerArgs.DisableDefaultTestTenant = true + params.ServerArgs.DefaultTestTenant = base.TestTenantDisabled const numAccounts = 1000 ctx := context.Background() @@ -1076,7 +1076,7 @@ func TestRestoreWithRecreatedDefaultDB(t *testing.T) { _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, // Disabling the default test tenant due to test failures. More // investigation is required. Tracked with #76378. - base.TestClusterArgs{ServerArgs: base.TestServerArgs{DisableDefaultTestTenant: true}}) + base.TestClusterArgs{ServerArgs: base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}}) defer cleanupFn() defer cleanupEmptyCluster() @@ -1101,7 +1101,7 @@ func TestRestoreWithDroppedDefaultDB(t *testing.T) { _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, // Disabling the default test tenant due to test failures. More // investigation is required. Tracked with #76378. - base.TestClusterArgs{ServerArgs: base.TestServerArgs{DisableDefaultTestTenant: true}}) + base.TestClusterArgs{ServerArgs: base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}}) defer cleanupFn() defer cleanupEmptyCluster() @@ -1127,7 +1127,7 @@ func TestFullClusterRestoreWithUserIDs(t *testing.T) { // helper function, that is not yet enabled to set up tenants within // clusters by default. Tracking issue // https://github.com/cockroachdb/cockroach/issues/76378 - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, diff --git a/pkg/ccl/backupccl/restore_mid_schema_change_test.go b/pkg/ccl/backupccl/restore_mid_schema_change_test.go index ba3afce459a8..850b90a65d9c 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -226,8 +226,8 @@ func restoreMidSchemaChange( // it relies on TestingGetTableDescriptor which isn't supported // in multi-tenancy. More work is required here. Tracked with // #76378. - DisableDefaultTestTenant: true, - Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, + DefaultTestTenant: base.TestTenantDisabled, + Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, }, } tc := testcluster.StartTestCluster(t, singleNode, params) diff --git a/pkg/ccl/backupccl/restore_old_versions_test.go b/pkg/ccl/backupccl/restore_old_versions_test.go index 00c52be2ab69..a9fe9b77b824 100644 --- a/pkg/ccl/backupccl/restore_old_versions_test.go +++ b/pkg/ccl/backupccl/restore_old_versions_test.go @@ -133,8 +133,8 @@ func restoreOldVersionClusterTest(exportDir string) func(t *testing.T) { ServerArgs: base.TestServerArgs{ // Disabling the test tenant due to test failures. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, - ExternalIODir: externalDir, + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: externalDir, }, }) sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) @@ -254,7 +254,7 @@ func TestRestoreWithDroppedSchemaCorruption(t *testing.T) { // reference a nil pointer below where we're expecting a database // descriptor to exist. More investigation is required. // Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, } s, sqlDB, _ := serverutils.StartServer(t, args) tdb := sqlutils.MakeSQLRunner(sqlDB) diff --git a/pkg/ccl/backupccl/system_schema_test.go b/pkg/ccl/backupccl/system_schema_test.go index 80c19039e319..7d524e05e1ae 100644 --- a/pkg/ccl/backupccl/system_schema_test.go +++ b/pkg/ccl/backupccl/system_schema_test.go @@ -31,7 +31,7 @@ func TestAllSystemTablesHaveBackupConfig(t *testing.T) { ServerArgs: base.TestServerArgs{ // Disabling the test tenant due to test failures. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}) defer tc.Stopper().Stop(ctx) sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) diff --git a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go index feec201a5086..a579fcea13bd 100644 --- a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go +++ b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go @@ -62,7 +62,7 @@ func TestTenantBackupWithCanceledImport(t *testing.T) { tc, hostSQLDB, hostClusterCleanupFn := backupRestoreTestSetupEmpty( t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, @@ -137,7 +137,7 @@ func TestTenantBackupNemesis(t *testing.T) { tc, hostSQLDB, hostClusterCleanupFn := backupRestoreTestSetupEmpty( t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 51841c6559f6..6aef46142141 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -148,11 +148,11 @@ func backupRestoreTestSetupWithParams( func backupRestoreTestSetup( t testing.TB, clusterSize int, numAccounts int, init func(*testcluster.TestCluster), ) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { - // TODO (msbutler): DisableDefaultTestTenant should be disabled by the caller of this function + // TODO (msbutler): The DefaultTestTenant should be disabled by the caller of this function return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}) } @@ -164,7 +164,7 @@ func backupRestoreTestSetupEmpty( params base.TestClusterArgs, ) (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.DefaultTestTenant = base.TestTenantDisabled return backupRestoreTestSetupEmptyWithParams(t, clusterSize, tempDir, init, params) } diff --git a/pkg/ccl/changefeedccl/alter_changefeed_test.go b/pkg/ccl/changefeedccl/alter_changefeed_test.go index 7283639da7fc..181b70d8bb30 100644 --- a/pkg/ccl/changefeedccl/alter_changefeed_test.go +++ b/pkg/ccl/changefeedccl/alter_changefeed_test.go @@ -53,7 +53,7 @@ func TestAlterChangefeedAddTargetPrivileges(t *testing.T) { defer log.Scope(t).Close(t) srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 70687259f535..09bec967c5c9 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -160,8 +160,8 @@ func TestChangefeedReplanning(t *testing.T) { }, JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, - UseDatabase: "d", - DisableDefaultTestTenant: true, + UseDatabase: "d", + DefaultTestTenant: base.TestTenantDisabled, } tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ @@ -2679,7 +2679,7 @@ func TestChangefeedCreateAuthorizationWithChangefeedPriv(t *testing.T) { defer log.Scope(t).Close(t) srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ @@ -6012,10 +6012,10 @@ func TestChangefeedHandlesDrainingNodes(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test uses SPLIT AT, which isn't currently supported for // secondary tenants. Tracked with #76378. - DisableDefaultTestTenant: true, - UseDatabase: "test", - Knobs: knobs, - ExternalIODir: sinkDir, + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: "test", + Knobs: knobs, + ExternalIODir: sinkDir, }}) defer tc.Stopper().Stop(context.Background()) @@ -6081,7 +6081,7 @@ func TestChangefeedPropagatesTerminalError(t *testing.T) { perServerKnobs[i] = base.TestServerArgs{ // Test uses SPLIT AT, which isn't currently supported for // secondary tenants. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ DrainFast: true, @@ -8374,7 +8374,7 @@ func TestChangefeedExecLocality(t *testing.T) { ExternalIODir: path.Join(dir, str(i)), Locality: roachpb.Locality{ Tiers: []roachpb.Tier{{Key: "x", Value: str(i / 2)}, {Key: "y", Value: str(i % 2)}}}, - DisableDefaultTestTenant: true, // need nodelocal and splits. + DefaultTestTenant: base.TestTenantDisabled, // need nodelocal and splits. } } diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index 53ee7c488705..676d84f8ecb8 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -392,10 +392,10 @@ func startTestFullServer( Knobs: knobs, // This test suite is already probabilistically running with // tenants. No need for the test tenant. - DisableDefaultTestTenant: true, - UseDatabase: `d`, - ExternalIODir: options.externalIODir, - Settings: options.settings, + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: `d`, + ExternalIODir: options.externalIODir, + Settings: options.settings, } if options.argsFn != nil { diff --git a/pkg/ccl/changefeedccl/scheduled_changefeed_test.go b/pkg/ccl/changefeedccl/scheduled_changefeed_test.go index 535fd118844d..8b66ee9c1827 100644 --- a/pkg/ccl/changefeedccl/scheduled_changefeed_test.go +++ b/pkg/ccl/changefeedccl/scheduled_changefeed_test.go @@ -292,7 +292,7 @@ func TestCreateChangefeedScheduleChecksPermissionsDuringDryRun(t *testing.T) { defer log.Scope(t).Close(t) srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter_datadriven_test.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter_datadriven_test.go index a455c6f8192f..445a5d87bfc2 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter_datadriven_test.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter_datadriven_test.go @@ -66,7 +66,7 @@ func TestDataDriven(t *testing.T) { ctx := context.Background() ts, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ // TODO(ajwerner): Otherwise the test gets skipped due to some CCL warning. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) tdb := sqlutils.MakeSQLRunner(sqlDB) defer ts.Stopper().Stop(ctx) diff --git a/pkg/ccl/cloudccl/externalconn/datadriven_test.go b/pkg/ccl/cloudccl/externalconn/datadriven_test.go index 4477a540886e..77f0d024db6b 100644 --- a/pkg/ccl/cloudccl/externalconn/datadriven_test.go +++ b/pkg/ccl/cloudccl/externalconn/datadriven_test.go @@ -51,6 +51,7 @@ func TestDataDriven(t *testing.T) { } tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantProbabilistic, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test ExternalConnection: ecTestingKnobs, diff --git a/pkg/ccl/importerccl/ccl_test.go b/pkg/ccl/importerccl/ccl_test.go index 5be9a746b203..c46b2fd2d07e 100644 --- a/pkg/ccl/importerccl/ccl_test.go +++ b/pkg/ccl/importerccl/ccl_test.go @@ -388,7 +388,10 @@ func TestExportInsideTenant(t *testing.T) { dir, cleanupDir := testutils.TempDir(t) defer cleanupDir() - srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{ExternalIODir: dir}) + srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + DefaultTestTenant: base.TestTenantProbabilistic, + ExternalIODir: dir, + }) defer srv.Stopper().Stop(context.Background()) _, conn10 := serverutils.StartTenant(t, srv, base.TestTenantArgs{TenantID: roachpb.MustMakeTenantID(10)}) @@ -418,7 +421,7 @@ func TestImportInTenant(t *testing.T) { ExternalIODir: baseDir, // Test is designed to run inside a tenant so no need to // probabilistically run it inside the default test tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, } tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: args}) defer tc.Stopper().Stop(ctx) @@ -469,7 +472,12 @@ func TestImportInMultiServerTenant(t *testing.T) { ctx := context.Background() baseDir := datapathutils.TestDataPath(t) - args := base.TestServerArgs{ExternalIODir: baseDir} + args := base.TestServerArgs{ + // Test is designed to run inside a tenant so no need to + // probabilistically run it inside the default test tenant. + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: baseDir, + } tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: args}) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go b/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go index 6de7c4c66bd5..2ba93168b76c 100644 --- a/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go +++ b/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go @@ -163,7 +163,7 @@ func TestJobsProtectedTimestamp(t *testing.T) { ServerArgs: base.TestServerArgs{ // Tests fail within a tenant. Disabling until we can // investigate further. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, @@ -289,7 +289,7 @@ func TestSchedulesProtectedTimestamp(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails within a tenant. Disabling pending further // investigation. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go index 043b702c3098..465453ef9e28 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go @@ -274,7 +274,7 @@ func TestBoundedStalenessDataDriven(t *testing.T) { for i := 0; i < numNodes; i++ { i := i clusterArgs.ServerArgsPerNode[i] = base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ WithStatementTrace: func(trace tracingpb.Recording, stmt string) { diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index acad55d723dd..0688a0ac4c1d 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -704,16 +704,16 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, - UseDatabase: "t", + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: "t", }, // n4 pretends to have low latency to n2 and n3, so that it tries to use // them for follower reads. // Also, we're going to collect a trace of the test's final query. ServerArgsPerNode: map[int]base.TestServerArgs{ 3: { - DisableDefaultTestTenant: true, - UseDatabase: "t", + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: "t", Knobs: base.TestingKnobs{ KVClient: &kvcoord.ClientTestingKnobs{ // Inhibit the checking of connection health done by the @@ -856,8 +856,8 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) { } localities[i] = locality serverArgs[i] = base.TestServerArgs{ - Locality: localities[i], - DisableDefaultTestTenant: true, // we'll create one ourselves below. + Locality: localities[i], + DefaultTestTenant: base.TestTenantDisabled, // we'll create one ourselves below. } } tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_kv_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_kv_test.go index 068374d3db99..10b9e06568a7 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_kv_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_kv_test.go @@ -36,7 +36,7 @@ func TestTenantRangeQPSStat(t *testing.T) { InsecureWebAccess: true, // Must disable test tenant because test below assumes that // it is connecting to the host tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ // We disable the split queue as an untimely split can cause the QPS diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_range_lookup_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_range_lookup_test.go index b9dd06fff596..adaf98e49244 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_range_lookup_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_range_lookup_test.go @@ -31,7 +31,7 @@ func TestRangeLookupPrefetchFiltering(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, // we're going to manually add tenants + DefaultTestTenant: base.TestTenantDisabled, // we're going to manually add tenants Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ DisableMergeQueue: true, diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_scan_range_descriptors_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_scan_range_descriptors_test.go index 18578b5c14d7..54f92b26bcbf 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_scan_range_descriptors_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_scan_range_descriptors_test.go @@ -28,7 +28,7 @@ func setup( ) (*testcluster.TestCluster, serverutils.TestTenantInterface, rangedesc.IteratorFactory) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, // we're going to manually add tenants + DefaultTestTenant: base.TestTenantDisabled, // we're going to manually add tenants Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ DisableMergeQueue: true, diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go index 561a0a20923f..246a06bc668d 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go @@ -55,7 +55,7 @@ func testTenantTracesAreRedactedImpl(t *testing.T, redactable bool) { args := base.TestServerArgs{ // Test hangs within a tenant. More investigation is required. // Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go index c274bfb0be49..4c09994af654 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go @@ -72,8 +72,8 @@ func TestTenantUpgrade(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test validates tenant behavior. No need for the default test // tenant. - DisableDefaultTestTenant: true, - Settings: settings, + DefaultTestTenant: base.TestTenantDisabled, + Settings: settings, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ DisableAutomaticVersionUpgrade: make(chan struct{}), @@ -223,8 +223,8 @@ func TestTenantUpgradeFailure(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test validates tenant behavior. No need for the default test // tenant here. - DisableDefaultTestTenant: true, - Settings: settings, + DefaultTestTenant: base.TestTenantDisabled, + Settings: settings, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ DisableAutomaticVersionUpgrade: make(chan struct{}), @@ -599,8 +599,8 @@ func TestTenantUpgradeInterlock(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test validates tenant behavior. No need for the default test // tenant. - DisableDefaultTestTenant: true, - Settings: settings, + DefaultTestTenant: base.TestTenantDisabled, + Settings: settings, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, diff --git a/pkg/ccl/multiregionccl/cold_start_latency_test.go b/pkg/ccl/multiregionccl/cold_start_latency_test.go index e5d7e6b9f4af..486c2d4e9e56 100644 --- a/pkg/ccl/multiregionccl/cold_start_latency_test.go +++ b/pkg/ccl/multiregionccl/cold_start_latency_test.go @@ -80,8 +80,8 @@ func TestColdStartLatency(t *testing.T) { for i := 0; i < numNodes; i++ { i := i args := base.TestServerArgs{ - DisableDefaultTestTenant: true, - Locality: localities[i], + DefaultTestTenant: base.TestTenantDisabled, + Locality: localities[i], } signalAfter[i] = make(chan struct{}) serverKnobs := &server.TestingKnobs{ diff --git a/pkg/ccl/multiregionccl/datadriven_test.go b/pkg/ccl/multiregionccl/datadriven_test.go index 70c9721ceafd..2f8e49ebd58f 100644 --- a/pkg/ccl/multiregionccl/datadriven_test.go +++ b/pkg/ccl/multiregionccl/datadriven_test.go @@ -157,7 +157,7 @@ func TestMultiRegionDataDriven(t *testing.T) { // "wait-for-zone-config-changes" only work correctly // when called from the system tenant. More // investigation is required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ WithStatementTrace: func(trace tracingpb.Recording, stmt string) { diff --git a/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go b/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go index f2c97cfbf776..9d419c208de5 100644 --- a/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go +++ b/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go @@ -116,7 +116,7 @@ func TestingCreateMultiRegionClusterWithRegionList( // bonus points, the code to re-enable this should also provide more // flexibility in disabling the default test tenant by callers of this // function. Re-enablement is tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Locality: roachpb.Locality{ Tiers: []roachpb.Tier{{Key: "region", Value: region}}, }, diff --git a/pkg/ccl/multiregionccl/regional_by_row_test.go b/pkg/ccl/multiregionccl/regional_by_row_test.go index bd1192e0b955..3944207f7a89 100644 --- a/pkg/ccl/multiregionccl/regional_by_row_test.go +++ b/pkg/ccl/multiregionccl/regional_by_row_test.go @@ -354,7 +354,7 @@ func TestAlterTableLocalityRegionalByRowError(t *testing.T) { // when running inside a tenant, for some reason // this test doesn't error when expected. More // investigation is required. Tracked with #76378. - params.DisableDefaultTestTenant = true + params.DefaultTestTenant = base.TestTenantDisabled var sqlDB *gosql.DB params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ diff --git a/pkg/ccl/multiregionccl/unique_test.go b/pkg/ccl/multiregionccl/unique_test.go index adab07fa5f51..dc25467c0b82 100644 --- a/pkg/ccl/multiregionccl/unique_test.go +++ b/pkg/ccl/multiregionccl/unique_test.go @@ -33,7 +33,7 @@ func TestValidateUniqueConstraints(t *testing.T) { defer log.Scope(t).Close(t) // This test fails when run within a tenant. More investigation is // required. Tracked with #76378. - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{DisableDefaultTestTenant: true}) + s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer s.Stopper().Stop(context.Background()) r := sqlutils.MakeSQLRunner(db) diff --git a/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go index 91299e4040c6..f1cddceb22eb 100644 --- a/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go +++ b/pkg/ccl/multitenantccl/tenantcapabilitiesccl/capabilities_test.go @@ -71,7 +71,7 @@ func TestDataDriven(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, // We'll create a tenant ourselves. + DefaultTestTenant: base.TestTenantDisabled, // We'll create a tenant ourselves. Knobs: base.TestingKnobs{ TenantCapabilitiesTestingKnobs: &tenantcapabilities.TestingKnobs{ WatcherTestingKnobs: &tenantcapabilitieswatcher.TestingKnobs{ diff --git a/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go b/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go index aa1fde2644b0..b74ec0771efd 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go @@ -61,8 +61,8 @@ func TestEstimateQueryRUConsumption(t *testing.T) { tenantcostclient.TargetPeriodSetting.Override(ctx, &st.SV, time.Millisecond*500) params := base.TestServerArgs{ - Settings: st, - DisableDefaultTestTenant: true, + Settings: st, + DefaultTestTenant: base.TestTenantDisabled, } s, mainDB, _ := serverutils.StartServer(t, params) diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go index 70843d1d0290..50c1f62933cd 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go @@ -868,7 +868,7 @@ func TestConsumption(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - hostServer, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + hostServer, _, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer hostServer.Stopper().Stop(context.Background()) st := cluster.MakeTestingClusterSettings() @@ -938,7 +938,7 @@ func TestSQLLivenessExemption(t *testing.T) { // This test fails when run with the default test tenant. Disabling and // tracking with #76378. - hostServer, hostDB, hostKV := serverutils.StartServer(t, base.TestServerArgs{DisableDefaultTestTenant: true}) + hostServer, hostDB, hostKV := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer hostServer.Stopper().Stop(context.Background()) tenantID := serverutils.TestTenantID() @@ -1005,7 +1005,7 @@ func TestScheduledJobsConsumption(t *testing.T) { stats.AutomaticStatisticsOnSystemTables.Override(ctx, &st.SV, false) tenantcostclient.TargetPeriodSetting.Override(ctx, &st.SV, time.Millisecond*20) - hostServer, _, _ := serverutils.StartServer(t, base.TestServerArgs{Settings: st}) + hostServer, _, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled, Settings: st}) defer hostServer.Stopper().Stop(ctx) testProvider := newTestProvider() @@ -1089,7 +1089,7 @@ func TestConsumptionChangefeeds(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - hostServer, hostDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + hostServer, hostDB, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer hostServer.Stopper().Stop(context.Background()) if _, err := hostDB.Exec("SET CLUSTER SETTING kv.rangefeed.enabled = true"); err != nil { t.Fatalf("changefeed setup failed: %s", err.Error()) @@ -1160,8 +1160,8 @@ func TestConsumptionExternalStorage(t *testing.T) { defer dirCleanupFn() hostServer, hostDB, _ := serverutils.StartServer(t, base.TestServerArgs{ // Test fails when run within the default tenant. Tracked with #76378. - DisableDefaultTestTenant: true, - ExternalIODir: dir, + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: dir, }) defer hostServer.Stopper().Stop(context.Background()) hostSQL := sqlutils.MakeSQLRunner(hostDB) @@ -1266,7 +1266,7 @@ func BenchmarkExternalIOAccounting(b *testing.B) { hostServer, hostSQL, _ := serverutils.StartServer(b, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer hostServer.Stopper().Stop(context.Background()) diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index e3c591deaf41..0ee4a42f0952 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -201,8 +201,8 @@ SELECT job_id ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, - Knobs: knobs, + DefaultTestTenant: base.TestTenantDisabled, + Knobs: knobs, }, }) defer tc.Stopper().Stop(ctx) @@ -247,8 +247,8 @@ range_max_bytes = 654321000`) ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, - Knobs: knobs, + DefaultTestTenant: base.TestTenantDisabled, + Knobs: knobs, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index c4a87f7520cc..488144e434bb 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -1418,8 +1418,8 @@ func TestRemovePartitioningExpiredLicense(t *testing.T) { ctx := context.Background() s, sqlDBRaw, _ := serverutils.StartServer(t, base.TestServerArgs{ - UseDatabase: "d", - DisableDefaultTestTenant: true, + UseDatabase: "d", + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) diff --git a/pkg/ccl/partitionccl/scrub_test.go b/pkg/ccl/partitionccl/scrub_test.go index 5f573d77302e..2dc91f181ee1 100644 --- a/pkg/ccl/partitionccl/scrub_test.go +++ b/pkg/ccl/partitionccl/scrub_test.go @@ -41,9 +41,7 @@ func TestScrubUniqueIndex(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) utilccl.TestingEnableEnterprise() - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: false, - }) + s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(context.Background()) // Create the table and row entries. diff --git a/pkg/ccl/partitionccl/zone_test.go b/pkg/ccl/partitionccl/zone_test.go index 732290675e30..6266078cdef9 100644 --- a/pkg/ccl/partitionccl/zone_test.go +++ b/pkg/ccl/partitionccl/zone_test.go @@ -43,7 +43,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) { defer log.Scope(t).Close(t) s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -246,7 +246,7 @@ func TestInvalidIndexPartitionSetShowZones(t *testing.T) { defer log.Scope(t).Close(t) s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) diff --git a/pkg/ccl/serverccl/admin_test.go b/pkg/ccl/serverccl/admin_test.go index fb64fd4d948a..c6fc2f981caf 100644 --- a/pkg/ccl/serverccl/admin_test.go +++ b/pkg/ccl/serverccl/admin_test.go @@ -50,11 +50,11 @@ func TestAdminAPIDataDistributionPartitioning(t *testing.T) { // Need to disable the test tenant because this test fails // when run through a tenant (with internal server error). // More investigation is required. Tracked with #76387. - disableDefaultTestTenant := true + defaultTestTenant := base.TestTenantDisabled testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: disableDefaultTestTenant, + DefaultTestTenant: defaultTestTenant, }, }) defer testCluster.Stopper().Stop(context.Background()) @@ -82,7 +82,7 @@ func TestAdminAPIDataDistributionPartitioning(t *testing.T) { sqlDB.Exec(t, `ALTER PARTITION us OF TABLE comments CONFIGURE ZONE USING gc.ttlseconds = 9001`) sqlDB.Exec(t, `ALTER PARTITION eu OF TABLE comments CONFIGURE ZONE USING gc.ttlseconds = 9002`) - if disableDefaultTestTenant { + if defaultTestTenant == base.TestTenantDisabled { // Make sure secondary tenants don't cause the endpoint to error. sqlDB.Exec(t, "CREATE TENANT 'app'") } @@ -131,8 +131,8 @@ func TestAdminAPIJobs(t *testing.T) { defer dirCleanupFn() s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Fails with the default test tenant. Tracked with #76378. - DisableDefaultTestTenant: true, - ExternalIODir: dir}) + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: dir}) defer s.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(conn) @@ -171,7 +171,7 @@ func TestListTenants(t *testing.T) { ctx := context.Background() s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) @@ -206,7 +206,7 @@ func TestTableAndDatabaseDetailsAndStats(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer s.Stopper().Stop(ctx) st, db := serverutils.StartTenant(t, s, base.TestTenantArgs{ diff --git a/pkg/ccl/serverccl/diagnosticsccl/reporter_test.go b/pkg/ccl/serverccl/diagnosticsccl/reporter_test.go index 7fc1b664422c..b78427a7c505 100644 --- a/pkg/ccl/serverccl/diagnosticsccl/reporter_test.go +++ b/pkg/ccl/serverccl/diagnosticsccl/reporter_test.go @@ -49,7 +49,7 @@ func TestTenantReport(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rt := startReporterTest(t) + rt := startReporterTest(t, base.TestTenantDisabled) defer rt.Close() tenantArgs := base.TestTenantArgs{ @@ -103,7 +103,8 @@ func TestServerReport(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rt := startReporterTest(t) + var defaultTestTenant base.DefaultTestTenantOptions + rt := startReporterTest(t, defaultTestTenant) defer rt.Close() ctx := context.Background() @@ -390,7 +391,9 @@ func (t *reporterTest) Close() { t.server.Stopper().Stop(context.Background()) } -func startReporterTest(t *testing.T) *reporterTest { +func startReporterTest( + t *testing.T, defaultTestTenant base.DefaultTestTenantOptions, +) *reporterTest { // Disable cloud info reporting, since it slows down tests. rt := &reporterTest{ cloudEnable: cloudinfo.Disable(), @@ -415,6 +418,7 @@ func startReporterTest(t *testing.T) *reporterTest { storeSpec := base.DefaultTestStoreSpec storeSpec.Attributes = roachpb.Attributes{Attrs: []string{elemName}} rt.serverArgs = base.TestServerArgs{ + DefaultTestTenant: defaultTestTenant, StoreSpecs: []base.StoreSpec{ storeSpec, base.DefaultTestStoreSpec, diff --git a/pkg/ccl/serverccl/role_authentication_test.go b/pkg/ccl/serverccl/role_authentication_test.go index 38ed2c321c16..e318da572272 100644 --- a/pkg/ccl/serverccl/role_authentication_test.go +++ b/pkg/ccl/serverccl/role_authentication_test.go @@ -39,7 +39,7 @@ func TestVerifyPassword(t *testing.T) { // Need to disable the test tenant here because it appears as // though we don't have all the same roles in the tenant as we // have in the host cluster (like root). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) defer s.Stopper().Stop(ctx) diff --git a/pkg/ccl/serverccl/server_controller_test.go b/pkg/ccl/serverccl/server_controller_test.go index be88d4c6b7e6..4708d9347f57 100644 --- a/pkg/ccl/serverccl/server_controller_test.go +++ b/pkg/ccl/serverccl/server_controller_test.go @@ -62,19 +62,19 @@ func TestSharedProcessTenantNodeLocalAccess(t *testing.T) { }() tc := serverutils.StartNewTestCluster(t, nodeCount, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{DisableDefaultTestTenant: true}, + ServerArgs: base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}, ServerArgsPerNode: map[int]base.TestServerArgs{ 0: { - DisableDefaultTestTenant: true, - ExternalIODir: dirs[0], + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: dirs[0], }, 1: { - DisableDefaultTestTenant: true, - ExternalIODir: dirs[1], + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: dirs[1], }, 2: { - DisableDefaultTestTenant: true, - ExternalIODir: dirs[2], + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: dirs[2], }, }, }) @@ -131,7 +131,7 @@ func TestServerControllerHTTP(t *testing.T) { ctx := context.Background() s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) @@ -308,7 +308,7 @@ func TestServerControllerDefaultHTTPTenant(t *testing.T) { ctx := context.Background() - s, _, _ := serverutils.StartServer(t, base.TestServerArgs{DisableDefaultTestTenant: true}) + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer s.Stopper().Stop(ctx) _, sql, err := s.StartSharedProcessTenant(ctx, base.TestSharedProcessTenantArgs{ @@ -390,7 +390,7 @@ func TestServerControllerMultiNodeTenantStartup(t *testing.T) { numNodes := 3 tc := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}) defer tc.Stopper().Stop(ctx) @@ -424,7 +424,7 @@ func TestServerStartStop(t *testing.T) { ctx := context.Background() s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) diff --git a/pkg/ccl/serverccl/server_sql_test.go b/pkg/ccl/serverccl/server_sql_test.go index d28161794732..d9dbe83d5f6b 100644 --- a/pkg/ccl/serverccl/server_sql_test.go +++ b/pkg/ccl/serverccl/server_sql_test.go @@ -51,7 +51,11 @@ func TestSQLServer(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + // We need to disable the default test tenant because we're going to create + // our own. + DefaultTestTenant: base.TestTenantDisabled, + }}) defer tc.Stopper().Stop(ctx) _, db := serverutils.StartTenant( @@ -76,7 +80,9 @@ func TestTenantCannotSetClusterSetting(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantDisabled, + }}) defer tc.Stopper().Stop(ctx) // StartTenant with the default permissions to @@ -104,7 +110,9 @@ func TestTenantCanUseEnterpriseFeatures(t *testing.T) { defer ccl.TestingDisableEnterprise()() defer envutil.TestSetEnv(t, "COCKROACH_TENANT_LICENSE", license)() - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantDisabled, + }}) defer tc.Stopper().Stop(context.Background()) _, db := serverutils.StartTenant(t, tc.Server(0), base.TestTenantArgs{TenantID: serverutils.TestTenantID()}) @@ -121,7 +129,9 @@ func TestTenantUnauthenticatedAccess(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantProbabilistic, + }}) defer tc.Stopper().Stop(ctx) _, err := tc.Server(0).StartTenant(ctx, @@ -144,7 +154,9 @@ func TestTenantHTTP(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantDisabled, + }}) defer tc.Stopper().Stop(ctx) tenant, err := tc.Server(0).StartTenant(ctx, @@ -185,7 +197,7 @@ func TestNonExistentTenant(t *testing.T) { tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) @@ -206,7 +218,9 @@ func TestTenantRowIDs(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantDisabled, + }}) defer tc.Stopper().Stop(ctx) const numRows = 10 tenant, db := serverutils.StartTenant( @@ -248,7 +262,7 @@ func TestTenantInstanceIDReclaimLoop(t *testing.T) { ServerArgs: base.TestServerArgs{ Settings: settings, // Don't use a default test tenant. We will explicitly create one. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/serverccl/server_startup_guardrails_test.go b/pkg/ccl/serverccl/server_startup_guardrails_test.go index 6402a56bce1f..e15b46d9336b 100644 --- a/pkg/ccl/serverccl/server_startup_guardrails_test.go +++ b/pkg/ccl/serverccl/server_startup_guardrails_test.go @@ -75,8 +75,8 @@ func TestServerStartupGuardrails(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant, since we create one explicitly // below. - DisableDefaultTestTenant: true, - Settings: storageSettings, + DefaultTestTenant: base.TestTenantDisabled, + Settings: storageSettings, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ BinaryVersionOverride: test.storageBinaryVersion, diff --git a/pkg/ccl/serverccl/shared_process_tenant_test.go b/pkg/ccl/serverccl/shared_process_tenant_test.go index b836ace5b25f..8b20da328b05 100644 --- a/pkg/ccl/serverccl/shared_process_tenant_test.go +++ b/pkg/ccl/serverccl/shared_process_tenant_test.go @@ -29,7 +29,7 @@ func TestSharedProcessTenantNoSpanLimit(t *testing.T) { tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }}) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/serverccl/statusccl/tenant_status_test.go b/pkg/ccl/serverccl/statusccl/tenant_status_test.go index 631206c1dafd..61a7b7b956d6 100644 --- a/pkg/ccl/serverccl/statusccl/tenant_status_test.go +++ b/pkg/ccl/serverccl/statusccl/tenant_status_test.go @@ -291,7 +291,7 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) { } // Need to disable the test tenant here as the non-tenant case below // assumes that it's operating within the system tenant. - serverParams.DisableDefaultTestTenant = true + serverParams.DefaultTestTenant = base.TestTenantDisabled testCluster := serverutils.StartNewTestCluster(t, 3 /* numNodes */, base.TestClusterArgs{ ServerArgs: serverParams, }) diff --git a/pkg/ccl/serverccl/tenant_decommissioned_host_test.go b/pkg/ccl/serverccl/tenant_decommissioned_host_test.go index 8c30d6361e47..3d5aa68ad186 100644 --- a/pkg/ccl/serverccl/tenant_decommissioned_host_test.go +++ b/pkg/ccl/serverccl/tenant_decommissioned_host_test.go @@ -40,7 +40,10 @@ func TestTenantWithDecommissionedID(t *testing.T) { // as sql connection timeouts. ctx := context.Background() - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + // Disable the default test tenant since we are creating our own. + DefaultTestTenant: base.TestTenantDisabled, + }}) defer tc.Stopper().Stop(ctx) server := tc.Server(0) diff --git a/pkg/ccl/serverccl/tenant_migration_test.go b/pkg/ccl/serverccl/tenant_migration_test.go index 16937a68e00f..359d8b288266 100644 --- a/pkg/ccl/serverccl/tenant_migration_test.go +++ b/pkg/ccl/serverccl/tenant_migration_test.go @@ -82,8 +82,8 @@ func TestValidateTargetTenantClusterVersion(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant, since we create one explicitly // below. - DisableDefaultTestTenant: true, - Settings: st, + DefaultTestTenant: base.TestTenantDisabled, + Settings: st, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ BinaryVersionOverride: test.binaryVersion, @@ -192,8 +192,8 @@ func TestBumpTenantClusterVersion(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default tenant because we're creating one // explicitly below. - DisableDefaultTestTenant: true, - Settings: st, + DefaultTestTenant: base.TestTenantDisabled, + Settings: st, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ // This test wants to bootstrap at the previously active diff --git a/pkg/ccl/serverccl/tenant_test_utils.go b/pkg/ccl/serverccl/tenant_test_utils.go index 139303a39a3b..c32a8b5d32b2 100644 --- a/pkg/ccl/serverccl/tenant_test_utils.go +++ b/pkg/ccl/serverccl/tenant_test_utils.go @@ -150,7 +150,7 @@ func NewTestTenantHelper( params, _ := tests.CreateTestServerParams() params.Knobs = knobs // We're running tenant tests, no need for a default tenant. - params.DisableDefaultTestTenant = true + params.DefaultTestTenant = base.TestTenantDisabled testCluster := serverutils.StartNewTestCluster(t, 1 /* numNodes */, base.TestClusterArgs{ ServerArgs: params, }) diff --git a/pkg/ccl/spanconfigccl/spanconfigkvaccessorccl/kvaccessor_test.go b/pkg/ccl/spanconfigccl/spanconfigkvaccessorccl/kvaccessor_test.go index 75712fe30e13..157152d07e9d 100644 --- a/pkg/ccl/spanconfigccl/spanconfigkvaccessorccl/kvaccessor_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigkvaccessorccl/kvaccessor_test.go @@ -39,7 +39,7 @@ func TestCommitTSIntervals(t *testing.T) { var i interceptor ts, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Manually starts a tenant below. No need to start one here. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ WallClock: manual, diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go index 34a5a4bfec24..1240ca12be23 100644 --- a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go @@ -73,6 +73,7 @@ func TestDataDriven(t *testing.T) { } tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantProbabilistic, Knobs: base.TestingKnobs{ SpanConfig: scKnobs, }, diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go index 3360caac43ba..bc7a115992b4 100644 --- a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go @@ -37,7 +37,9 @@ func TestDropTableLowersSpanCount(t *testing.T) { defer gcjob.SetSmallMaxGCIntervalForTest()() ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestTenantProbabilistic, + }}) defer tc.Stopper().Stop(ctx) ts := tc.Server(0) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go index 2ab3473a79a2..1574492374ee 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go @@ -101,7 +101,7 @@ func TestDataDriven(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails when run under the default test tenant. More // investigation is required. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test SpanConfig: scKnobs, diff --git a/pkg/ccl/spanconfigccl/spanconfigsplitterccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsplitterccl/datadriven_test.go index 84f849186297..04b72e99ea70 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsplitterccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsplitterccl/datadriven_test.go @@ -68,7 +68,7 @@ func TestDataDriven(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Fails with nil pointer dereference. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: scKnobs, }, diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index 2cbca17abf36..58f9f101ad57 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -111,7 +111,7 @@ func TestDataDriven(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails when run within a tenant. More investigation // is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ GCJob: gcTestingKnobs, SpanConfig: scKnobs, diff --git a/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go b/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go index bfd291a0cd13..defaee44f47e 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go @@ -62,7 +62,7 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { ServerArgs: base.TestServerArgs{ ExternalIODir: dir, // Test already runs from a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation. @@ -289,7 +289,7 @@ func TestSQLWatcherMultiple(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Test already runs from a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation. @@ -421,7 +421,7 @@ func TestSQLWatcherOnEventError(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Test already runs from a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation. @@ -472,7 +472,7 @@ func TestSQLWatcherHandlerError(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Test already runs from a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation. @@ -550,7 +550,7 @@ func TestWatcherReceivesNoopCheckpoints(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Test already runs from a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, // disable the automatic job creation. diff --git a/pkg/ccl/sqlproxyccl/backend_dialer_test.go b/pkg/ccl/sqlproxyccl/backend_dialer_test.go index affffd31c2f6..444284d5fab9 100644 --- a/pkg/ccl/sqlproxyccl/backend_dialer_test.go +++ b/pkg/ccl/sqlproxyccl/backend_dialer_test.go @@ -65,7 +65,7 @@ func TestBackendDialTLS(t *testing.T) { // storage server to be the system tenant, otherwise the // tenant10ToStorage test will fail, since the storage server will // server tenant 10. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer storageServer.Stopper().Stop(ctx) diff --git a/pkg/ccl/sqlproxyccl/proxy_handler_test.go b/pkg/ccl/sqlproxyccl/proxy_handler_test.go index 6c9749a98c96..7bcb1e0f8a97 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler_test.go @@ -86,7 +86,7 @@ func TestProxyProtocol(t *testing.T) { // Need to disable the test tenant here because it appears as though // we're not able to establish the necessary connections from within // it. More investigation required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) sql.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) pgs := sql.(*server.TestServer).PGServer().(*pgwire.Server) @@ -370,7 +370,7 @@ func TestProxyAgainstSecureCRDB(t *testing.T) { // Need to disable the test tenant here because it appears as though // we're not able to establish the necessary connections from within // it. More investigation required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) sql.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) @@ -533,7 +533,7 @@ func TestProxyTLSClose(t *testing.T) { // Need to disable the test tenant here because it appears as though // we're not able to establish the necessary connections from within // it. More investigation required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) sql.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) @@ -589,7 +589,7 @@ func TestProxyModifyRequestParams(t *testing.T) { // Need to disable the test tenant here because it appears as though // we're not able to establish the necessary connections from within // it. More investigation required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) sql.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) @@ -653,8 +653,8 @@ func TestInsecureProxy(t *testing.T) { // complains about not being able to find the user. This may be // because of the connection through the proxy server. More // investigation is required (tracked with #76378). - DisableDefaultTestTenant: true, - Insecure: false, + DefaultTestTenant: base.TestTenantDisabled, + Insecure: false, }, ) sql.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) @@ -833,7 +833,7 @@ func TestDenylistUpdate(t *testing.T) { // Need to disable the test tenant here because it appears as though // we're not able to establish the necessary connections from within // it. More investigation required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) sql.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) @@ -939,7 +939,7 @@ func TestDirectoryConnect(t *testing.T) { // Need to disable the test tenant here because it appears as though // we're not able to establish the necessary connections from within // it. More investigation required (tracked with #76378). - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) srv.(*server.TestServer).PGPreServer().TestingSetTrustClientProvidedRemoteAddr(true) @@ -1565,7 +1565,7 @@ func TestConnectionMigration(t *testing.T) { params, _ := tests.CreateTestServerParams() // Test must be run from the system tenant as it's altering tenants. - params.DisableDefaultTestTenant = true + params.DefaultTestTenant = base.TestTenantDisabled s, mainDB, _ := serverutils.StartServer(t, params) defer s.Stopper().Stop(ctx) tenantID := serverutils.TestTenantID() diff --git a/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go b/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go index 0d34b9943cfd..f1b2665b5c0a 100644 --- a/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go +++ b/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go @@ -500,7 +500,7 @@ func newTestDirectoryCache( Insecure: true, // Test fails when run within a tenant. More investigation // is required here. Tracked with #76387. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) clusterStopper := tc.Stopper() diff --git a/pkg/ccl/streamingccl/replicationtestutils/testutils.go b/pkg/ccl/streamingccl/replicationtestutils/testutils.go index 7a57b42eeb38..1595f69efce1 100644 --- a/pkg/ccl/streamingccl/replicationtestutils/testutils.go +++ b/pkg/ccl/streamingccl/replicationtestutils/testutils.go @@ -223,7 +223,7 @@ func CreateTenantStreamingClusters( serverArgs := base.TestServerArgs{ // Test fails because it tries to set a cluster setting only accessible // to system tenants. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ diff --git a/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go b/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go index 6f4f6c8a704b..300697be5596 100644 --- a/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go +++ b/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go @@ -70,7 +70,7 @@ func TestPartitionedStreamReplicationClient(t *testing.T) { base.TestServerArgs{ // Need to disable the test tenant until tenant-level restore is // supported. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, diff --git a/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go b/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go index 39ecb91d7cc4..e1615675636a 100644 --- a/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go +++ b/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go @@ -204,7 +204,7 @@ func TestStreamIngestionJobWithRandomClient(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test hangs with test tenant. More investigation is required. // Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, } params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go index 3c8833f44a22..066c3c24a7b5 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go @@ -61,7 +61,7 @@ func TestTenantStreaming(t *testing.T) { // a secondary tenant, it won't be able to see the streaming job. // This may also be impacted by the fact that we don't currently support // tenant->tenant streaming. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, } @@ -103,7 +103,7 @@ SET CLUSTER SETTING stream_replication.stream_liveness_track_frequency = '500ms' // is required. Tracked with #76378. // TODO(ajstorm): This may be the right course of action here as the // replication is now being run inside a tenant. - base.TestServerArgs{DisableDefaultTestTenant: true}) + base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer cleanupDest() // destSQL refers to the system tenant as that's the one that's running the // job. @@ -178,9 +178,9 @@ func TestTenantStreamingCreationErrors(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - srcServer, srcDB, _ := serverutils.StartServer(t, base.TestServerArgs{DisableDefaultTestTenant: true}) + srcServer, srcDB, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer srcServer.Stopper().Stop(ctx) - destServer, destDB, _ := serverutils.StartServer(t, base.TestServerArgs{DisableDefaultTestTenant: true}) + destServer, destDB, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer destServer.Stopper().Stop(ctx) srcTenantID := serverutils.TestTenantID() @@ -225,7 +225,7 @@ func TestCutoverBuiltin(t *testing.T) { // Disable the test tenant as the test below looks for a // streaming job assuming that it's within the system tenant. // Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, @@ -413,7 +413,7 @@ func TestCutoverFractionProgressed(t *testing.T) { }, }, }, - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) diff --git a/pkg/ccl/streamingccl/streamproducer/producer_job_test.go b/pkg/ccl/streamingccl/streamproducer/producer_job_test.go index 46dc66d1b6c3..c6eaedb3df72 100644 --- a/pkg/ccl/streamingccl/streamproducer/producer_job_test.go +++ b/pkg/ccl/streamingccl/streamproducer/producer_job_test.go @@ -105,7 +105,7 @@ func TestStreamReplicationProducerJob(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails within a test tenant. More investigation // is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, diff --git a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go index 69557138c390..fc9b440613a8 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go @@ -235,7 +235,7 @@ func TestReplicationStreamInitialization(t *testing.T) { // This test fails when run from within a test tenant. This is likely // due to the lack of support for tenant streaming, but more // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, @@ -331,7 +331,7 @@ func TestStreamPartition(t *testing.T) { base.TestServerArgs{ // Test fails within a test tenant. More investigation is required. // Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer cleanup() testTenantName := roachpb.TenantName("test-tenant") @@ -481,7 +481,7 @@ func TestStreamAddSSTable(t *testing.T) { h, cleanup := replicationtestutils.NewReplicationHelper(t, base.TestServerArgs{ // Test hangs when run within the default test tenant. Tracked with // #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer cleanup() testTenantName := roachpb.TenantName("test-tenant") @@ -571,7 +571,7 @@ func TestCompleteStreamReplication(t *testing.T) { Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer cleanup() srcTenantID := serverutils.TestTenantID() @@ -654,7 +654,7 @@ func TestStreamDeleteRange(t *testing.T) { h, cleanup := replicationtestutils.NewReplicationHelper(t, base.TestServerArgs{ // Test hangs when run within the default test tenant. Tracked with // #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer cleanup() testTenantName := roachpb.TenantName("test-tenant") diff --git a/pkg/ccl/testccl/sqlccl/run_control_test.go b/pkg/ccl/testccl/sqlccl/run_control_test.go index edcb77c86392..98abe8260dd5 100644 --- a/pkg/ccl/testccl/sqlccl/run_control_test.go +++ b/pkg/ccl/testccl/sqlccl/run_control_test.go @@ -44,7 +44,7 @@ func makeRunControlTestCases(t *testing.T) ([]runControlTestCase, func()) { t, 2 /* numNodes */, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Disable the implicit default test tenant so that we can start our own. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ReplicationMode: base.ReplicationManual, }, diff --git a/pkg/ccl/testccl/sqlccl/temp_table_clean_test.go b/pkg/ccl/testccl/sqlccl/temp_table_clean_test.go index 2f4775ec7242..6617e54034cb 100644 --- a/pkg/ccl/testccl/sqlccl/temp_table_clean_test.go +++ b/pkg/ccl/testccl/sqlccl/temp_table_clean_test.go @@ -95,7 +95,9 @@ func TestTenantTempTableCleanup(t *testing.T) { tc := serverutils.StartNewTestCluster( t, 3 /* numNodes */, base.TestClusterArgs{ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ - Settings: settings, + // Disable the default test tenant so that we can start it. + DefaultTestTenant: base.TestTenantDisabled, + Settings: settings, }, }, ) diff --git a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go index 479e24f12b5f..3d90f250887c 100644 --- a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go +++ b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go @@ -53,6 +53,7 @@ func TestGCTenantRemovesSpanConfigs(t *testing.T) { ctx := context.Background() ts, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + DefaultTestTenant: base.TestTenantProbabilistic, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ // Disable the system tenant's reconciliation process so that we can @@ -475,7 +476,7 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { ctx := context.Background() args := base.TestServerArgs{ // Disable the implicit default test tenant so that we can start our own. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, diff --git a/pkg/ccl/testccl/sqlstatsccl/sql_stats_test.go b/pkg/ccl/testccl/sqlstatsccl/sql_stats_test.go index 9372a25d8a6d..c1e6ac1bee5d 100644 --- a/pkg/ccl/testccl/sqlstatsccl/sql_stats_test.go +++ b/pkg/ccl/testccl/sqlstatsccl/sql_stats_test.go @@ -59,7 +59,7 @@ func TestSQLStatsRegions(t *testing.T) { Tiers: []roachpb.Tier{{Key: "region", Value: regionNames[i%len(regionNames)]}}, }, // We'll start our own test tenant manually below. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, } } diff --git a/pkg/ccl/testutilsccl/alter_primary_key.go b/pkg/ccl/testutilsccl/alter_primary_key.go index eebe19e37998..50b7d2b7af8e 100644 --- a/pkg/ccl/testutilsccl/alter_primary_key.go +++ b/pkg/ccl/testutilsccl/alter_primary_key.go @@ -64,7 +64,7 @@ func AlterPrimaryKeyCorrectZoneConfigTest( var db *gosql.DB params, _ := tests.CreateTestServerParams() // Test fails within a test tenant. Tracked with #76378. - params.DisableDefaultTestTenant = true + params.DefaultTestTenant = base.TestTenantDisabled params.Locality.Tiers = []roachpb.Tier{ {Key: "region", Value: "ajstorm-1"}, } diff --git a/pkg/ccl/workloadccl/allccl/all_test.go b/pkg/ccl/workloadccl/allccl/all_test.go index 6d1123afcc84..b6bb703ac3dc 100644 --- a/pkg/ccl/workloadccl/allccl/all_test.go +++ b/pkg/ccl/workloadccl/allccl/all_test.go @@ -90,9 +90,9 @@ func TestAllRegisteredImportFixture(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // The test tenant needs to be disabled for this test until // we address #75449. - DisableDefaultTestTenant: true, - UseDatabase: "d", - SQLMemoryPoolSize: sqlMemoryPoolSize, + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: "d", + SQLMemoryPoolSize: sqlMemoryPoolSize, }) defer s.Stopper().Stop(ctx) sqlutils.MakeSQLRunner(db).Exec(t, `CREATE DATABASE d`) @@ -153,8 +153,8 @@ func TestAllRegisteredSetup(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Need to disable the test tenant here until we resolve // #75449 as this test makes use of import through a fixture. - DisableDefaultTestTenant: true, - UseDatabase: "d", + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: "d", }) defer s.Stopper().Stop(ctx) sqlutils.MakeSQLRunner(db).Exec(t, `CREATE DATABASE d`) diff --git a/pkg/cli/democluster/demo_cluster.go b/pkg/cli/democluster/demo_cluster.go index 7eaa775eb1a5..02a591065170 100644 --- a/pkg/cli/democluster/demo_cluster.go +++ b/pkg/cli/democluster/demo_cluster.go @@ -888,7 +888,7 @@ func (demoCtx *Context) testServerArgsForTransientCluster( EnableDemoLoginEndpoint: true, // Demo clusters by default will create their own tenants, so we // don't need to create them here. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ diff --git a/pkg/cli/democluster/demo_cluster_test.go b/pkg/cli/democluster/demo_cluster_test.go index 879a8dda4ba8..ef1a401154fa 100644 --- a/pkg/cli/democluster/demo_cluster_test.go +++ b/pkg/cli/democluster/demo_cluster_test.go @@ -67,7 +67,7 @@ func TestTestServerArgsForTransientCluster(t *testing.T) { sqlPoolMemorySize: 2 << 10, cacheSize: 1 << 10, expected: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, PartOfCluster: true, JoinAddr: "127.0.0.1", DisableTLSForHTTP: true, @@ -92,7 +92,7 @@ func TestTestServerArgsForTransientCluster(t *testing.T) { sqlPoolMemorySize: 4 << 10, cacheSize: 4 << 10, expected: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, PartOfCluster: true, JoinAddr: "127.0.0.1", Addr: "127.0.0.1:1336", diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher_test.go b/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher_test.go index cd16d98783fc..5747078368fd 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher_test.go @@ -33,7 +33,7 @@ func TestWatchAuthErr(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - host, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + host, _, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer host.Stopper().Stop(ctx) var _ = kvtenantccl.Connector{} diff --git a/pkg/kv/kvserver/client_tenant_test.go b/pkg/kv/kvserver/client_tenant_test.go index 599064a9d9f5..973ed252c38f 100644 --- a/pkg/kv/kvserver/client_tenant_test.go +++ b/pkg/kv/kvserver/client_tenant_test.go @@ -166,6 +166,8 @@ func TestTenantRateLimiter(t *testing.T) { timeSource := timeutil.NewManualTime(t0) s, sqlDB, db := serverutils.StartServer(t, base.TestServerArgs{ + // Disable the default test tenant so that we can start our own. + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ TenantRateKnobs: tenantrate.TestingKnobs{ @@ -326,6 +328,8 @@ func TestTenantCtx(t *testing.T) { getErr := make(chan error) pushErr := make(chan error) s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + // Disable the default test tenant since we're going to create our own. + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { diff --git a/pkg/kv/kvserver/gc/gc_int_test.go b/pkg/kv/kvserver/gc/gc_int_test.go index 325067ba2a3c..e89bb5c98ec4 100644 --- a/pkg/kv/kvserver/gc/gc_int_test.go +++ b/pkg/kv/kvserver/gc/gc_int_test.go @@ -92,7 +92,7 @@ func TestEndToEndGC(t *testing.T) { WallClock: manualClock, }, }, - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) tc.Start(t) diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index 437db05d7be9..5286ca9394b6 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -539,7 +539,7 @@ func TestReplicateQueueUpAndDownReplicateNonVoters(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails with the default tenant. Disabling and // tracking with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ ConfigureScratchRange: true, @@ -2091,10 +2091,10 @@ func TestReplicateQueueAcquiresInvalidLeases(t *testing.T) { // statuses pre and post enabling the replicate queue. ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ - Settings: st, - DisableDefaultTestTenant: true, - ScanMinIdleTime: time.Millisecond, - ScanMaxIdleTime: time.Millisecond, + Settings: st, + DefaultTestTenant: base.TestTenantDisabled, + ScanMinIdleTime: time.Millisecond, + ScanMaxIdleTime: time.Millisecond, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ StickyEngineRegistry: stickyEngineRegistry, diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go index 2a853fb0416a..d73a03f2b008 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go @@ -36,7 +36,7 @@ func TestDecodeCapabilities(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, // system.tenants only exists for the system tenant + DefaultTestTenant: base.TestTenantDisabled, // system.tenants only exists for the system tenant }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 63cde4a4b840..d65add5b89c0 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -144,7 +144,7 @@ func TestAdminDebugExpVar(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -169,7 +169,7 @@ func TestAdminDebugMetrics(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -194,7 +194,7 @@ func TestAdminDebugPprof(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -215,7 +215,7 @@ func TestAdminDebugTrace(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -244,7 +244,7 @@ func TestAdminDebugAuth(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) ts := s.(*TestServer) @@ -302,7 +302,7 @@ func TestAdminDebugRedirect(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) ts := s.(*TestServer) @@ -402,7 +402,7 @@ func TestAdminAPIDatabases(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) ts := s.(*TestServer) @@ -549,7 +549,7 @@ func TestAdminAPIDatabaseDoesNotExist(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -565,7 +565,7 @@ func TestAdminAPIDatabaseSQLInjection(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -709,7 +709,7 @@ func TestAdminAPITableDoesNotExist(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -733,7 +733,7 @@ func TestAdminAPITableSQLInjection(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails with // it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -761,7 +761,7 @@ func TestAdminAPITableDetails(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) ts := s.(*TestServer) @@ -931,7 +931,7 @@ func TestAdminAPIZoneDetails(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) ts := s.(*TestServer) @@ -1038,7 +1038,7 @@ func TestAdminAPIUsers(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -1079,7 +1079,7 @@ func TestAdminAPIEvents(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -1209,7 +1209,7 @@ func TestAdminAPISettings(t *testing.T) { s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -1330,7 +1330,7 @@ func TestAdminAPIUIData(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -1441,7 +1441,7 @@ func TestAdminAPIUISeparateData(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -1484,7 +1484,7 @@ func TestClusterAPI(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -1538,7 +1538,7 @@ func TestHealthAPI(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) ts := s.(*TestServer) @@ -1633,7 +1633,7 @@ func TestAdminAPIJobs(t *testing.T) { s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: &jobs.TestingKnobs{ IntervalOverrides: jobs.TestingIntervalOverrides{ @@ -1844,7 +1844,7 @@ func TestAdminAPIJobsDetails(t *testing.T) { s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(conn) @@ -1969,7 +1969,7 @@ func TestAdminAPILocations(t *testing.T) { s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(conn) @@ -2014,7 +2014,7 @@ func TestAdminAPIQueryPlan(t *testing.T) { s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(conn) @@ -2052,7 +2052,7 @@ func TestAdminAPIRangeLogByRangeID(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -2121,7 +2121,7 @@ func TestAdminAPIFullRangeLog(t *testing.T) { base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ DisableSplitQueue: true, @@ -2462,7 +2462,7 @@ func TestEndpointTelemetryBasic(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(context.Background()) @@ -3014,8 +3014,8 @@ func TestAdminDecommissionedOperations(t *testing.T) { ServerArgs: base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, - Insecure: true, // allows admin client without setting up certs + DefaultTestTenant: base.TestTenantDisabled, + Insecure: true, // allows admin client without setting up certs }, }) defer tc.Stopper().Stop(ctx) @@ -3182,7 +3182,7 @@ func TestAdminPrivilegeChecker(t *testing.T) { s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) @@ -3333,7 +3333,7 @@ func TestDatabaseAndTableIndexRecommendations(t *testing.T) { s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disable the default test tenant for now as this tests fails // with it enabled. Tracked with #81590. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ UnusedIndexRecommendKnobs: &idxusage.UnusedIndexRecommendationTestingKnobs{ GetCreatedAt: stubTime.getCreatedAt, diff --git a/pkg/server/server_controller_test.go b/pkg/server/server_controller_test.go index 0949ce473701..24ab209e5011 100644 --- a/pkg/server/server_controller_test.go +++ b/pkg/server/server_controller_test.go @@ -28,7 +28,7 @@ func TestServerController(t *testing.T) { ctx := context.Background() s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) @@ -70,7 +70,7 @@ func TestSQLErrorUponInvalidTenant(t *testing.T) { ctx := context.Background() s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) diff --git a/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go b/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go index 06a0e7c77b89..2d85c8177fc5 100644 --- a/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go +++ b/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go @@ -50,7 +50,7 @@ func TestSystemConfigWatcher(t *testing.T, skipSecondary bool) { base.TestServerArgs{ // Test runs against tenant, so no need to create the default // test tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, ) defer s.Stopper().Stop(ctx) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index e84ec554bc41..1af1c4467031 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -295,7 +295,7 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { cfg.TempStorageConfig.Settings = st } - cfg.DisableDefaultTestTenant = params.DisableDefaultTestTenant + cfg.DisableDefaultTestTenant = params.DefaultTestTenant == base.TestTenantDisabled if cfg.TestingKnobs.Store == nil { cfg.TestingKnobs.Store = &kvserver.StoreTestingKnobs{} @@ -339,6 +339,11 @@ type TestServer struct { // by default, but longer term we may allow for the creation of multiple // test tenants for more advanced testing. testTenants []serverutils.TestTenantInterface + // disableStartTenantError is set to an error if the test server should + // prevent starting any tenants manually. This is used to prevent tests that + // have not explicitly disabled probabilistic testing, or opted in to it, from + // starting a tenant to avoid unexpected behavior. + disableStartTenantError error } var _ serverutils.TestServerInterface = &TestServer{} @@ -543,7 +548,7 @@ func (ts *TestServer) maybeStartDefaultTestTenant(ctx context.Context) error { // If the flag has been set to disable the default test tenant, don't start // it here. - if ts.params.DisableDefaultTestTenant || ts.cfg.DisableDefaultTestTenant { + if ts.params.DefaultTestTenant == base.TestTenantDisabled || ts.cfg.DisableDefaultTestTenant { return nil } @@ -571,6 +576,18 @@ func (ts *TestServer) maybeStartDefaultTestTenant(ctx context.Context) error { // setup in StartTenant below. params.TestingKnobs.Server = &TestingKnobs{} + // Temporarily disable the error that is returned if a tenant should not be started manually, + // so that we can start the default test tenant internally here. + disableStartTenantError := ts.disableStartTenantError + if ts.disableStartTenantError != nil { + ts.disableStartTenantError = nil + } + defer func() { + if disableStartTenantError != nil { + ts.disableStartTenantError = disableStartTenantError + } + }() + tenant, err := ts.StartTenant(ctx, params) if err != nil { return err @@ -916,6 +933,11 @@ func (ts *TestServer) StartSharedProcessTenant( return testTenant, sqlDB, err } +// DisableStartTenant is part of TestServerInterface. +func (ts *TestServer) DisableStartTenant(reason error) { + ts.disableStartTenantError = reason +} + // MigrationServer is part of the TestTenantInterface. func (t *TestTenant) MigrationServer() interface{} { return t.migrationServer @@ -925,6 +947,9 @@ func (t *TestTenant) MigrationServer() interface{} { func (ts *TestServer) StartTenant( ctx context.Context, params base.TestTenantArgs, ) (serverutils.TestTenantInterface, error) { + if ts.disableStartTenantError != nil { + return nil, ts.disableStartTenantError + } // Determine if we need to create the tenant before starting it. ie := ts.InternalExecutor().(*sql.InternalExecutor) diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go index 805817c915d3..32d535a34634 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go @@ -151,7 +151,7 @@ func BenchmarkKVAccessorUpdate(b *testing.B) { ServerArgs: base.TestServerArgs{ // Requires span_configuration table which is not visible // from secondary tenants. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) @@ -193,7 +193,7 @@ func TestKVAccessorPagination(t *testing.T) { ServerArgs: base.TestServerArgs{ // Requires span_configuration table which is not visible // from secondary tenants. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/sql/backfill_protected_timestamp_test.go b/pkg/sql/backfill_protected_timestamp_test.go index 7c1439972c7d..611ab7578212 100644 --- a/pkg/sql/backfill_protected_timestamp_test.go +++ b/pkg/sql/backfill_protected_timestamp_test.go @@ -71,8 +71,8 @@ func TestValidationWithProtectedTS(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - Settings: settings, - DisableDefaultTestTenant: true, + Settings: settings, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ BeforeExecute: func(ctx context.Context, sql string, descriptors *descs.Collection) { diff --git a/pkg/sql/catalog/internal/catkv/catalog_reader_test.go b/pkg/sql/catalog/internal/catkv/catalog_reader_test.go index 6df53bacf777..0ebd7dce48be 100644 --- a/pkg/sql/catalog/internal/catkv/catalog_reader_test.go +++ b/pkg/sql/catalog/internal/catkv/catalog_reader_test.go @@ -51,7 +51,7 @@ func TestDataDriven(t *testing.T) { ctx := context.Background() datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ UseTransactionalDescIDGenerator: true, diff --git a/pkg/sql/colenc/encode_test.go b/pkg/sql/colenc/encode_test.go index 0a0b6b494c7d..1aa1b862aac0 100644 --- a/pkg/sql/colenc/encode_test.go +++ b/pkg/sql/colenc/encode_test.go @@ -55,7 +55,7 @@ import ( // TestingGetTableDescriptor which isn't supported in multi-tenancy. // Tracked with #76378. var testArgs = base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, } // TestEncoderEquality tests that the vector encoder and the row based encoder diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index 301ae303a785..69836b37147c 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -1191,7 +1191,8 @@ func TestTransactionDeadline(t *testing.T) { } testClusterArgs := base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - Knobs: knobs, + DefaultTestTenant: base.TestTenantDisabled, + Knobs: knobs, }, } tc := serverutils.StartNewTestCluster(t, 1, testClusterArgs) diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index a5d6bf7874e4..8541882e9fd6 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -933,7 +933,7 @@ func TestTxnContentionEventsTableMultiTenant(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test is designed to run with explicit tenants. No need to // implicitly create a tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/sql/generate_objects_test.go b/pkg/sql/generate_objects_test.go index 9903629b83bf..f5d34062b0db 100644 --- a/pkg/sql/generate_objects_test.go +++ b/pkg/sql/generate_objects_test.go @@ -38,7 +38,7 @@ func BenchmarkGenerateObjects(b *testing.B) { s, sqlDB, _ := serverutils.StartServer(b, base.TestServerArgs{ // Secondary tenants have unreasonably low span config limits. We // can't use them yet for this test. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) diff --git a/pkg/sql/importer/exportcsv_test.go b/pkg/sql/importer/exportcsv_test.go index 894ef94adf7f..4c6e75d526f7 100644 --- a/pkg/sql/importer/exportcsv_test.go +++ b/pkg/sql/importer/exportcsv_test.go @@ -64,10 +64,10 @@ func setupExportableBank(t *testing.T, nodes, rows int) (*sqlutils.SQLRunner, st base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Disabled due to underlying tests' use of SCATTER. - DisableDefaultTestTenant: true, - ExternalIODir: dir, - UseDatabase: "test", - DisableSpanConfigs: true, + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: dir, + UseDatabase: "test", + DisableSpanConfigs: true, }, }, ) diff --git a/pkg/sql/importer/exportparquet_test.go b/pkg/sql/importer/exportparquet_test.go index c97625d29c9a..17d2aff60a5b 100644 --- a/pkg/sql/importer/exportparquet_test.go +++ b/pkg/sql/importer/exportparquet_test.go @@ -201,9 +201,9 @@ func TestRandomParquetExports(t *testing.T) { srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Test fails when run within a test tenant. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, - UseDatabase: dbName, - ExternalIODir: dir, + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: dbName, + ExternalIODir: dir, }) ctx := context.Background() defer srv.Stopper().Stop(ctx) @@ -298,9 +298,9 @@ func TestBasicParquetTypes(t *testing.T) { srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Test fails when run within a test tenant. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, - UseDatabase: dbName, - ExternalIODir: dir, + DefaultTestTenant: base.TestTenantDisabled, + UseDatabase: dbName, + ExternalIODir: dir, }) ctx := context.Background() defer srv.Stopper().Stop(ctx) diff --git a/pkg/sql/importer/import_into_test.go b/pkg/sql/importer/import_into_test.go index 30255210fe7d..9b3db42e408f 100644 --- a/pkg/sql/importer/import_into_test.go +++ b/pkg/sql/importer/import_into_test.go @@ -60,7 +60,7 @@ func TestProtectedTimestampsDuringImportInto(t *testing.T) { args := base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ // Test hangs within a test tenant. More investigation is required. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, } tc := testcluster.StartTestCluster(t, 1, args) diff --git a/pkg/sql/importer/import_processor_test.go b/pkg/sql/importer/import_processor_test.go index 8ca7c926adb3..15a703b3ddde 100644 --- a/pkg/sql/importer/import_processor_test.go +++ b/pkg/sql/importer/import_processor_test.go @@ -689,7 +689,7 @@ func TestCSVImportCanBeResumed(t *testing.T) { base.TestServerArgs{ // Hangs when run from a test tenant. More investigation is // required here. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ @@ -797,7 +797,7 @@ func TestCSVImportMarksFilesFullyProcessed(t *testing.T) { base.TestServerArgs{ // Test hangs when run within a test tenant. More investigation // is required here. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 909c21615ef3..34d4692d805e 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -2042,9 +2042,9 @@ func TestFailedImportGC(t *testing.T) { // Test fails within a test tenant. This may be because we're trying // to access files in nodelocal://1, which is off node. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, - SQLMemoryPoolSize: 256 << 20, - ExternalIODir: baseDir, + DefaultTestTenant: base.TestTenantDisabled, + SQLMemoryPoolSize: 256 << 20, + ExternalIODir: baseDir, Knobs: base.TestingKnobs{ GCJob: &sql.GCJobTestingKnobs{ RunBeforeResume: func(_ jobspb.JobID) error { <-blockGC; return nil }, @@ -2151,8 +2151,8 @@ func TestImportIntoCSVCancel(t *testing.T) { }, JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, - DisableDefaultTestTenant: true, - ExternalIODir: baseDir, + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: baseDir, }}) defer tc.Stopper().Stop(ctx) conn := tc.ServerConn(0) @@ -2208,9 +2208,9 @@ func TestImportCSVStmt(t *testing.T) { tc := serverutils.StartNewTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ // Test fails when run within a test tenant. More // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, - SQLMemoryPoolSize: 256 << 20, - ExternalIODir: baseDir, + DefaultTestTenant: base.TestTenantDisabled, + SQLMemoryPoolSize: 256 << 20, + ExternalIODir: baseDir, }}) defer tc.Stopper().Stop(ctx) conn := tc.ServerConn(0) @@ -2781,9 +2781,9 @@ func TestImportObjectLevelRBAC(t *testing.T) { tc := serverutils.StartNewTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ // Test fails when run within a test tenant. More investigation // is required. Tracked with #76378. - DisableDefaultTestTenant: true, - ExternalIODir: baseDir, - SQLMemoryPoolSize: 256 << 20, + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: baseDir, + SQLMemoryPoolSize: 256 << 20, }}) defer tc.Stopper().Stop(ctx) conn := tc.ServerConn(0) @@ -2960,8 +2960,8 @@ func TestImportRetriesBreakerOpenFailure(t *testing.T) { ctx := context.Background() tc := serverutils.StartNewTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, - ExternalIODir: datapathutils.TestDataPath(t, "csv")}}) + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: datapathutils.TestDataPath(t, "csv")}}) defer tc.Stopper().Stop(ctx) aboutToRunDSP := make(chan struct{}) @@ -3041,8 +3041,8 @@ func TestImportIntoCSV(t *testing.T) { }, // Test fails when run within a test tenant. More investigation // is required. Tracked with #76378. - DisableDefaultTestTenant: true, - ExternalIODir: baseDir}}) + DefaultTestTenant: base.TestTenantDisabled, + ExternalIODir: baseDir}}) defer tc.Stopper().Stop(ctx) conn := tc.ServerConn(0) @@ -4707,7 +4707,7 @@ func TestImportDefaultWithResume(t *testing.T) { base.TestServerArgs{ // Test hangs when run within a test tenant. More investigation // is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), DistSQL: &execinfra.TestingKnobs{ @@ -5229,7 +5229,7 @@ func TestImportControlJobRBAC(t *testing.T) { ServerArgs: base.TestServerArgs{ // Test fails when run within a test tenant. More investigation // is required. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) defer tc.Stopper().Stop(ctx) @@ -6413,7 +6413,7 @@ func TestImportPgDumpSchemas(t *testing.T) { // Test fails within a test tenant. More investigation is required. // Tracked with #76378. args := mkArgs() - args.DisableDefaultTestTenant = true + args.DefaultTestTenant = base.TestTenantDisabled tc := serverutils.StartNewTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: args}) defer tc.Stopper().Stop(ctx) conn := tc.ServerConn(0) @@ -7047,7 +7047,7 @@ func TestImportJobEventLogging(t *testing.T) { args := base.TestServerArgs{ExternalIODir: baseDir} // Test fails within a test tenant. More investigation is required. // Tracked with #76378. - args.DisableDefaultTestTenant = true + args.DefaultTestTenant = base.TestTenantDisabled args.Knobs = base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()} params := base.TestClusterArgs{ServerArgs: args} tc := serverutils.StartNewTestCluster(t, nodes, params) diff --git a/pkg/sql/importer/read_import_mysql_test.go b/pkg/sql/importer/read_import_mysql_test.go index 2a46fa9dee36..edc696322129 100644 --- a/pkg/sql/importer/read_import_mysql_test.go +++ b/pkg/sql/importer/read_import_mysql_test.go @@ -135,8 +135,8 @@ func readMysqlCreateFrom( s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ // Test relies on descriptor validation, which doesn't seem to be // supported within secondary tenants. Tracked with #76378. - DisableDefaultTestTenant: true, - Settings: cluster.MakeTestingClusterSettings(), + DefaultTestTenant: base.TestTenantDisabled, + Settings: cluster.MakeTestingClusterSettings(), }) ctx := context.Background() defer s.Stopper().Stop(ctx) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 2da438683a9a..609603aa79ea 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1315,13 +1315,21 @@ func (t *logicTest) newCluster( shouldUseMVCCRangeTombstonesForPointDeletes := useMVCCRangeTombstonesForPointDeletes && !serverArgs.DisableUseMVCCRangeTombstonesForPointDeletes ignoreMVCCRangeTombstoneErrors := globalMVCCRangeTombstone || shouldUseMVCCRangeTombstonesForPointDeletes + defaultTestTenant := t.cfg.DefaultTestTenant + if defaultTestTenant == base.TestTenantEnabled { + // If the test tenant is explicitly enabled then `logic test` will handle + // the creation of a configured test tenant, thus for this case we disable + // the implicit creation of the default test tenant. + defaultTestTenant = base.TestTenantDisabled + } + params := base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ SQLMemoryPoolSize: serverArgs.MaxSQLMemoryLimit, TempStorageConfig: base.DefaultTestTempStorageConfigWithSize( cluster.MakeTestingClusterSettings(), tempStorageDiskLimit, ), - DisableDefaultTestTenant: t.cfg.UseTenant || t.cfg.DisableDefaultTestTenant, + DefaultTestTenant: defaultTestTenant, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ // The consistency queue makes a lot of noisy logs during logic tests. @@ -1363,7 +1371,7 @@ func (t *logicTest) newCluster( } cfg := t.cfg - if cfg.UseTenant { + if cfg.DefaultTestTenant == base.TestTenantEnabled { // In the tenant case we need to enable replication in order to split and // relocate ranges correctly. params.ReplicationMode = base.ReplicationAuto @@ -1434,7 +1442,7 @@ func (t *logicTest) newCluster( } connsForClusterSettingChanges := []*gosql.DB{t.cluster.ServerConn(0)} - if cfg.UseTenant { + if cfg.DefaultTestTenant == base.TestTenantEnabled { t.tenantAddrs = make([]string, cfg.NumNodes) for i := 0; i < cfg.NumNodes; i++ { tenantArgs := base.TestTenantArgs{ @@ -1500,7 +1508,7 @@ func (t *logicTest) newCluster( // If we've created a tenant (either explicitly, or probabilistically and // implicitly) set any necessary cluster settings to override blocked // behavior. - if cfg.UseTenant || t.cluster.StartedDefaultTestTenant() { + if cfg.DefaultTestTenant == base.TestTenantEnabled || t.cluster.StartedDefaultTestTenant() { conn := t.cluster.StorageClusterConn() clusterSettings := toa.clusterSettings @@ -2909,7 +2917,7 @@ func (t *logicTest) processSubtest( t.setUser(fields[1], nodeIdx) // In multi-tenant tests, we may need to also create database test when // we switch to a different tenant. - if t.cfg.UseTenant && strings.HasPrefix(fields[1], "host-cluster-") { + if t.cfg.DefaultTestTenant == base.TestTenantEnabled && strings.HasPrefix(fields[1], "host-cluster-") { if _, err := t.db.Exec("CREATE DATABASE IF NOT EXISTS test; USE test;"); err != nil { return errors.Wrapf(err, "error creating database on admin tenant") } diff --git a/pkg/sql/logictest/logictestbase/BUILD.bazel b/pkg/sql/logictest/logictestbase/BUILD.bazel index 2241a66bd885..f955657d7260 100644 --- a/pkg/sql/logictest/logictestbase/BUILD.bazel +++ b/pkg/sql/logictest/logictestbase/BUILD.bazel @@ -8,6 +8,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/logictest/logictestbase", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/build", "//pkg/clusterversion", "//pkg/roachpb", diff --git a/pkg/sql/logictest/logictestbase/logictestbase.go b/pkg/sql/logictest/logictestbase/logictestbase.go index 5d26e04f21e4..d6a58b4aeecd 100644 --- a/pkg/sql/logictest/logictestbase/logictestbase.go +++ b/pkg/sql/logictest/logictestbase/logictestbase.go @@ -21,6 +21,7 @@ import ( "strconv" "strings" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -59,11 +60,13 @@ type TestClusterConfig struct { // DisableUpgrade prevents the cluster from automatically upgrading to the // latest version. DisableUpgrade bool - // If true, a sql tenant server will be started and pointed at a node in the - // cluster. Connections on behalf of the logic test will go to that tenant. - UseTenant bool - // Disable the default test tenant. - DisableDefaultTestTenant bool + + // If a test tenant is explicitly enabled, a sql tenant server will be started + // and pointed at a node in the cluster. Connections on behalf of the logic + // test will go to that tenant. Otherwise, the default test tenant logic will + // be followed + DefaultTestTenant base.DefaultTestTenantOptions + // IsCCLConfig should be true for any config that can only be run with a CCL // binary. IsCCLConfig bool @@ -262,7 +265,7 @@ var LogicTestConfigs = []TestClusterConfig{ OverrideDistSQLMode: "off", // local is the configuration where we run all tests which have bad // interactions with the default test tenant. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, DeclarativeCorpusCollection: true, }, { @@ -306,7 +309,7 @@ var LogicTestConfigs = []TestClusterConfig{ // this mode which try to modify zone configurations and we're more // restrictive in the way we allow zone configs to be modified by // secondary tenants. See #75569 for more info. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, { Name: "5node-disk", @@ -323,7 +326,7 @@ var LogicTestConfigs = []TestClusterConfig{ // dev testlogic ccl --files 3node-tenant --subtest $SUBTEST Name: threeNodeTenantConfigName, NumNodes: 3, - UseTenant: true, + DefaultTestTenant: base.TestTenantEnabled, IsCCLConfig: true, OverrideDistSQLMode: "on", DeclarativeCorpusCollection: true, @@ -337,7 +340,7 @@ var LogicTestConfigs = []TestClusterConfig{ // dev testlogic ccl --files 3node-tenant-multiregion --subtests $SUBTESTS Name: "3node-tenant-multiregion", NumNodes: 3, - UseTenant: true, + DefaultTestTenant: base.TestTenantEnabled, IsCCLConfig: true, OverrideDistSQLMode: "on", DeclarativeCorpusCollection: true, @@ -414,14 +417,14 @@ var LogicTestConfigs = []TestClusterConfig{ // Need to disable the default test tenant here until we have the // locality optimized search working in multi-tenant configurations. // Tracked with #80678. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, DeclarativeCorpusCollection: true, }, { Name: "multiregion-9node-3region-3azs-tenant", NumNodes: 9, Localities: multiregion9node3region3azsLocalities, - UseTenant: true, + DefaultTestTenant: base.TestTenantEnabled, DeclarativeCorpusCollection: true, }, { diff --git a/pkg/sql/multitenant_admin_function_test.go b/pkg/sql/multitenant_admin_function_test.go index 9e15144c3078..49466d4a4d98 100644 --- a/pkg/sql/multitenant_admin_function_test.go +++ b/pkg/sql/multitenant_admin_function_test.go @@ -245,6 +245,7 @@ func (tc testCase) runTest( if numNodes == 0 { numNodes = 1 } + cfg.ServerArgs.DefaultTestTenant = base.TestTenantProbabilistic testCluster := serverutils.StartNewTestCluster(t, numNodes, cfg.TestClusterArgs) defer testCluster.Stopper().Stop(ctx) diff --git a/pkg/sql/run_control_test.go b/pkg/sql/run_control_test.go index 5f6e15bfbbc4..c32c0c5a81c8 100644 --- a/pkg/sql/run_control_test.go +++ b/pkg/sql/run_control_test.go @@ -879,6 +879,7 @@ func TestTenantStatementTimeoutAdmissionQueueCancelation(t *testing.T) { } params := base.TestServerArgs{ + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ AdmissionControl: &admission.Options{ MaxCPUSlots: numBlockers, diff --git a/pkg/sql/schemachanger/scbuild/builder_test.go b/pkg/sql/schemachanger/scbuild/builder_test.go index a07e96057991..66b21573683d 100644 --- a/pkg/sql/schemachanger/scbuild/builder_test.go +++ b/pkg/sql/schemachanger/scbuild/builder_test.go @@ -112,7 +112,7 @@ func TestBuildDataDriven(t *testing.T) { } { t.Run(depsType.name, func(t *testing.T) { s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }) defer s.Stopper().Stop(ctx) tdb := sqlutils.MakeSQLRunner(sqlDB) diff --git a/pkg/sql/schemachanger/scdecomp/decomp_test.go b/pkg/sql/schemachanger/scdecomp/decomp_test.go index a318409b8b63..25d1145deffb 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp_test.go +++ b/pkg/sql/schemachanger/scdecomp/decomp_test.go @@ -33,7 +33,7 @@ func TestDecomposeToElements(t *testing.T) { newCluster := func(t *testing.T, knobs *scexec.TestingKnobs) (_ serverutils.TestServerInterface, _ *gosql.DB, cleanup func()) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }) return nil, tc.ServerConn(0), func() { tc.Stopper().Stop(ctx) } diff --git a/pkg/sql/schemachanger/sctest/end_to_end.go b/pkg/sql/schemachanger/sctest/end_to_end.go index 7e008e756a7b..b356c100c894 100644 --- a/pkg/sql/schemachanger/sctest/end_to_end.go +++ b/pkg/sql/schemachanger/sctest/end_to_end.go @@ -73,7 +73,7 @@ func SingleNodeCluster( ) (serverutils.TestServerInterface, *gosql.DB, func()) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disabled due to a failure in TestBackupRestore. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ SQLDeclarativeSchemaChanger: knobs, JobsTestingKnobs: newJobsKnobs(), @@ -98,7 +98,7 @@ func SingleNodeMixedCluster( } s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ // Disabled due to a failure in TestBackupRestore. Tracked with #76378. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ BinaryVersionOverride: targetVersion, diff --git a/pkg/sql/sqlinstance/instancestorage/instancecache_test.go b/pkg/sql/sqlinstance/instancestorage/instancecache_test.go index 180677e328ba..cda075290af3 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancecache_test.go +++ b/pkg/sql/sqlinstance/instancestorage/instancecache_test.go @@ -71,7 +71,7 @@ func TestRangeFeed(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - host, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + host, _, _ := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestTenantDisabled}) defer host.Stopper().Stop(ctx) var _ = kvtenantccl.Connector{} diff --git a/pkg/sql/sqltestutils/telemetry.go b/pkg/sql/sqltestutils/telemetry.go index d9a4d957be3b..2b4f27fb0e4b 100644 --- a/pkg/sql/sqltestutils/telemetry.go +++ b/pkg/sql/sqltestutils/telemetry.go @@ -157,6 +157,7 @@ func (tt *telemetryTest) Start(t *testing.T, serverArgs []base.TestServerArgs) { diagSrvURL := tt.diagSrv.URL() mapServerArgs := make(map[int]base.TestServerArgs, len(serverArgs)) for i, v := range serverArgs { + v.DefaultTestTenant = base.TestTenantDisabled v.Knobs.Server = &server.TestingKnobs{ DiagnosticsTestingKnobs: diagnostics.TestingKnobs{ OverrideReportingURL: &diagSrvURL, diff --git a/pkg/sql/tests/server_params.go b/pkg/sql/tests/server_params.go index b1b31b10b54e..588e17f606ed 100644 --- a/pkg/sql/tests/server_params.go +++ b/pkg/sql/tests/server_params.go @@ -28,7 +28,7 @@ func CreateTestServerParams() (base.TestServerArgs, *CommandFilters) { params := base.TestServerArgs{} // Disable the default test tenant as limits to the number of spans in a // secondary tenant cause this test to fail. Tracked with #76378. - params.DisableDefaultTestTenant = true + params.DefaultTestTenant = base.TestTenantDisabled params.Knobs = CreateTestingKnobs() params.Knobs.Store = &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ diff --git a/pkg/sql/ttl/ttljob/ttljob_test.go b/pkg/sql/ttl/ttljob/ttljob_test.go index 277e0fa51757..54b9003a66a8 100644 --- a/pkg/sql/ttl/ttljob/ttljob_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_test.go @@ -96,9 +96,17 @@ func newRowLevelTTLTestJobTestHelper( if numNodes > 1 { replicationMode = base.ReplicationManual } + + var defaultTestTenant base.DefaultTestTenantOptions + // Disable the default test tenant when running multi-tenant tests. + if testMultiTenant { + defaultTestTenant = base.TestTenantDisabled + } + testCluster := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: replicationMode, ServerArgs: base.TestServerArgs{ + DefaultTestTenant: defaultTestTenant, Knobs: baseTestingKnobs, InsecureWebAccess: true, }, diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 192bb8794df9..f464fe338b92 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -66,13 +66,25 @@ const ( tenantModeDefault = "default" ) +var PreventStartTenantError = errors.New("attempting to manually start a tenant while " + + "DefaultTestTenant is set to TestTenantProbabilisticOnly") + // ShouldStartDefaultTestTenant determines whether a default test tenant // should be started for test servers or clusters, to serve SQL traffic by // default. It defaults to 50% probability, but can be overridden by the // tenantMode test flag or the COCKROACH_TEST_TENANT_MODE environment variable. // If both the environment variable and the test flag are set, the environment // variable wins out. -func ShouldStartDefaultTestTenant(t testing.TB) bool { +func ShouldStartDefaultTestTenant(t testing.TB, serverArgs base.TestServerArgs) bool { + // Explicit cases for enabling or disabling the default test tenant. + if serverArgs.DefaultTestTenant == base.TestTenantDisabled { + return false + } + if serverArgs.DefaultTestTenant == base.TestTenantEnabled { + return true + } + + // Probabilistic cases for enabling or disabling the default test tenant. var defaultProbabilityOfStartingTestTenant = 0.5 if skip.UnderBench() { // Until #83461 is resolved, we want to make sure that we don't use the @@ -256,6 +268,10 @@ type TestServerInterface interface { // StartSharedProcessTenant() for a tenant simulating a shared-memory server. StartTenant(ctx context.Context, params base.TestTenantArgs) (TestTenantInterface, error) + // DisableStartTenant prevents manual starting of tenants. If an attempt at + // starting a tenant is made, the server will return the specified error. + DisableStartTenant(reason error) + // ScratchRange splits off a range suitable to be used as KV scratch space. // (it doesn't overlap system spans or SQL tables). // @@ -317,15 +333,14 @@ func InitTestServerFactory(impl TestServerFactory) { func StartServer( t testing.TB, params base.TestServerArgs, ) (TestServerInterface, *gosql.DB, *kv.DB) { - if !params.DisableDefaultTestTenant { - // Determine if we should probabilistically start a test tenant - // for this server. - startDefaultSQLServer := ShouldStartDefaultTestTenant(t) - if !startDefaultSQLServer { - // If we're told not to start a test tenant, set the - // disable flag explicitly. - params.DisableDefaultTestTenant = true - } + preventFurtherTenants := params.DefaultTestTenant == base.TestTenantProbabilisticOnly + // Determine if we should probabilistically start a test tenant + // for this server. + startDefaultSQLServer := ShouldStartDefaultTestTenant(t, params) + if !startDefaultSQLServer { + // If we're told not to start a test tenant, set the + // disable flag explicitly. + params.DefaultTestTenant = base.TestTenantDisabled } s, err := NewServer(params) @@ -341,6 +356,10 @@ func StartServer( t.Log(DefaultTestTenantMessage) } + if preventFurtherTenants { + s.DisableStartTenant(PreventStartTenantError) + } + goDB := OpenDBConn( t, s.ServingSQLAddr(), params.UseDatabase, params.Insecure, s.Stopper()) @@ -434,6 +453,7 @@ func StartServerRaw(t testing.TB, args base.TestServerArgs) (TestServerInterface func StartTenant( t testing.TB, ts TestServerInterface, params base.TestTenantArgs, ) (TestTenantInterface, *gosql.DB) { + tenant, err := ts.StartTenant(context.Background(), params) if err != nil { t.Fatalf("%+v", err) diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 92c3ba7c961c..e5b4a4aad3d6 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -63,8 +63,8 @@ type TestCluster struct { // Connection to the storage cluster. Typically, the first connection in // Conns, but could be different if we're transparently running in a test - // tenant (see the DisableDefaultTestTenant flag of base.TestServerArgs for - // more detail). + // tenant (see the DefaultTestTenant flag of base.TestServerArgs for more + // detail). storageConn *gosql.DB stopper *stop.Stopper mu struct { @@ -355,7 +355,7 @@ func (tc *TestCluster) Start(t testing.TB) { // (validated below). probabilisticallyStartTestTenant := false if !tc.Servers[0].Cfg.DisableDefaultTestTenant { - probabilisticallyStartTestTenant = serverutils.ShouldStartDefaultTestTenant(t) + probabilisticallyStartTestTenant = serverutils.ShouldStartDefaultTestTenant(t, tc.serverArgs[0]) } startedTestTenant := true @@ -571,6 +571,12 @@ func (tc *TestCluster) AddServer(serverArgs base.TestServerArgs) (*server.TestSe } s := srv.(*server.TestServer) + // If we only allowed probabilistic starting of the test tenant, we disable + // starting additional tenants, even if we didn't start the test tenant. + if serverArgs.DefaultTestTenant == base.TestTenantProbabilisticOnly { + s.DisableStartTenant(serverutils.PreventStartTenantError) + } + tc.Servers = append(tc.Servers, s) tc.serverArgs = append(tc.serverArgs, serverArgs) diff --git a/pkg/ts/server_test.go b/pkg/ts/server_test.go index 8939be3d627e..4a2ae7281833 100644 --- a/pkg/ts/server_test.go +++ b/pkg/ts/server_test.go @@ -304,7 +304,7 @@ func TestServerQueryTenant(t *testing.T) { defer leaktest.AfterTest(t)() testCluster := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ DisableTimeSeriesMaintenanceQueue: true, diff --git a/pkg/util/tracing/collector/collector_test.go b/pkg/util/tracing/collector/collector_test.go index 67a3c2674c2a..9e60522c887a 100644 --- a/pkg/util/tracing/collector/collector_test.go +++ b/pkg/util/tracing/collector/collector_test.go @@ -219,7 +219,7 @@ func TestClusterInflightTraces(t *testing.T) { ServerArgs: base.TestServerArgs{ // We'll create our own tenants, to ensure they exist as opposed to them // being created randomly. - DisableDefaultTestTenant: true, + DefaultTestTenant: base.TestTenantDisabled, }, }