From ed516d0e1e103ad9d2e52736fbc030182aefd41c Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 9 Aug 2023 03:16:26 +0200 Subject: [PATCH] server,testutils: remove complexity There is a saying (paraphrasing) that it always takes more work removing unwanted complexity than it takes to add it. This is an example of that. Prior to this commit, there was an "interesting" propagation of the flag that decides whether or not to define a test tenant for test servers and clusters. In a nutshell, we had: - an "input" flag in `base.TestServerArgs`, which remained mostly immutable - a boolean decided once by `ShouldStartDefaultTestTenant()` either in: - `serverutils.StartServerOnlyE` - or `testcluster.Start` - that boolean choice was then propagated to `server.testServer` via _another_ boolean config flag in `server.BaseConfig` - both the 2nd boolean and the original input flag were then again checked when the time came to do the work (in `maybeStartDefaultTestTenant`). Additional complexity was then incurred by the need of `TestCluster` to make the determination just once (and not once per server). This commit cuts through all the layers of complexity by simply propagating the choice of `ShouldStartDefaultTestTenant()` back into the `TestServerArgs` and only ever reading from that subsequently. Release note: None --- pkg/base/test_server_args.go | 3 - pkg/server/config.go | 3 - pkg/server/testserver.go | 40 +++++++----- pkg/testutils/lint/lint_test.go | 1 + pkg/testutils/serverutils/api.go | 9 +-- pkg/testutils/serverutils/test_server_shim.go | 61 +++++++++++-------- pkg/testutils/testcluster/testcluster.go | 57 ++++++----------- 7 files changed, 81 insertions(+), 93 deletions(-) diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 38a5ffee2fd8..8dc4b6c3bc46 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -281,9 +281,6 @@ var ( // InternalNonDefaultDecision is a sentinel value used inside a // mechanism in serverutils. Should not be used by tests directly. - // - // TODO(#76378): Investigate how we can remove the need for this - // sentinel value. InternalNonDefaultDecision = DefaultTestTenantOptions{testBehavior: ttDisabled, allowAdditionalTenants: true} ) diff --git a/pkg/server/config.go b/pkg/server/config.go index 8a29c24d4510..b918e38fb3f0 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -204,9 +204,6 @@ type BaseConfig struct { // Environment Variable: COCKROACH_DISABLE_SPAN_CONFIGS SpanConfigsDisabled bool - // Disables the default test tenant. - DisableDefaultTestTenant bool - // TestingKnobs is used for internal test controls only. TestingKnobs base.TestingKnobs diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 85abf40c11da..8903ec18997b 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -307,9 +307,6 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { cfg.TempStorageConfig.Settings = st } - // TODO(#76378): Review this assignment to ensure it does not interfere with randomization. - cfg.DisableDefaultTestTenant = params.DefaultTestTenant.TestTenantAlwaysDisabled() - if cfg.TestingKnobs.Store == nil { cfg.TestingKnobs.Store = &kvserver.StoreTestingKnobs{} } @@ -568,12 +565,7 @@ func (ts *testServer) TestTenants() []serverutils.ApplicationLayerInterface { // DefaultTestTenantDisabled is part of the serverutils.TenantControlInterface. func (ts *testServer) DefaultTestTenantDisabled() bool { - return ts.cfg.DisableDefaultTestTenant -} - -// DisableDefaultTestTenant is part of the serverutils.TenantControlInterface. -func (ts *testServer) DisableDefaultTestTenant() { - ts.cfg.DisableDefaultTestTenant = true + return ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() } // maybeStartDefaultTestTenant might start a test tenant. This can then be used @@ -583,9 +575,16 @@ func (ts *testServer) DisableDefaultTestTenant() { // enterprise enabled build. This is due to licensing restrictions on the MT // capabilities. func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { + if !(ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() || + ts.params.DefaultTestTenant.TestTenantAlwaysEnabled()) { + return errors.WithHint( + errors.AssertionFailedf("programming error: no decision taken about the default test tenant"), + "Maybe add the missing call to serverutils.ShouldStartDefaultTestTenant()?") + } + // If the flag has been set to disable the default test tenant, don't start // it here. - if ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() || ts.cfg.DisableDefaultTestTenant { + if ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() { return nil } @@ -594,7 +593,10 @@ func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { log.Shoutf(ctx, severity.WARNING, "test tenant requested by configuration, but code organization prevents start!\n%v", err) // If not enterprise enabled, we won't be able to use SQL Servers so eat // the error and return without creating/starting a SQL server. - ts.cfg.DisableDefaultTestTenant = true + // + // TODO(knz/yahor): Remove this - as we discussed this ought to work + // now even when not enterprise enabled. + ts.params.DefaultTestTenant = base.TODOTestTenantDisabled return nil // nolint:returnerrcheck } @@ -665,7 +667,15 @@ func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { // testServer.AdvRPCAddr() after Start() for client connections. // Use testServer.Stopper().Stop() to shutdown the server after the test // completes. -func (ts *testServer) Start(ctx context.Context) error { +func (ts *testServer) Start(ctx context.Context) (retErr error) { + defer func() { + if retErr != nil { + // Use a separate context to avoid using an already-cancelled + // context in closers. + ts.Stopper().Stop(context.Background()) + } + }() + if err := ts.topLevelServer.PreStart(ctx); err != nil { return err } @@ -678,16 +688,16 @@ func (ts *testServer) Start(ctx context.Context) error { ); err != nil { return err } + + // Let clients connect. if err := ts.topLevelServer.AcceptClients(ctx); err != nil { return err } if err := ts.maybeStartDefaultTestTenant(ctx); err != nil { - // We're failing the call to this function but we've already started - // the testServer above. Stop it here to avoid leaking the server. - ts.Stopper().Stop(context.Background()) return err } + go func() { // If the server requests a shutdown, do that simply by stopping the // stopper. diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 19800bdc43a1..7ef6f766c493 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -2337,6 +2337,7 @@ func TestLint(t *testing.T) { ":!server/storage_api/decommission_test.go", ":!server/storage_api/health_test.go", ":!server/storage_api/rangelog_test.go", + ":!server/testserver.go", ":!sql/catalog/internal/catkv/catalog_reader_test.go", ":!sql/importer/import_processor_test.go", ":!sql/importer/import_stmt_test.go", diff --git a/pkg/testutils/serverutils/api.go b/pkg/testutils/serverutils/api.go index 055d96e1ff80..37ed5357125c 100644 --- a/pkg/testutils/serverutils/api.go +++ b/pkg/testutils/serverutils/api.go @@ -53,6 +53,9 @@ type TestServerInterface interface { // Start runs the server. This is pre-called by StartServer(). // It is provided for tests that use the TestServerFactory directly // (mostly 'cockroach demo'). + // + // For convenience, the caller can assume that Stop() has been called + // already if Start() fails with an error. Start(context.Context) error // Stop stops the server. This must be called at the end of a test @@ -444,12 +447,6 @@ type TenantControlInterface interface { // TODO(knz): Verify whether this accessor is needed. This should // be simplified. DefaultTestTenantDisabled() bool - - // DisableDefaultTestTenant prevents the server from starting the - // service for the default test tenant. - // TODO(knz): Verify whether this accessor is needed. This should - // be simplified. - DisableDefaultTestTenant() } // StorageLayerInterface defines accessors to the storage layer of a diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 99ce3b9b7bd1..cc2cb5ffbf63 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -59,26 +59,40 @@ var PreventStartTenantError = errors.New("attempting to manually start a server // ShouldStartDefaultTestTenant determines whether a default test tenant // should be started for test servers or clusters, to serve SQL traffic by -// default. -// This can be overridden either via the build tag `metamorphic_disable` +// default. It returns a new base.DefaultTestTenantOptions that reflects +// the decision that was taken. +// +// The decision can be overridden either via the build tag `metamorphic_disable` // or just for test tenants via COCKROACH_TEST_TENANT. -func ShouldStartDefaultTestTenant(t TestLogger, serverArgs base.TestServerArgs) bool { +// +// This function is included in package 'serverutils' instead of 'server.testServer' +// directly so that it only gets linked into test code (and to avoid a linter +// error that 'skip' must only be used in test code). +func ShouldStartDefaultTestTenant( + t TestLogger, baseArg base.DefaultTestTenantOptions, +) (retval base.DefaultTestTenantOptions) { + defer func() { + if !(retval.TestTenantAlwaysEnabled() || retval.TestTenantAlwaysDisabled()) { + panic(errors.AssertionFailedf("programming error: no decision was actually taken")) + } + }() + // Explicit cases for enabling or disabling the default test tenant. - if serverArgs.DefaultTestTenant.TestTenantAlwaysEnabled() { - return true + if baseArg.TestTenantAlwaysEnabled() { + return baseArg } - if serverArgs.DefaultTestTenant.TestTenantAlwaysDisabled() { - if issueNum, label := serverArgs.DefaultTestTenant.IssueRef(); issueNum != 0 { + if baseArg.TestTenantAlwaysDisabled() { + if issueNum, label := baseArg.IssueRef(); issueNum != 0 { t.Logf("cluster virtualization disabled due to issue: #%d (expected label: %s)", issueNum, label) } - return false + return baseArg } if skip.UnderBench() { // Until #83461 is resolved, we want to make sure that we don't use the // multi-tenant setup so that the comparison against old single-tenant // SHAs in the benchmarks is fair. - return false + return base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(83461) } // Obey the env override if present. @@ -87,7 +101,10 @@ func ShouldStartDefaultTestTenant(t TestLogger, serverArgs base.TestServerArgs) if err != nil { panic(err) } - return v + if v { + return base.TestTenantAlwaysEnabled + } + return base.InternalNonDefaultDecision } // Note: we ask the metamorphic framework for a "disable" value, instead @@ -97,7 +114,10 @@ func ShouldStartDefaultTestTenant(t TestLogger, serverArgs base.TestServerArgs) if enabled && t != nil { t.Log(DefaultTestTenantMessage) } - return enabled + if enabled { + return base.TestTenantAlwaysEnabled + } + return base.InternalNonDefaultDecision } var srvFactoryImpl TestServerFactory @@ -134,17 +154,9 @@ type TestFataler interface { // server configuration messages. func StartServerOnlyE(t TestLogger, params base.TestServerArgs) (TestServerInterface, error) { allowAdditionalTenants := params.DefaultTestTenant.AllowAdditionalTenants() - // 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. - // - // TODO(#76378): review the definition of params.DefaultTestTenant - // so we do not need this weird sentinel value. - params.DefaultTestTenant = base.InternalNonDefaultDecision - } + // Update the flags with the actual decision as to whether we should + // start the service for a default test tenant. + params.DefaultTestTenant = ShouldStartDefaultTestTenant(t, params.DefaultTestTenant) s, err := NewServer(params) if err != nil { @@ -154,14 +166,9 @@ func StartServerOnlyE(t TestLogger, params base.TestServerArgs) (TestServerInter ctx := context.Background() if err := s.Start(ctx); err != nil { - s.Stopper().Stop(ctx) return nil, err } - if s.StartedDefaultTestTenant() && t != nil { - t.Log(DefaultTestTenantMessage) - } - if !allowAdditionalTenants { s.DisableStartTenant(PreventStartTenantError) } diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index ff7c6babb0f8..edf405d92ddc 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -73,6 +73,8 @@ type TestCluster struct { serverArgs []base.TestServerArgs clusterArgs base.TestClusterArgs + defaultTestTenantOptions base.DefaultTestTenantOptions + t serverutils.TestFataler } @@ -276,6 +278,17 @@ func NewTestCluster( noLocalities = false } + // Find out how to do the default test tenant. Pick the setting + // from either the first server with explicit args, or the top-level server args. + defaultTestTenantOptions := tc.clusterArgs.ServerArgs.DefaultTestTenant + for i := 0; i < nodes; i++ { + if args, ok := tc.clusterArgs.ServerArgsPerNode[i]; ok { + defaultTestTenantOptions = args.DefaultTestTenant + break + } + } + tc.defaultTestTenantOptions = serverutils.ShouldStartDefaultTestTenant(t, defaultTestTenantOptions) + var firstListener net.Listener for i := 0; i < nodes; i++ { var serverArgs base.TestServerArgs @@ -365,16 +378,6 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { errCh = make(chan error, nodes) } - // Determine if we should probabilistically start a test tenant for the - // cluster. We key off of the DisableDefaultTestTenant flag of the first - // server in the cluster since they should all be set to the same value - // (validated below). - probabilisticallyStartTestTenant := false - if !tc.Servers[0].DefaultTestTenantDisabled() { - probabilisticallyStartTestTenant = serverutils.ShouldStartDefaultTestTenant(t, tc.serverArgs[0]) - } - - startedTestTenant := true disableLBS := false for i := 0; i < nodes; i++ { // Disable LBS if any server has a very low scan interval. @@ -382,21 +385,6 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { disableLBS = true } - // If we're not probabilistically starting the test tenant, disable - // its start and set the "started" flag accordingly. We need to do this - // with two separate if checks because the DisableDefaultTestTenant flag - // could have been set coming into this function by the caller. - if !probabilisticallyStartTestTenant { - tc.Servers[i].DisableDefaultTestTenant() - } - if tc.Servers[i].DefaultTestTenantDisabled() { - if startedTestTenant && i > 0 { - t.Fatal(errors.Newf("starting only some nodes with a test tenant is not"+ - "currently supported - attempted to disable SQL sever on node %d", i)) - } - startedTestTenant = false - } - if tc.clusterArgs.ParallelStart { go func(i int) { errCh <- tc.startServer(i, tc.serverArgs[i]) @@ -412,10 +400,6 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { } } - if tc.StartedDefaultTestTenant() { - t.Log(serverutils.DefaultTestTenantMessage) - } - if tc.clusterArgs.ParallelStart { for i := 0; i < nodes; i++ { if err := <-errCh; err != nil { @@ -439,10 +423,7 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { } } - // No need to disable the merge queue for SQL servers, as they don't have - // access to that cluster setting (and ALTER TABLE ... SPLIT AT is not - // supported in SQL servers either). - if !startedTestTenant && tc.clusterArgs.ReplicationMode == base.ReplicationManual { + if tc.clusterArgs.ReplicationMode == base.ReplicationManual { // We've already disabled the merge queue via testing knobs above, but ALTER // TABLE ... SPLIT AT will throw an error unless we also disable merges via // the cluster setting. @@ -602,17 +583,15 @@ func (tc *TestCluster) AddServer( serverArgs.Addr = serverArgs.Listener.Addr().String() } + // Inject the decision that was made about whether or not to start a + // test tenant server, into this new server's configuration. + serverArgs.DefaultTestTenant = tc.defaultTestTenantOptions + s, err := serverutils.NewServer(serverArgs) if err != nil { return nil, err } - // 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)