Skip to content

Commit

Permalink
serverutils: implement the testserver interfaces indirectly
Browse files Browse the repository at this point in the history
TLDR: unit tests using TestServer/TestCluster now produce warning and
notice messages upon certain suspicious API usage.

For example:
```
$ dev test //pkg/server/storage_api -f TestStatusGetFiles
=== RUN   TestStatusGetFiles
    ...
        pkg/server/storage_api/storage_api_test_test/pkg/server/storage_api/files_test.go:46: (TestStatusGetFiles)
                NOTICE: .GetStatusClient() called via implicit interface ApplicationLayerInterface;
        HINT: consider using .ApplicationLayer().GetStatusClient() instead.
    conditional_wrap.go:193: TIP: consider replacing the test server initialization from:

            ts, db, kvDB := serverutils.StartServer(t, ...)
            // or:
            tc := serverutils.StartCluster(t, ...)
            ts := tc.Server(0)

        To:

            srv, db, kvDB := serverutils.StartServer(t, ...)
            defer srv.Stop(...)
            ts := srv.ApplicationLayer()
            // or:
            tc := serverutils.StartCluster(t, ...)
            ts := tc.Server(0).ApplicationLayer()
```

The following warnings/notices are implemented.

**ApplicationLayerInterface**

The problem here is that the implicit implementation of
`ApplicationLayerInterface` inside `TestServerInterface` is likely
misused. If/when the test server automatically starts a secondary
tenant, the *implicit* `ApplicationLayerInterface` refers to the
system tenant, whereas only `.ApplicationLayer()` refers to the
secondary tenant. It's likely that the test code should use the
latter.

For this, a warning is printed when a method from the *implicit*
`ApplicationLayerInterface` is called.

The warning is not printed if the test has indicated (via
`DefaultTestTenant: base.TestIsForStorageLayerAndNeedsASystemTenant`)
that it is ever only interested in the storage layer.

**StorageLayerInterface**

We want to promote the use of the explicit interface accessor
`.StorageLayer()` to access the methods of `StorageLayerInterface`.

For this, a notice is printed when a method from the *implicit*
`StorageLayerInterface` is called.

The notice is not printed if the test has indicated (via
`DefaultTestTenant: base.TestIsForStorageLayerAndNeedsASystemTenant`)
that it is ever only interested in the storage layer.

**TenantControlInterface**

We want to promote the use of the explicit interface accessor
`.TenantController()` to access the methods of
`TenantControlInterface`.

For this, a notice is printed when a method from the *implicit*
`TenantControlInterface` is called.

Release note: None
  • Loading branch information
knz committed Sep 1, 2023
1 parent 01c013a commit 365db1f
Show file tree
Hide file tree
Showing 21 changed files with 1,005 additions and 71 deletions.
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ pkg/ccl/backupccl/data_driven_generated_test.go
pkg/ccl/backupccl/restore_memory_monitoring_generated_test.go
pkg/ccl/backupccl/restore_entry_cover_generated_test.go
pkg/ccl/backupccl/restore_mid_schema_change_generated_test.go
pkg/testutils/serverutils/*_generated.go

# Temporary directories during gomock generate
**/gomock_reflect_*
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -609,6 +609,7 @@ ALL_TESTS = [
"//pkg/testutils/lint:lint_test",
"//pkg/testutils/listenerutil:listenerutil_test",
"//pkg/testutils/release:release_test",
"//pkg/testutils/serverutils:serverutils_test",
"//pkg/testutils/sqlutils:sqlutils_test",
"//pkg/testutils/testcluster:testcluster_test",
"//pkg/testutils/zerofields:zerofields_test",
Expand Down Expand Up @@ -2210,8 +2211,11 @@ GO_TARGETS = [
"//pkg/testutils/physicalplanutils:physicalplanutils",
"//pkg/testutils/release:release",
"//pkg/testutils/release:release_test",
"//pkg/testutils/serverutils/fwgen:fwgen",
"//pkg/testutils/serverutils/fwgen:fwgen_lib",
"//pkg/testutils/serverutils/regionlatency:regionlatency",
"//pkg/testutils/serverutils:serverutils",
"//pkg/testutils/serverutils:serverutils_test",
"//pkg/testutils/skip:skip",
"//pkg/testutils/sqlutils:sqlutils",
"//pkg/testutils/sqlutils:sqlutils_test",
Expand Down
43 changes: 37 additions & 6 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,6 +226,13 @@ type DefaultTestTenantOptions struct {
// additional virtual clusters. (Default is to block.)
allowAdditionalTenants bool

// Whether implicit uses of the ApplicationLayerInterface or
// StorageLayerInterface result in warnings/notices.
//
// We use a "no" boolean, so that the default value results in "do
// warn".
noWarnImplicitInterfaces bool

// If test tenant is disabled, issue and label to link in log message.
issueNum int
label string
Expand Down Expand Up @@ -267,13 +274,21 @@ var (

// TestControlsTenantsExplicitly is used when the test wants to
// manage its own secondary tenants and tenant servers.
TestControlsTenantsExplicitly = DefaultTestTenantOptions{testBehavior: ttDisabled, allowAdditionalTenants: true}
TestControlsTenantsExplicitly = DefaultTestTenantOptions{
testBehavior: ttDisabled,
allowAdditionalTenants: true,
noWarnImplicitInterfaces: true,
}

// TestIsSpecificToStorageLayerAndNeedsASystemTenant is used when
// the test needs to be given access to a SQL conn to a tenant with
// sufficient capabilities to access all the storage layer.
// (Initially that'd be "the" system tenant.)
TestIsSpecificToStorageLayerAndNeedsASystemTenant = DefaultTestTenantOptions{testBehavior: ttDisabled, allowAdditionalTenants: true}
TestIsSpecificToStorageLayerAndNeedsASystemTenant = DefaultTestTenantOptions{
testBehavior: ttDisabled,
allowAdditionalTenants: true,
noWarnImplicitInterfaces: true,
}

// TestNeedsTightIntegrationBetweenAPIsAndTestingKnobs is used when
// a test wants to use a single set of testing knobs for both the
Expand All @@ -284,10 +299,6 @@ var (
// worth the cost of never running that test with the virtualization
// layer active.
TestNeedsTightIntegrationBetweenAPIsAndTestingKnobs = TestIsSpecificToStorageLayerAndNeedsASystemTenant

// InternalNonDefaultDecision is a sentinel value used inside a
// mechanism in serverutils. Should not be used by tests directly.
InternalNonDefaultDecision = DefaultTestTenantOptions{testBehavior: ttDisabled, allowAdditionalTenants: true}
)

func (do DefaultTestTenantOptions) AllowAdditionalTenants() bool {
Expand All @@ -302,6 +313,13 @@ func (do DefaultTestTenantOptions) TestTenantAlwaysDisabled() bool {
return do.testBehavior == ttDisabled
}

// WarnImplicitInterfaces indicates whether to warn when the test code
// uses ApplicationLayerInterface or StorageLayerInterface
// implicitely.
func (do DefaultTestTenantOptions) WarnImplicitInterfaces() bool {
return !do.noWarnImplicitInterfaces
}

func (do DefaultTestTenantOptions) IssueRef() (int, string) {
return do.issueNum, do.label
}
Expand Down Expand Up @@ -339,6 +357,19 @@ func TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(
}
}

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

var (
// DefaultTestStoreSpec is just a single in memory store of 512 MiB
// with no special attributes.
Expand Down
17 changes: 15 additions & 2 deletions pkg/cli/democluster/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -644,7 +644,7 @@ func (c *transientCluster) createAndAddNode(
if err != nil {
return nil, err
}
s := srv.(serverutils.TestServerInterface)
s := &wrap{srv.(serverutils.TestServerInterfaceRaw)}

// Ensure that this server gets stopped when the top level demo
// stopper instructs the cluster to stop.
Expand Down Expand Up @@ -1159,7 +1159,7 @@ func (c *transientCluster) startServerInternal(
if err != nil {
return 0, err
}
s := srv.(serverutils.TestServerInterface)
s := &wrap{srv.(serverutils.TestServerInterfaceRaw)}

// We want to only return after the server is ready.
readyCh := make(chan struct{})
Expand Down Expand Up @@ -2143,3 +2143,16 @@ func (c *transientCluster) TenantName() string {
}
return catconstants.SystemTenantName
}

type wrap struct {
serverutils.TestServerInterfaceRaw
}

var _ serverutils.TestServerInterface = (*wrap)(nil)

func (w *wrap) ApplicationLayer() serverutils.ApplicationLayerInterface {
return w.TestServerInterfaceRaw
}
func (w *wrap) SystemLayer() serverutils.ApplicationLayerInterface { return w.TestServerInterfaceRaw }
func (w *wrap) TenantController() serverutils.TenantControlInterface { return w.TestServerInterfaceRaw }
func (w *wrap) StorageLayer() serverutils.StorageLayerInterface { return w.TestServerInterfaceRaw }
4 changes: 4 additions & 0 deletions pkg/gen/misc.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,10 @@ MISC_SRCS = [
"//pkg/sql/scanner:token_names_test.go",
"//pkg/sql:txnstatetransitions_diagram.gv",
"//pkg/sql:txnstatetransitions_report.txt",
"//pkg/testutils/serverutils:app_forwarder_generated.go",
"//pkg/testutils/serverutils:storage_forwarder_generated.go",
"//pkg/testutils/serverutils:tenant_control_forwarder_generated.go",
"//pkg/testutils/serverutils:ts_control_forwarder_generated.go",
"//pkg/util/interval/generic:example_interval_btree.go",
"//pkg/util/interval/generic:example_interval_btree_test.go",
"//pkg/util/log/channel:channel_generated.go",
Expand Down
9 changes: 5 additions & 4 deletions pkg/server/server_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,22 +37,23 @@ func TestServerController(t *testing.T) {
})
defer s.Stopper().Stop(ctx)

ts := s.(*testServer)
sc := s.TenantController().ServerController().(*serverController)
ts := s.SystemLayer().(*testServer)

d, err := ts.serverController.getServer(ctx, "system")
d, err := sc.getServer(ctx, "system")
require.NoError(t, err)
if d.(*systemServerWrapper).server != ts.topLevelServer {
t.Fatal("expected wrapped system server")
}

d, err = ts.serverController.getServer(ctx, "somename")
d, err = sc.getServer(ctx, "somename")
require.Nil(t, d)
require.Error(t, err, `no tenant found with name "somename"`)

_, err = db.Exec("CREATE TENANT hello; ALTER TENANT hello START SERVICE SHARED")
require.NoError(t, err)

_, err = ts.serverController.getServer(ctx, "hello")
_, err = sc.getServer(ctx, "hello")
// TODO(knz): We're not really expecting an error here.
// The actual error seen will exist as long as in-memory
// servers use the standard KV connector.
Expand Down
15 changes: 10 additions & 5 deletions pkg/server/server_special_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/netutil/addr"
pgx "github.com/jackc/pgx/v4"
"github.com/stretchr/testify/require"
Expand All @@ -39,10 +40,12 @@ func TestPanicRecovery(t *testing.T) {

s := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.Background())
ts := s.(*testServer)
ts := s.ApplicationLayer()

hs := ts.HTTPServer().(*httpServer)

// Enable a test-only endpoint that induces a panic.
ts.http.mux.Handle("/panic", http.HandlerFunc(func(http.ResponseWriter, *http.Request) {
hs.mux.Handle("/panic", http.HandlerFunc(func(http.ResponseWriter, *http.Request) {
panic("induced panic for testing")
}))

Expand All @@ -53,7 +56,7 @@ func TestPanicRecovery(t *testing.T) {
// Create a ResponseRecorder to record the response.
rr := httptest.NewRecorder()
require.NotPanics(t, func() {
ts.http.baseHandler(rr, req)
hs.baseHandler(rr, req)
})

// Check that the status code is correct.
Expand Down Expand Up @@ -109,7 +112,8 @@ func TestSocketAutoNumbering(t *testing.T) {
_, expectedPort, err := addr.SplitHostPort(s.SQLAddr(), "")
require.NoError(t, err)

if socketPath := s.(*testServer).Cfg.SocketFile; !strings.HasSuffix(socketPath, "."+expectedPort) {
srv := s.SystemLayer().(*testServer)
if socketPath := srv.Cfg.SocketFile; !strings.HasSuffix(socketPath, "."+expectedPort) {
t.Errorf("expected unix socket ending with port %q, got %q", expectedPort, socketPath)
}
}
Expand All @@ -121,13 +125,14 @@ func TestInternalSQL(t *testing.T) {
ctx := context.Background()
s := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
ts := s.ApplicationLayer()

conf, err := pgx.ParseConfig("")
require.NoError(t, err)
conf.User = "root"
// Configure pgx to connect on the loopback listener.
conf.DialFunc = func(ctx context.Context, network, addr string) (net.Conn, error) {
return s.(*testServer).topLevelServer.loopbackPgL.Connect(ctx)
return ts.SQLLoopbackListener().(*netutil.LoopbackListener).Connect(ctx)
}
conn, err := pgx.ConnectConfig(ctx, conf)
require.NoError(t, err)
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/settings_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,8 @@ func TestCachedSettingsServerRestart(t *testing.T) {
{
getDialOpts := s.RPCContext().GRPCDialOptions

initConfig := newInitServerConfig(ctx, s.(*testServer).topLevelServer.cfg, getDialOpts)
cfg := s.SystemLayer().(*testServer).topLevelServer.cfg
initConfig := newInitServerConfig(ctx, cfg, getDialOpts)
inspectState, err := inspectEngines(
context.Background(),
s.Engines(),
Expand Down
5 changes: 3 additions & 2 deletions pkg/server/storage_api/certs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,9 @@ import (
func TestCertificatesResponse(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ts := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer ts.Stopper().Stop(context.Background())
srv := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer srv.Stopper().Stop(context.Background())
ts := srv.ApplicationLayer()

var response serverpb.CertificatesResponse
if err := srvtestutils.GetStatusJSONProto(ts, "certificates/local", &response); err != nil {
Expand Down
54 changes: 29 additions & 25 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -355,7 +355,7 @@ type testServer struct {
disableStartTenantError error
}

var _ serverutils.TestServerInterface = &testServer{}
var _ serverutils.TestServerInterfaceRaw = &testServer{}

// Node returns the Node as an interface{}.
func (ts *testServer) Node() interface{} {
Expand Down Expand Up @@ -566,7 +566,7 @@ func (ts *testServer) TenantStatusServer() interface{} {
return ts.status
}

// TestTenant provides access to the test tenant service.
// TestTenant is part of serverutils.TenantControlInterface.
func (ts *testServer) TestTenant() serverutils.ApplicationLayerInterface {
return ts.testTenants[0]
}
Expand Down Expand Up @@ -763,6 +763,7 @@ type testTenant struct {
SQLCfg *SQLConfig
*httpTestServer
drain *drainServer
http *httpServer

pgL *netutil.LoopbackListener

Expand Down Expand Up @@ -1259,6 +1260,16 @@ func (t *testTenant) HTTPAuthServer() interface{} {
return t.t.authentication
}

// HTTPServer is part of the serverutils.ApplicationLayerInterface.
func (t *testTenant) HTTPServer() interface{} {
return t.http
}

// SQLLoopbackListener is part of the serverutils.ApplicationLayerInterface.
func (t *testTenant) SQLLoopbackListener() interface{} {
return t.pgL
}

func (ts *testServer) waitForTenantReadinessImpl(
ctx context.Context, tenantID roachpb.TenantID,
) error {
Expand Down Expand Up @@ -1580,6 +1591,7 @@ func (ts *testServer) StartTenant(
SQLCfg: &sqlCfg,
pgPreServer: sw.pgPreServer,
httpTestServer: hts,
http: sw.http,
drain: sw.drainServer,
pgL: sw.loopbackPgL,
}, err
Expand Down Expand Up @@ -1981,29 +1993,6 @@ func (ts *testServer) StartedDefaultTestTenant() bool {
return len(ts.testTenants) > 0
}

// ApplicationLayer is part of the serverutils.TestServerInterface.
func (ts *testServer) ApplicationLayer() serverutils.ApplicationLayerInterface {
if ts.StartedDefaultTestTenant() {
return ts.testTenants[0]
}
return ts
}

// StorageLayer is part of the serverutils.TestServerInterface.
func (ts *testServer) StorageLayer() serverutils.StorageLayerInterface {
return ts
}

// TenantController is part of the serverutils.TestServerInterface.
func (ts *testServer) TenantController() serverutils.TenantControlInterface {
return ts
}

// SystemLayer is part of the serverutils.TestServerInterface.
func (ts *testServer) SystemLayer() serverutils.ApplicationLayerInterface {
return ts
}

// TracerI is part of the serverutils.ApplicationLayerInterface.
func (ts *testServer) TracerI() interface{} {
return ts.Tracer()
Expand Down Expand Up @@ -2194,6 +2183,21 @@ func (ts *testServer) HTTPAuthServer() interface{} {
return ts.t.authentication
}

// HTTPServer is part of the serverutils.ApplicationLayerInterface.
func (ts *testServer) HTTPServer() interface{} {
return ts.topLevelServer.http
}

// SQLLoopbackListener is part of the serverutils.ApplicationLayerInterface.
func (ts *testServer) SQLLoopbackListener() interface{} {
return ts.topLevelServer.loopbackPgL
}

// ServerController is part of the serverutils.TenantControlInterface.
func (ts *testServer) ServerController() interface{} {
return ts.topLevelServer.serverController
}

type testServerFactoryImpl struct{}

// TestServerFactory can be passed to serverutils.InitTestServerFactory
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 @@ -1409,6 +1409,7 @@ func TestLint(t *testing.T) {
":!sql/opt/optgen",
":!sql/colexec/execgen",
":!kv/kvpb/gen/main.go",
":!pkg/testutils/serverutils/fwgen/gen.go",
)
if err != nil {
t.Fatal(err)
Expand Down
Loading

0 comments on commit 365db1f

Please sign in to comment.