diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 85b1b8f98b8f..c126a3adfb6c 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -130,7 +130,7 @@ type systemAdminServer struct { *adminServer nodeLiveness *liveness.NodeLiveness - server *Server + server *topLevelServer } // noteworthyAdminMemoryUsageBytes is the minimum size tracked by the @@ -161,7 +161,7 @@ func newSystemAdminServer( distSender *kvcoord.DistSender, grpc *grpcServer, drainServer *drainServer, - s *Server, + s *topLevelServer, ) *systemAdminServer { adminServer := newAdminServer( sqlServer, diff --git a/pkg/server/auto_upgrade.go b/pkg/server/auto_upgrade.go index 4cfa987f4991..16d884b77150 100644 --- a/pkg/server/auto_upgrade.go +++ b/pkg/server/auto_upgrade.go @@ -24,7 +24,7 @@ import ( ) // startAttemptUpgrade attempts to upgrade cluster version. -func (s *Server) startAttemptUpgrade(ctx context.Context) error { +func (s *topLevelServer) startAttemptUpgrade(ctx context.Context) error { return s.stopper.RunAsyncTask(ctx, "auto-upgrade", func(ctx context.Context) { ctx, cancel := s.stopper.WithCancelOnQuiesce(ctx) defer cancel() @@ -120,7 +120,7 @@ const ( // upgradeStatus lets the main checking loop know if we should do upgrade, // keep checking upgrade status, or stop attempting upgrade. -func (s *Server) upgradeStatus( +func (s *topLevelServer) upgradeStatus( ctx context.Context, clusterVersion string, ) (st upgradeStatus, err error) { nodes, err := s.status.ListNodesInternal(ctx, nil) @@ -205,7 +205,7 @@ func (s *Server) upgradeStatus( // clusterVersion returns the current cluster version from the SQL subsystem // (which returns the version from the KV store as opposed to the possibly // lagging settings subsystem). -func (s *Server) clusterVersion(ctx context.Context) (string, error) { +func (s *topLevelServer) clusterVersion(ctx context.Context) (string, error) { row, err := s.sqlServer.internalExecutor.QueryRowEx( ctx, "show-version", nil, /* txn */ sessiondata.RootUserSessionDataOverride, diff --git a/pkg/server/clock_monotonicity.go b/pkg/server/clock_monotonicity.go index 42928eac60be..82ca03f1c1cb 100644 --- a/pkg/server/clock_monotonicity.go +++ b/pkg/server/clock_monotonicity.go @@ -45,7 +45,7 @@ var ( // startMonitoringForwardClockJumps starts a background task to monitor forward // clock jumps based on a cluster setting. -func (s *Server) startMonitoringForwardClockJumps(ctx context.Context) error { +func (s *topLevelServer) startMonitoringForwardClockJumps(ctx context.Context) error { forwardJumpCheckEnabled := make(chan bool, 1) s.stopper.AddCloser(stop.CloserFn(func() { close(forwardJumpCheckEnabled) })) @@ -69,7 +69,7 @@ func (s *Server) startMonitoringForwardClockJumps(ctx context.Context) error { // checkHLCUpperBoundExists determines whether there's an HLC // upper bound that will need to refreshed/persisted after // the server has initialized. -func (s *Server) checkHLCUpperBoundExistsAndEnsureMonotonicity( +func (s *topLevelServer) checkHLCUpperBoundExistsAndEnsureMonotonicity( ctx context.Context, initialStart bool, ) (hlcUpperBoundExists bool, err error) { if initialStart { @@ -238,7 +238,9 @@ func periodicallyPersistHLCUpperBound( // // tickCallback is called whenever persistHLCUpperBoundCh or a ticker tick is // processed -func (s *Server) startPersistingHLCUpperBound(ctx context.Context, hlcUpperBoundExists bool) error { +func (s *topLevelServer) startPersistingHLCUpperBound( + ctx context.Context, hlcUpperBoundExists bool, +) error { tickerFn := time.NewTicker persistHLCUpperBoundFn := func(t int64) error { /* function to persist upper bound of HLC to all stores */ return s.node.SetHLCUpperBound(context.Background(), t) diff --git a/pkg/server/decommission.go b/pkg/server/decommission.go index 857a4cce205c..48172e67bef6 100644 --- a/pkg/server/decommission.go +++ b/pkg/server/decommission.go @@ -141,7 +141,7 @@ func getPingCheckDecommissionFn( // or remove actions. If maxErrors >0, range checks will stop once maxError is // reached. // The error returned is a gRPC error. -func (s *Server) DecommissionPreCheck( +func (s *topLevelServer) DecommissionPreCheck( ctx context.Context, nodeIDs []roachpb.NodeID, strictReadiness bool, @@ -314,7 +314,7 @@ func evaluateRangeCheckResult( // Decommission idempotently sets the decommissioning flag for specified nodes. // The error return is a gRPC error. -func (s *Server) Decommission( +func (s *topLevelServer) Decommission( ctx context.Context, targetStatus livenesspb.MembershipStatus, nodeIDs []roachpb.NodeID, ) error { // If we're asked to decommission ourself we may lose access to cluster RPC, @@ -396,7 +396,7 @@ func (s *Server) Decommission( // DecommissioningNodeMap returns the set of node IDs that are decommissioning // from the perspective of the server. -func (s *Server) DecommissioningNodeMap() map[roachpb.NodeID]interface{} { +func (s *topLevelServer) DecommissioningNodeMap() map[roachpb.NodeID]interface{} { s.decomNodeMap.RLock() defer s.decomNodeMap.RUnlock() nodes := make(map[roachpb.NodeID]interface{}) diff --git a/pkg/server/import_ts.go b/pkg/server/import_ts.go index 40e43859ac13..03b8d971a99a 100644 --- a/pkg/server/import_ts.go +++ b/pkg/server/import_ts.go @@ -38,7 +38,7 @@ import ( // to be written to the DB. const maxBatchSize = 10000 -func maybeImportTS(ctx context.Context, s *Server) (returnErr error) { +func maybeImportTS(ctx context.Context, s *topLevelServer) (returnErr error) { // We don't want to do startup retries as this is not meant to be run in // production. ctx = startup.WithoutChecks(ctx) diff --git a/pkg/server/initial_sql.go b/pkg/server/initial_sql.go index 4e01a5ed0370..37551294d137 100644 --- a/pkg/server/initial_sql.go +++ b/pkg/server/initial_sql.go @@ -29,7 +29,7 @@ import ( // and `cockroach demo` with 2 nodes or fewer. // If adminUser is non-empty, an admin user with that name is // created upon initialization. Its password is then also returned. -func (s *Server) RunInitialSQL( +func (s *topLevelServer) RunInitialSQL( ctx context.Context, startSingleNode bool, adminUser, adminPassword string, ) error { if s.cfg.ObsServiceAddr == base.ObsServiceEmbedFlagValue { @@ -75,7 +75,9 @@ func (s *SQLServerWrapper) RunInitialSQL(context.Context, bool, string, string) } // createAdminUser creates an admin user with the given name. -func (s *Server) createAdminUser(ctx context.Context, adminUser, adminPassword string) error { +func (s *topLevelServer) createAdminUser( + ctx context.Context, adminUser, adminPassword string, +) error { ie := s.sqlServer.internalExecutor _, err := ie.Exec( ctx, "admin-user", nil, @@ -97,7 +99,7 @@ func (s *Server) createAdminUser(ctx context.Context, adminUser, adminPassword s // // The change is effected using the internal SQL interface of the // given server object. -func (s *Server) disableReplication(ctx context.Context) (retErr error) { +func (s *topLevelServer) disableReplication(ctx context.Context) (retErr error) { ie := s.sqlServer.internalExecutor it, err := ie.QueryIterator(ctx, "get-zones", nil, diff --git a/pkg/server/loss_of_quorum.go b/pkg/server/loss_of_quorum.go index 050cb266893c..fc1107be5cdd 100644 --- a/pkg/server/loss_of_quorum.go +++ b/pkg/server/loss_of_quorum.go @@ -93,7 +93,7 @@ func maybeRunLossOfQuorumRecoveryCleanup( ctx context.Context, ie isql.Executor, stores *kvserver.Stores, - server *Server, + server *topLevelServer, stopper *stop.Stopper, ) { publishCtx, publishCancel := stopper.WithCancelOnQuiesce(ctx) diff --git a/pkg/server/migration.go b/pkg/server/migration.go index ee93d4638fa6..ba021fe2b0c7 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -29,7 +29,7 @@ import ( // migrationServer is an implementation of the Migration service. The RPCs here // are used to power the upgrades infrastructure in pkg/upgrades. type migrationServer struct { - server *Server + server *topLevelServer // We use this mutex to serialize attempts to bump the cluster version. syncutil.Mutex diff --git a/pkg/server/server.go b/pkg/server/server.go index e2a40962811c..ae2150f59054 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -130,8 +130,8 @@ import ( "google.golang.org/grpc/codes" ) -// Server is the cockroach server node. -type Server struct { +// topLevelServer is the cockroach server node. +type topLevelServer struct { // The following fields are populated in NewServer. nodeIDContainer *base.NodeIDContainer @@ -225,7 +225,7 @@ type Server struct { // // The caller is responsible for listening on the server's ShutdownRequested() // channel and calling stopper.Stop(). -func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { +func NewServer(cfg Config, stopper *stop.Stopper) (*topLevelServer, error) { ctx := cfg.AmbientCtx.AnnotateCtx(context.Background()) if err := cfg.ValidateAddrs(ctx); err != nil { @@ -946,7 +946,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { db, node.stores, storePool, st, nodeLiveness, internalExecutor, systemConfigWatcher, ) - lateBoundServer := &Server{} + lateBoundServer := &topLevelServer{} // The following initialization is mirrored in NewTenantServer(). // Please keep them in sync. @@ -1261,7 +1261,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { }, ) - *lateBoundServer = Server{ + *lateBoundServer = topLevelServer{ nodeIDContainer: nodeIDContainer, cfg: cfg, st: st, @@ -1319,36 +1319,36 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { } // ClusterSettings returns the cluster settings. -func (s *Server) ClusterSettings() *cluster.Settings { +func (s *topLevelServer) ClusterSettings() *cluster.Settings { return s.st } // AnnotateCtx is a convenience wrapper; see AmbientContext. -func (s *Server) AnnotateCtx(ctx context.Context) context.Context { +func (s *topLevelServer) AnnotateCtx(ctx context.Context) context.Context { return s.cfg.AmbientCtx.AnnotateCtx(ctx) } // AnnotateCtxWithSpan is a convenience wrapper; see AmbientContext. -func (s *Server) AnnotateCtxWithSpan( +func (s *topLevelServer) AnnotateCtxWithSpan( ctx context.Context, opName string, ) (context.Context, *tracing.Span) { return s.cfg.AmbientCtx.AnnotateCtxWithSpan(ctx, opName) } // StorageClusterID returns the ID of the storage cluster this server is a part of. -func (s *Server) StorageClusterID() uuid.UUID { +func (s *topLevelServer) StorageClusterID() uuid.UUID { return s.rpcContext.StorageClusterID.Get() } // NodeID returns the ID of this node within its cluster. -func (s *Server) NodeID() roachpb.NodeID { +func (s *topLevelServer) NodeID() roachpb.NodeID { return s.node.Descriptor.NodeID } // InitialStart returns whether this is the first time the node has started (as // opposed to being restarted). Only intended to help print debugging info // during server startup. -func (s *Server) InitialStart() bool { +func (s *topLevelServer) InitialStart() bool { return s.node.initialStart } @@ -1401,7 +1401,7 @@ func (li listenerInfo) Iter() map[string]string { // // The passed context can be used to trace the server startup. The context // should represent the general startup operation. -func (s *Server) PreStart(ctx context.Context) error { +func (s *topLevelServer) PreStart(ctx context.Context) error { ctx = s.AnnotateCtx(ctx) done := startup.Begin(ctx) defer done() @@ -2173,7 +2173,7 @@ func (s *Server) PreStart(ctx context.Context) error { // AcceptClients starts listening for incoming SQL clients over the network. // This mirrors the implementation of (*SQLServerWrapper).AcceptClients. // TODO(knz): Find a way to implement this method only once for both. -func (s *Server) AcceptClients(ctx context.Context) error { +func (s *topLevelServer) AcceptClients(ctx context.Context) error { workersCtx := s.AnnotateCtx(context.Background()) if err := startServeSQL( @@ -2205,7 +2205,7 @@ func (s *Server) AcceptClients(ctx context.Context) error { // AcceptInternalClients starts listening for incoming SQL connections on the // internal loopback interface. -func (s *Server) AcceptInternalClients(ctx context.Context) error { +func (s *topLevelServer) AcceptInternalClients(ctx context.Context) error { connManager := netutil.MakeTCPServer(ctx, s.stopper) return s.stopper.RunAsyncTaskEx(ctx, @@ -2231,34 +2231,34 @@ func (s *Server) AcceptInternalClients(ctx context.Context) error { // ShutdownRequested returns a channel that is signaled when a subsystem wants // the server to be shut down. -func (s *Server) ShutdownRequested() <-chan ShutdownRequest { +func (s *topLevelServer) ShutdownRequested() <-chan ShutdownRequest { return s.stopTrigger.C() } // TempDir returns the filepath of the temporary directory used for temp storage. // It is empty for an in-memory temp storage. -func (s *Server) TempDir() string { +func (s *topLevelServer) TempDir() string { return s.cfg.TempStorageConfig.Path } // PGServer exports the pgwire server. Used by tests. -func (s *Server) PGServer() *pgwire.Server { +func (s *topLevelServer) PGServer() *pgwire.Server { return s.sqlServer.pgServer } // SpanConfigReporter returns the spanconfig.Reporter. Used by tests. -func (s *Server) SpanConfigReporter() spanconfig.Reporter { +func (s *topLevelServer) SpanConfigReporter() spanconfig.Reporter { return s.spanConfigReporter } // LogicalClusterID implements cli.serverStartupInterface. This // implementation exports the logical cluster ID of the system tenant. -func (s *Server) LogicalClusterID() uuid.UUID { +func (s *topLevelServer) LogicalClusterID() uuid.UUID { return s.sqlServer.LogicalClusterID() } // startDiagnostics starts periodic diagnostics reporting and update checking. -func (s *Server) startDiagnostics(ctx context.Context) { +func (s *topLevelServer) startDiagnostics(ctx context.Context) { s.updates.PeriodicallyCheckForUpdates(ctx, s.stopper) s.sqlServer.StartDiagnostics(ctx) } @@ -2268,12 +2268,12 @@ func init() { } // Insecure returns true iff the server has security disabled. -func (s *Server) Insecure() bool { +func (s *topLevelServer) Insecure() bool { return s.cfg.Insecure } // TenantCapabilitiesReader returns the Server's tenantcapabilities.Reader. -func (s *Server) TenantCapabilitiesReader() tenantcapabilities.Reader { +func (s *topLevelServer) TenantCapabilitiesReader() tenantcapabilities.Reader { return s.tenantCapabilitiesWatcher } @@ -2292,7 +2292,7 @@ func (s *Server) TenantCapabilitiesReader() tenantcapabilities.Reader { // TODO(knz): This method is currently exported for use by the // shutdown code in cli/start.go; however, this is a mis-design. The // start code should use the Drain() RPC like quit does. -func (s *Server) Drain( +func (s *topLevelServer) Drain( ctx context.Context, verbose bool, ) (remaining uint64, info redact.RedactableString, err error) { return s.drain.runDrain(ctx, verbose) diff --git a/pkg/server/server_controller.go b/pkg/server/server_controller.go index a52466b7d731..e40470c55690 100644 --- a/pkg/server/server_controller.go +++ b/pkg/server/server_controller.go @@ -196,7 +196,7 @@ func (s *tenantServerWrapper) gracefulDrain( // We do not implement the onDemandServer interface methods on *Server // directly so as to not add noise to its go documentation. type systemServerWrapper struct { - server *Server + server *topLevelServer } var _ onDemandServer = (*systemServerWrapper)(nil) diff --git a/pkg/server/server_controller_new_server.go b/pkg/server/server_controller_new_server.go index 7c712a6cddfd..55f5d86dbef8 100644 --- a/pkg/server/server_controller_new_server.go +++ b/pkg/server/server_controller_new_server.go @@ -61,10 +61,10 @@ type tenantServerCreator interface { ) (onDemandServer, error) } -var _ tenantServerCreator = &Server{} +var _ tenantServerCreator = &topLevelServer{} // newTenantServer implements the tenantServerCreator interface. -func (s *Server) newTenantServer( +func (s *topLevelServer) newTenantServer( ctx context.Context, tenantNameContainer *roachpb.TenantNameContainer, tenantStopper *stop.Stopper, @@ -103,7 +103,7 @@ func (errInvalidTenantMarker) Error() string { return "invalid tenant" } // is not shared. var ErrInvalidTenant error = errInvalidTenantMarker{} -func (s *Server) getTenantID( +func (s *topLevelServer) getTenantID( ctx context.Context, tenantName roachpb.TenantName, ) (roachpb.TenantID, error) { var rec *mtinfopb.TenantInfo @@ -149,7 +149,7 @@ func newTenantServerInternal( return newSharedProcessTenantServer(newCtx, stopper, baseCfg, sqlCfg, tenantNameContainer) } -func (s *Server) makeSharedProcessTenantConfig( +func (s *topLevelServer) makeSharedProcessTenantConfig( ctx context.Context, tenantID roachpb.TenantID, index int, stopper *stop.Stopper, ) (BaseConfig, SQLConfig, error) { // Create a configuration for the new tenant. diff --git a/pkg/server/server_controller_test.go b/pkg/server/server_controller_test.go index b391d53b6c0f..d0885f2a5604 100644 --- a/pkg/server/server_controller_test.go +++ b/pkg/server/server_controller_test.go @@ -37,7 +37,7 @@ func TestServerController(t *testing.T) { d, err := ts.serverController.getServer(ctx, "system") require.NoError(t, err) - if d.(*systemServerWrapper).server != ts.Server { + if d.(*systemServerWrapper).server != ts.topLevelServer { t.Fatal("expected wrapped system server") } diff --git a/pkg/server/server_obs_service.go b/pkg/server/server_obs_service.go index 5cd14a031c98..5bc3bc5ba258 100644 --- a/pkg/server/server_obs_service.go +++ b/pkg/server/server_obs_service.go @@ -31,7 +31,7 @@ import ( // startEmbeddedObsService creates the schema for the Observability Service (if // it doesn't exist already), starts the internal RPC service for event // ingestion and hooks up the event exporter to talk to the local service. -func (s *Server) startEmbeddedObsService( +func (s *topLevelServer) startEmbeddedObsService( ctx context.Context, knobs *obs.EventExporterTestingKnobs, ) error { // Create the Obs Service schema. diff --git a/pkg/server/server_special_test.go b/pkg/server/server_special_test.go index 6d3149f74f4e..10d535e459e8 100644 --- a/pkg/server/server_special_test.go +++ b/pkg/server/server_special_test.go @@ -127,7 +127,7 @@ func TestInternalSQL(t *testing.T) { 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).Server.loopbackPgL.Connect(ctx) + return s.(*testServer).topLevelServer.loopbackPgL.Connect(ctx) } conn, err := pgx.ConnectConfig(ctx, conf) require.NoError(t, err) diff --git a/pkg/server/settings_cache_test.go b/pkg/server/settings_cache_test.go index 0325adc54820..6a56f5bc770a 100644 --- a/pkg/server/settings_cache_test.go +++ b/pkg/server/settings_cache_test.go @@ -109,7 +109,7 @@ func TestCachedSettingsServerRestart(t *testing.T) { { getDialOpts := s.RPCContext().GRPCDialOptions - initConfig := newInitServerConfig(ctx, s.(*testServer).Server.cfg, getDialOpts) + initConfig := newInitServerConfig(ctx, s.(*testServer).topLevelServer.cfg, getDialOpts) inspectState, err := inspectEngines( context.Background(), s.Engines(), diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 1764519549bf..a3fb2e7f6243 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -341,7 +341,7 @@ type testServer struct { Cfg *Config params base.TestServerArgs // server is the embedded Cockroach server struct. - *Server + *topLevelServer // httpTestServer provides the HTTP APIs of the // serverutils.ApplicationLayerInterface. *httpTestServer @@ -376,7 +376,7 @@ func (ts *testServer) Stopper() *stop.Stopper { // GossipI is part of the serverutils.StorageLayerInterface. func (ts *testServer) GossipI() interface{} { - return ts.Server.gossip + return ts.topLevelServer.gossip } // RangeFeedFactory is part of serverutils.ApplicationLayerInterface. @@ -496,7 +496,7 @@ func (ts *testServer) SQLConnE(dbName string) (*gosql.DB, error) { // SQLConnForUserE is part of the serverutils.ApplicationLayerInterface. func (ts *testServer) SQLConnForUserE(userName string, dbName string) (*gosql.DB, error) { return openTestSQLConn(userName, dbName, ts.Stopper(), - ts.Server.loopbackPgL, + ts.topLevelServer.loopbackPgL, ts.cfg.SQLAdvertiseAddr, ts.cfg.Insecure, ) @@ -666,19 +666,19 @@ func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { // Use testServer.Stopper().Stop() to shutdown the server after the test // completes. func (ts *testServer) Start(ctx context.Context) error { - if err := ts.Server.PreStart(ctx); err != nil { + if err := ts.topLevelServer.PreStart(ctx); err != nil { return err } - if err := ts.Server.AcceptInternalClients(ctx); err != nil { + if err := ts.topLevelServer.AcceptInternalClients(ctx); err != nil { return err } // In tests we need some, but not all of RunInitialSQL functionality. - if err := ts.Server.RunInitialSQL( + if err := ts.topLevelServer.RunInitialSQL( ctx, false /* startSingleNode */, "" /* adminUser */, "", /* adminPassword */ ); err != nil { return err } - if err := ts.Server.AcceptClients(ctx); err != nil { + if err := ts.topLevelServer.AcceptClients(ctx); err != nil { return err } @@ -692,8 +692,8 @@ func (ts *testServer) Start(ctx context.Context) error { // If the server requests a shutdown, do that simply by stopping the // stopper. select { - case req := <-ts.Server.ShutdownRequested(): - shutdownCtx := ts.Server.AnnotateCtx(context.Background()) + case req := <-ts.topLevelServer.ShutdownRequested(): + shutdownCtx := ts.topLevelServer.AnnotateCtx(context.Background()) log.Infof(shutdownCtx, "server requesting spontaneous shutdown: %v", req.ShutdownCause()) ts.Stopper().Stop(shutdownCtx) case <-ts.Stopper().ShouldQuiesce(): @@ -704,8 +704,8 @@ func (ts *testServer) Start(ctx context.Context) error { // Stop is part of the serverutils.TestServerInterface. func (ts *testServer) Stop(ctx context.Context) { - ctx = ts.Server.AnnotateCtx(ctx) - ts.Server.stopper.Stop(ctx) + ctx = ts.topLevelServer.AnnotateCtx(ctx) + ts.topLevelServer.stopper.Stop(ctx) } // testTenant is an in-memory instantiation of the SQL-only process created for @@ -1016,7 +1016,7 @@ func (ts *testServer) StartSharedProcessTenant( } // Save the args for use if the server needs to be created. - ts.Server.serverController.testArgs[args.TenantName] = args + ts.topLevelServer.serverController.testArgs[args.TenantName] = args tenantRow, err := ts.InternalExecutor().(*sql.InternalExecutor).QueryRow( ctx, "testserver-check-tenant-active", nil, /* txn */ @@ -1095,7 +1095,7 @@ func (ts *testServer) StartSharedProcessTenant( } // Instantiate the tenant server. - s, err := ts.Server.serverController.startAndWaitForRunningServer(ctx, args.TenantName) + s, err := ts.topLevelServer.serverController.startAndWaitForRunningServer(ctx, args.TenantName) if err != nil { return nil, nil, err } @@ -1608,7 +1608,7 @@ func (ts *testServer) Readiness(ctx context.Context) error { // SetReadyFn is part of TestServerInterface. func (ts *testServer) SetReadyFn(fn func(bool)) { - ts.Server.cfg.ReadyFn = fn + ts.topLevelServer.cfg.ReadyFn = fn } // WriteSummaries implements the serverutils.StorageLayerInterface. @@ -1618,12 +1618,12 @@ func (ts *testServer) WriteSummaries() error { // UpdateChecker implements the serverutils.StorageLayerInterface. func (ts *testServer) UpdateChecker() interface{} { - return ts.Server.updates + return ts.topLevelServer.updates } // DiagnosticsReporter implements the serverutils.ApplicationLayerInterface. func (ts *testServer) DiagnosticsReporter() interface{} { - return ts.Server.sqlServer.diagnosticsReporter + return ts.topLevelServer.sqlServer.diagnosticsReporter } type v2AuthDecorator struct { @@ -1683,17 +1683,17 @@ func (ts *testServer) DistSenderI() interface{} { // MigrationServer is part of the serverutils.ApplicationLayerInterface. func (ts *testServer) MigrationServer() interface{} { - return ts.Server.migrationServer + return ts.topLevelServer.migrationServer } // SpanConfigKVAccessor is part of the serverutils.StorageLayerInterface. func (ts *testServer) SpanConfigKVAccessor() interface{} { - return ts.Server.node.spanConfigAccessor + return ts.topLevelServer.node.spanConfigAccessor } // SpanConfigReporter is part of the serverutils.StorageLayerInterface. func (ts *testServer) SpanConfigReporter() interface{} { - return ts.Server.node.spanConfigReporter + return ts.topLevelServer.node.spanConfigReporter } // SpanConfigReconciler is part of the serverutils.ApplicationLayerInterface. @@ -1981,7 +1981,7 @@ func (ts *testServer) DefaultZoneConfig() zonepb.ZoneConfig { // DefaultSystemZoneConfig is part of the serverutils.StorageLayerInterface. func (ts *testServer) DefaultSystemZoneConfig() zonepb.ZoneConfig { - return ts.Server.cfg.DefaultSystemZoneConfig + return ts.topLevelServer.cfg.DefaultSystemZoneConfig } // ScratchRange is part of the serverutils.StorageLayerInterface. @@ -2080,7 +2080,7 @@ func (ts *testServer) BinaryVersionOverride() roachpb.Version { // KvProber is part of the serverutils.StorageLayerInterface. func (ts *testServer) KvProber() *kvprober.Prober { - return ts.Server.kvProber + return ts.topLevelServer.kvProber } // QueryDatabaseID is part of the serverutils.ApplicationLayerInterface. @@ -2204,7 +2204,7 @@ func (testServerFactoryImpl) New(params base.TestServerArgs) (interface{}, error } var err error - ts.Server, err = NewServer(*ts.Cfg, params.Stopper) + ts.topLevelServer, err = NewServer(*ts.Cfg, params.Stopper) if err != nil { params.Stopper.Stop(ctx) return nil, err @@ -2219,15 +2219,15 @@ func (testServerFactoryImpl) New(params base.TestServerArgs) (interface{}, error } // Our context must be shared with our server. - ts.Cfg = &ts.Server.cfg + ts.Cfg = &ts.topLevelServer.cfg // The HTTP APIs on ApplicationLayerInterface are implemented by // httpTestServer. ts.httpTestServer = &httpTestServer{} - ts.httpTestServer.t.authentication = ts.Server.authentication - ts.httpTestServer.t.sqlServer = ts.Server.sqlServer - ts.httpTestServer.t.admin = ts.Server.admin.adminServer - ts.httpTestServer.t.status = ts.Server.status.statusServer + ts.httpTestServer.t.authentication = ts.topLevelServer.authentication + ts.httpTestServer.t.sqlServer = ts.topLevelServer.sqlServer + ts.httpTestServer.t.admin = ts.topLevelServer.admin.adminServer + ts.httpTestServer.t.status = ts.topLevelServer.status.statusServer return ts, nil } @@ -2289,9 +2289,9 @@ func (ts *testServer) NewClientRPCContext( ctx context.Context, user username.SQLUsername, ) *rpc.Context { return newClientRPCContext(ctx, user, - ts.Server.cfg.Config, - ts.Server.cfg.TestingKnobs.Server, - ts.Server.cfg.ClusterIDContainer, + ts.topLevelServer.cfg.Config, + ts.topLevelServer.cfg.TestingKnobs.Server, + ts.topLevelServer.cfg.ClusterIDContainer, ts) }