Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion pkg/base/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,11 +56,12 @@ go_test(
"main_test.go",
"node_id_test.go",
"store_spec_test.go",
"test_server_args_test.go",
],
args = ["-test.timeout=55s"],
data = glob(["testdata/**"]),
embed = [":base"],
deps = [
":base",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
Expand Down
137 changes: 124 additions & 13 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,25 +240,26 @@ type DefaultTestTenantOptions struct {
label string
}

type testBehavior int8
type testBehavior int16

const (
ttProb testBehavior = iota
ttEnabled
ttEnabled testBehavior = 1 << iota
ttDisabled
ttSharedProcess
ttExternalProcess
)

var (
// TestTenantProbabilisticOnly starts the test under a virtual
// cluster on a probabilistic basis. It will also prevent the
// starting of additional virtual clusters by raising an error if it
// is attempted. This is the default behavior.
TestTenantProbabilisticOnly = DefaultTestTenantOptions{testBehavior: ttProb, allowAdditionalTenants: false}
TestTenantProbabilisticOnly = DefaultTestTenantOptions{allowAdditionalTenants: false}

// TestTenantProbabilistic starts the test under a virtual
// cluster on a probabilistic basis. It allows the starting of
// additional virtual clusters.
TestTenantProbabilistic = DefaultTestTenantOptions{testBehavior: ttProb, allowAdditionalTenants: true}
TestTenantProbabilistic = DefaultTestTenantOptions{allowAdditionalTenants: true}

// TestTenantAlwaysEnabled will always redirect the test workload to
// a virtual cluster. This is useful for quickly verifying that a
Expand All @@ -269,6 +270,24 @@ var (
// to use TestTenantProbabilistic or TestTenantProbabilisticOnly.
TestTenantAlwaysEnabled = DefaultTestTenantOptions{testBehavior: ttEnabled, allowAdditionalTenants: true}

// ExternalTestTenantAlwaysEnabled will always redirect the test workload to
// an external process virtual cluster. This is useful for quickly verifying that a
// test works under cluster virtualization.
//
// Note: this value should not be used for checked in test code
// unless there is a good reason to do so. We want the common case
// to use TestTenantProbabilistic or TestTenantProbabilisticOnly.
ExternalTestTenantAlwaysEnabled = DefaultTestTenantOptions{testBehavior: ttEnabled | ttExternalProcess, allowAdditionalTenants: true}

// SharedTestTenantAlwaysEnabled will always redirect the test workload to
// a shared process virtual cluster. This is useful for quickly verifying that a
// test works under cluster virtualization.
//
// Note: this value should not be used for checked in test code
// unless there is a good reason to do so. We want the common case
// to use TestTenantProbabilistic or TestTenantProbabilisticOnly.
SharedTestTenantAlwaysEnabled = DefaultTestTenantOptions{testBehavior: ttEnabled | ttSharedProcess, allowAdditionalTenants: true}

// TODOTestTenantDisabled should not be used anymore. Use the
// other values instead.
// TODO(#76378): Review existing tests and use the proper value instead.
Expand Down Expand Up @@ -308,16 +327,39 @@ func (do DefaultTestTenantOptions) AllowAdditionalTenants() bool {
}

func (do DefaultTestTenantOptions) TestTenantAlwaysEnabled() bool {
return do.testBehavior == ttEnabled
return do.testBehavior&ttEnabled != 0
}

func (do DefaultTestTenantOptions) TestTenantAlwaysDisabled() bool {
return do.testBehavior == ttDisabled
return do.testBehavior&ttDisabled != 0
}

func (do DefaultTestTenantOptions) TestTenantNoDecisionMade() bool {
// Exactly one of ttEnabled or ttDisabled must be set.
if (do.testBehavior&ttEnabled != 0) == (do.testBehavior&ttDisabled != 0) {
return true
}
if do.testBehavior&ttEnabled != 0 {
// If ttEnabled is set, then exactly one of ttSharedProcess or
// ttExternalProcess must be set.
if (do.testBehavior&ttExternalProcess != 0) == (do.testBehavior&ttSharedProcess != 0) {
return true
}
}
return false
}

func (do DefaultTestTenantOptions) SharedProcessMode() bool {
return do.testBehavior&ttSharedProcess != 0
}

func (do DefaultTestTenantOptions) ExternalProcessMode() bool {
return do.testBehavior&ttExternalProcess != 0
}

// WarnImplicitInterfaces indicates whether to warn when the test code
// uses ApplicationLayerInterface or StorageLayerInterface
// implicitely.
// implicitly.
func (do DefaultTestTenantOptions) WarnImplicitInterfaces() bool {
return !do.noWarnImplicitInterfaces
}
Expand All @@ -343,7 +385,7 @@ func TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(

// TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet can be
// used to disable virtualization because the test exercises a feature
// known not to work with virtualization enabled yet but we wish it to
// known not to work with virtualization enabled yet, but we wish it to
// eventually.
//
// It should link to a github issue with label C-bug
Expand All @@ -359,16 +401,82 @@ func TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(
}
}

// TestDoesNotWorkWithSharedProcessModeButWeDontKnowWhyYet can be used to
// disable selecting a shared process test virtual cluster probabilistically
// because the test doesn't appear to be compatible with it, and we don't
// understand it yet.
//
// The `baseOptions` are adjusted to restrict the default test virtual cluster
// process mode selection to an external process virtual cluster only. Using
// this function with `baseOptions` that explicitly disable a test virtual
// cluster, or opt specifically for a shared process test virtual cluster does
// not make sense and will cause a panic.
//
// It should link to a github issue with label C-test-failure.
func TestDoesNotWorkWithSharedProcessModeButWeDontKnowWhyYet(
baseOptions DefaultTestTenantOptions, issueNumber int,
) DefaultTestTenantOptions {
if baseOptions.testBehavior&ttDisabled != 0 {
panic("test behavior cannot be disabled, please refer to one of the other options to disable secondary virtual clusters with a reason")
}
if baseOptions.testBehavior&ttSharedProcess != 0 {
panic("test behavior cannot be set to a shared process.")
}
return DefaultTestTenantOptions{
testBehavior: baseOptions.testBehavior | ttExternalProcess,
allowAdditionalTenants: baseOptions.allowAdditionalTenants,
issueNum: issueNumber,
label: "C-test-failure",
}
}

// TestIsForStuffThatShouldWorkWithSharedProcessModeButDoesntYet can be used to
// disable selecting a shared process test virtual cluster probabilistically
// because the test exercises a feature known not to work with a shared process
// virtual cluster, but we wish it to eventually.
//
// The `baseOptions` are adjusted to restrict the default test virtual cluster
// process mode selection to an external process virtual cluster only. Using
// this function with `baseOptions` that explicitly disable a test virtual
// cluster, or opt specifically for a shared process test virtual cluster does
// not make sense and will cause a panic.
//
// It should link to a github issue with label C-bug and the issue should be
// linked to an epic under INI-213 or INI-214.
func TestIsForStuffThatShouldWorkWithSharedProcessModeButDoesntYet(
baseOptions DefaultTestTenantOptions, issueNumber int,
) DefaultTestTenantOptions {
if baseOptions.testBehavior&ttDisabled != 0 {
panic("test behavior cannot be disabled, please refer to one of the other options to disable secondary virtual clusters with a reason")
}
if baseOptions.testBehavior&ttSharedProcess != 0 {
panic("test behavior cannot be set to a shared process.")
}
return DefaultTestTenantOptions{
testBehavior: baseOptions.testBehavior | ttExternalProcess,
allowAdditionalTenants: baseOptions.allowAdditionalTenants,
issueNum: issueNumber,
label: "C-bug",
}
}

// InternalNonDefaultDecision builds a sentinel value used inside a
// mechanism in serverutils. Should not be used by tests directly.
func InternalNonDefaultDecision(
baseArg DefaultTestTenantOptions, enable bool,
baseArg DefaultTestTenantOptions, enable bool, shared bool,
) DefaultTestTenantOptions {
mode := ttDisabled
var tb testBehavior
if enable {
mode = ttEnabled
tb |= ttEnabled
if shared {
tb |= ttSharedProcess
} else {
tb |= ttExternalProcess
}
} else {
tb |= ttDisabled
}
baseArg.testBehavior = mode
baseArg.testBehavior = tb
return baseArg
}

Expand Down Expand Up @@ -450,6 +558,9 @@ type TestSharedProcessTenantArgs struct {
// automatically open a connection to the server. That's equivalent to running
// SET DATABASE=foo, which works even if the database doesn't (yet) exist.
UseDatabase string

// Skip check for tenant existence when running the test.
SkipTenantCheck bool
}

// TestTenantArgs are the arguments to TestServer.StartTenant.
Expand Down
46 changes: 46 additions & 0 deletions pkg/base/test_server_args_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package base

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/require"
)

func TestDefaultTestTenantOptionsBehavior(t *testing.T) {
defer leaktest.AfterTest(t)()
testCases := []struct {
name string
tb testBehavior
expectedNoDecision bool
}{
// Decision missing enabled or disabled flag
{name: "no decision made", tb: 0, expectedNoDecision: true},
// Decision missing process mode
{name: "no decision on tenant process mode", tb: ttEnabled, expectedNoDecision: true},
// Decision made to not run test tenant
{name: "decision to not run test tenant", tb: ttDisabled, expectedNoDecision: false},
// Decision made to run test tenant as an external process
{name: "decision to run external test tenant", tb: ttEnabled | ttExternalProcess, expectedNoDecision: false},
// Decision made to run test tenant as a shared process
{name: "decision to run shared test tenant", tb: ttEnabled | ttSharedProcess, expectedNoDecision: false},
// Decision missing enabled or disabled flag with additional erroneous flag
{name: "no decision made with erroneous flag", tb: ttExternalProcess, expectedNoDecision: true},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
d := DefaultTestTenantOptions{testBehavior: tc.tb}
require.Equal(t, tc.expectedNoDecision, d.TestTenantNoDecisionMade())
})
}
}
5 changes: 3 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7427,8 +7427,9 @@ func TestClientDisconnect(t *testing.T) {
// Make credentials for the new connection.
sqlDB.Exec(t, `CREATE USER testuser`)
sqlDB.Exec(t, `GRANT admin TO testuser`)
pgURL, cleanup := sqlutils.PGUrl(t, tc.ApplicationLayer(0).AdvSQLAddr(),
"TestClientDisconnect-testuser", url.User("testuser"))
pgURL, cleanup := tc.ApplicationLayer(0).PGUrl(t,
serverutils.CertsDirPrefix("TestClientDisconnect-testuser"), serverutils.User("testuser"),
)
defer cleanup()

// Kick off the job on a new connection which we're going to close.
Expand Down
6 changes: 4 additions & 2 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,8 +240,10 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, name string, user string) *
if db, ok := d.sqlDBs[key]; ok {
return db
}
addr := d.firstNode[name].ApplicationLayer().AdvSQLAddr()
pgURL, cleanup := sqlutils.PGUrl(t, addr, "TestBackupRestoreDataDriven", url.User(user))
s := d.firstNode[name].ApplicationLayer()
pgURL, cleanup := s.PGUrl(
t, serverutils.CertsDirPrefix("TestBackupRestoreDataDriven"), serverutils.User(user),
)
d.cleanupFns = append(d.cleanupFns, cleanup)

base, err := pq.NewConnector(pgURL.String())
Expand Down
13 changes: 12 additions & 1 deletion pkg/ccl/changefeedccl/sink_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -475,11 +475,19 @@ func TestSQLSink(t *testing.T) {
}

ctx := context.Background()
s, sqlDBRaw, _ := serverutils.StartServer(t, base.TestServerArgs{UseDatabase: "d"})
s, sqlDBRaw, _ := serverutils.StartServer(t, base.TestServerArgs{
DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSharedProcessModeButDoesntYet(
base.TestTenantProbabilistic, 112863,
),
UseDatabase: "d",
})
defer s.Stopper().Stop(ctx)
sqlDB := sqlutils.MakeSQLRunner(sqlDBRaw)
sqlDB.Exec(t, `CREATE DATABASE d`)

// TODO(herko): When the issue relating to this test is fixed, update this
// to use PGUrl on the server interface instead.
// See: https://github.com/cockroachdb/cockroach/issues/112863
pgURL, cleanup := sqlutils.PGUrl(t, s.ApplicationLayer().AdvSQLAddr(), t.Name(), url.User(username.RootUser))
defer cleanup()
pgURL.Path = `d`
Expand All @@ -491,6 +499,9 @@ func TestSQLSink(t *testing.T) {
targets.Add(barTopic.GetTargetSpecification())

const testTableName = `sink`
// TODO(herko): `makeSQLSink` does not expect "options" to be present in the URL, find out if
// this is a bug, or if we should add it to to `consumeParam` in the `makeSQLSink` function.
// See: https://github.com/cockroachdb/cockroach/issues/112863.
sink, err := makeSQLSink(sinkURL{URL: &pgURL}, testTableName, targets, nilMetricsRecorderBuilder)
require.NoError(t, err)
require.NoError(t, sink.(*sqlSink).Dial())
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importerccl/ccl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -388,7 +388,7 @@ func TestExportInsideTenant(t *testing.T) {
defer cleanupDir()

srv := serverutils.StartServerOnly(t, base.TestServerArgs{
DefaultTestTenant: base.TestTenantProbabilistic,
DefaultTestTenant: base.TestControlsTenantsExplicitly,
ExternalIODir: dir,
})
defer srv.Stopper().Stop(context.Background())
Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/serverccl/server_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,9 @@ func TestTenantHTTP(t *testing.T) {
s := serverutils.StartServerOnly(t, base.TestServerArgs{
// This test is specific to secondary tenants; no need to run it
// using the system tenant.
DefaultTestTenant: base.TestTenantAlwaysEnabled,
DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSharedProcessModeButDoesntYet(
base.TestTenantAlwaysEnabled, 113187,
),
})
defer s.Stopper().Stop(ctx)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ func TestDropTableLowersSpanCount(t *testing.T) {

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
DefaultTestTenant: base.TestTenantProbabilistic,
DefaultTestTenant: base.TestControlsTenantsExplicitly,
}})

defer tc.Stopper().Stop(ctx)
Expand Down
Loading