Skip to content

Commit

Permalink
testutils: move default test tenant message
Browse files Browse the repository at this point in the history
In order to reduce logging noise but still inform test authors of the default
test tenant, the message has been moved to where there is a `testing.TB`
interface.

Epic: CRDB-18499
  • Loading branch information
herkolategan committed Mar 30, 2023
1 parent 37ac1e5 commit 346bbc9
Show file tree
Hide file tree
Showing 10 changed files with 70 additions and 38 deletions.
4 changes: 2 additions & 2 deletions pkg/cli/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func newCLITestWithArgs(params TestCLIParams, argsFn func(args *base.TestServerA
if params.NoNodelocal {
args.ExternalIODir = ""
}
s, err := serverutils.StartServerRaw(args)
s, err := serverutils.StartServerRaw(params.T, args)
if err != nil {
c.fail(err)
}
Expand Down Expand Up @@ -208,7 +208,7 @@ func (c *TestCLI) stopServer() {
func (c *TestCLI) RestartServer(params TestCLIParams) {
c.stopServer()
log.Info(context.Background(), "restarting server")
s, err := serverutils.StartServerRaw(base.TestServerArgs{
s, err := serverutils.StartServerRaw(params.T, base.TestServerArgs{
Insecure: params.Insecure,
SSLCertsDir: c.certsDir,
StoreSpecs: params.StoreSpecs,
Expand Down
7 changes: 6 additions & 1 deletion pkg/cmd/reduce/reduce/reducesql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,10 @@ go_library(
go_test(
name = "reducesql_test",
size = "small",
srcs = ["reducesql_test.go"],
srcs = [
"main_test.go",
"reducesql_test.go",
],
args = ["-test.timeout=55s"],
data = glob(["testdata/**"]),
deps = [
Expand All @@ -26,8 +29,10 @@ go_test(
"//pkg/cmd/reduce/reduce",
"//pkg/security/username",
"//pkg/server",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_stretchr_testify//require",
],
)

Expand Down
25 changes: 25 additions & 0 deletions pkg/cmd/reduce/reduce/reducesql/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
// 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 reducesql_test

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
)

func TestMain(m *testing.M) {
serverutils.InitTestServerFactory(server.TestServerFactory)
code := m.Run()
os.Exit(code)
}
26 changes: 12 additions & 14 deletions pkg/cmd/reduce/reduce/reducesql/reducesql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/reduce/reduce"
"github.com/cockroachdb/cockroach/pkg/cmd/reduce/reduce/reducesql"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/jackc/pgx/v4"
"github.com/stretchr/testify/require"
)

var printUnknown = flag.Bool("unknown", false, "print unknown types during walk")
Expand All @@ -33,24 +34,23 @@ func TestReduceSQL(t *testing.T) {
skip.IgnoreLint(t, "unnecessary")
reducesql.LogUnknown = *printUnknown

reduce.Walk(t, "testdata", reducesql.Pretty, isInterestingSQL, reduce.ModeInteresting,
isInterestingSQLWrapper := func(contains string) reduce.InterestingFn {
return isInterestingSQL(t, contains)
}

reduce.Walk(t, "testdata", reducesql.Pretty, isInterestingSQLWrapper, reduce.ModeInteresting,
nil /* chunkReducer */, reducesql.SQLPasses)
}

func isInterestingSQL(contains string) reduce.InterestingFn {
func isInterestingSQL(t *testing.T, contains string) reduce.InterestingFn {
return func(ctx context.Context, f string) (bool, func()) {
args := base.TestServerArgs{
Insecure: true,
}
ts, err := server.TestServerFactory.New(args)
if err != nil {
panic(err)
}
serv := ts.(*server.TestServer)

serv, err := serverutils.StartServerRaw(t, args)
require.NoError(t, err)
defer serv.Stopper().Stop(ctx)
if err := serv.Start(context.Background()); err != nil {
panic(err)
}

options := url.Values{}
options.Add("sslmode", "disable")
Expand All @@ -62,9 +62,7 @@ func isInterestingSQL(contains string) reduce.InterestingFn {
}

db, err := pgx.Connect(ctx, url.String())
if err != nil {
panic(err)
}
require.NoError(t, err)
_, err = db.Exec(ctx, f)
if err == nil {
return false, nil
Expand Down
2 changes: 0 additions & 2 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -263,7 +263,6 @@ go_library(
"//pkg/storage/enginepb",
"//pkg/storage/fs",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/ts",
"//pkg/ts/catalog",
"//pkg/ui",
Expand All @@ -289,7 +288,6 @@ go_library(
"//pkg/util/log/eventpb",
"//pkg/util/log/logcrash",
"//pkg/util/log/logpb",
"//pkg/util/log/severity",
"//pkg/util/metric",
"//pkg/util/mon",
"//pkg/util/netutil",
Expand Down
8 changes: 4 additions & 4 deletions pkg/server/server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ import (
func TestSelfBootstrap(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
s, err := serverutils.StartServerRaw(base.TestServerArgs{})
s, err := serverutils.StartServerRaw(t, base.TestServerArgs{})
if err != nil {
t.Fatal(err)
}
Expand All @@ -88,7 +88,7 @@ func TestPanicRecovery(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

s, err := serverutils.StartServerRaw(base.TestServerArgs{})
s, err := serverutils.StartServerRaw(t, base.TestServerArgs{})
require.NoError(t, err)
defer s.Stopper().Stop(context.Background())
ts := s.(*TestServer)
Expand Down Expand Up @@ -129,7 +129,7 @@ func TestHealthCheck(t *testing.T) {

cfg := zonepb.DefaultZoneConfig()
cfg.NumReplicas = proto.Int32(1)
s, err := serverutils.StartServerRaw(base.TestServerArgs{
s, err := serverutils.StartServerRaw(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &TestingKnobs{
DefaultZoneConfigOverride: &cfg,
Expand Down Expand Up @@ -419,7 +419,7 @@ func TestListenerFileCreation(t *testing.T) {
dir, cleanupFn := testutils.TempDir(t)
defer cleanupFn()

s, err := serverutils.StartServerRaw(base.TestServerArgs{
s, err := serverutils.StartServerRaw(t, base.TestServerArgs{
StoreSpecs: []base.StoreSpec{{
Path: dir,
}},
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,7 @@ func TestStatusEngineStatsJson(t *testing.T) {
dir, cleanupFn := testutils.TempDir(t)
defer cleanupFn()

s, err := serverutils.StartServerRaw(base.TestServerArgs{
s, err := serverutils.StartServerRaw(t, base.TestServerArgs{
StoreSpecs: []base.StoreSpec{{
Path: dir,
}},
Expand Down
12 changes: 0 additions & 12 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,14 +52,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/ts"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/severity"
"github.com/cockroachdb/cockroach/pkg/util/metric"
addrutil "github.com/cockroachdb/cockroach/pkg/util/netutil/addr"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand Down Expand Up @@ -579,16 +577,6 @@ func (ts *TestServer) maybeStartDefaultTestTenant(ctx context.Context) error {
if len(ts.testTenants) == 0 {
ts.testTenants = make([]serverutils.TestTenantInterface, 1)
ts.testTenants[0] = tenant

if !skip.UnderBench() {
// Now that we've started the first tenant, log this fact for easier
// debugging. Skip the logging if we're running a benchmark (because
// these INFO messages break the benchstat utility).
log.Shout(context.Background(), severity.INFO,
"Running test with the default test tenant. "+
"If you are only seeing a test case failure when this message appears, there may be a "+
"problem with your test case running within tenants.")
}
} else {
// We restrict the creation of multiple default tenants because if
// we allow for more than one to be created, it's not clear what we
Expand Down
18 changes: 16 additions & 2 deletions pkg/testutils/serverutils/test_server_shim.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,12 @@ import (
"github.com/cockroachdb/errors"
)

// DefaultTestTenantMessage is a message that is printed when a test is run
// with the default test tenant. This is useful for debugging test failures.
const DefaultTestTenantMessage = "Running test with the default test tenant. " +
"If you are only seeing a test case failure when this message appears, there may be a " +
"problem with your test case running within tenants."

// TenantModeFlagName is the exported name of the tenantMode flag, for use
// in other packages.
const TenantModeFlagName = "tenantMode"
Expand Down Expand Up @@ -330,6 +336,11 @@ func StartServer(
if err := s.Start(context.Background()); err != nil {
t.Fatalf("%+v", err)
}

if s.StartedDefaultTestTenant() {
t.Log(DefaultTestTenantMessage)
}

goDB := OpenDBConn(
t, s.ServingSQLAddr(), params.UseDatabase, params.Insecure, s.Stopper())

Expand Down Expand Up @@ -397,16 +408,19 @@ func OpenDBConn(
}

// StartServerRaw creates and starts a TestServer.
// Generally StartServer() should be used. However this function can be used
// Generally StartServer() should be used. However, this function can be used
// directly when opening a connection to the server is not desired.
func StartServerRaw(args base.TestServerArgs) (TestServerInterface, error) {
func StartServerRaw(t testing.TB, args base.TestServerArgs) (TestServerInterface, error) {
server, err := NewServer(args)
if err != nil {
return nil, err
}
if err := server.Start(context.Background()); err != nil {
return nil, err
}
if server.StartedDefaultTestTenant() {
t.Log(DefaultTestTenantMessage)
}
return server, nil
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/testutils/testcluster/testcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -396,6 +396,10 @@ func (tc *TestCluster) Start(t testing.TB) {
}
}

if tc.StartedDefaultTestTenant() {
t.Log(serverutils.DefaultTestTenantMessage)
}

if tc.clusterArgs.ParallelStart {
for i := 0; i < nodes; i++ {
if err := <-errCh; err != nil {
Expand Down

0 comments on commit 346bbc9

Please sign in to comment.