Skip to content

Commit

Permalink
server,testutils: remove complexity
Browse files Browse the repository at this point in the history
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
  • Loading branch information
knz committed Aug 9, 2023
1 parent 1925449 commit ed516d0
Show file tree
Hide file tree
Showing 7 changed files with 81 additions and 93 deletions.
3 changes: 0 additions & 3 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -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}
)

Expand Down
3 changes: 0 additions & 3 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
40 changes: 25 additions & 15 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{}
}
Expand Down Expand Up @@ -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
Expand All @@ -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
}

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

Expand Down Expand Up @@ -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
}
Expand All @@ -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.
Expand Down
1 change: 1 addition & 0 deletions pkg/testutils/lint/lint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
9 changes: 3 additions & 6 deletions pkg/testutils/serverutils/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
61 changes: 34 additions & 27 deletions pkg/testutils/serverutils/test_server_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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 {
Expand All @@ -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)
}
Expand Down
57 changes: 18 additions & 39 deletions pkg/testutils/testcluster/testcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,8 @@ type TestCluster struct {
serverArgs []base.TestServerArgs
clusterArgs base.TestClusterArgs

defaultTestTenantOptions base.DefaultTestTenantOptions

t serverutils.TestFataler
}

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -365,38 +378,13 @@ 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.
if tc.serverArgs[i].ScanInterval > 0 && tc.serverArgs[i].ScanInterval <= 100*time.Millisecond {
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])
Expand All @@ -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 {
Expand All @@ -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.
Expand Down Expand Up @@ -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)

Expand Down

0 comments on commit ed516d0

Please sign in to comment.