Skip to content

Commit

Permalink
serverutils: prevent start tenant when probabilistic behaviour is active
Browse files Browse the repository at this point in the history
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
  • Loading branch information
herkolategan committed Apr 3, 2023
1 parent bebd2dd commit cfa4375
Show file tree
Hide file tree
Showing 115 changed files with 508 additions and 370 deletions.
36 changes: 29 additions & 7 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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.
Expand Down
12 changes: 6 additions & 6 deletions pkg/bench/foreachdb.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())

Expand All @@ -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)

Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/alter_backup_schedule_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_tenant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
64 changes: 33 additions & 31 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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"},
Expand Down Expand Up @@ -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"},
Expand Down Expand Up @@ -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

Expand Down Expand Up @@ -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`
Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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,
}},
)

Expand Down Expand Up @@ -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)
Expand All @@ -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)
Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backuprand/backup_rand_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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/"
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backuptestutils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
},
Expand Down
Loading

0 comments on commit cfa4375

Please sign in to comment.