Skip to content

Commit

Permalink
cli,server: make PreStart responsible for starting diagnostics
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
knz committed Nov 14, 2022
1 parent 4c864d7 commit 281aff2
Show file tree
Hide file tree
Showing 12 changed files with 52 additions and 48 deletions.
12 changes: 12 additions & 0 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,12 @@ type TestServerArgs struct {
// or if some of the functionality being tested is not accessible from
// within tenants.
DisableDefaultTestTenant bool

// StartDiagnosticsReporting checks cluster.TelemetryOptOut(), and
// if not disabled starts the asynchronous goroutine that checks for
// CockroachDB upgrades and periodically reports diagnostics to
// Cockroach Labs. Should remain disabled during unit testing.
StartDiagnosticsReporting bool
}

// TestClusterArgs contains the parameters one can set when creating a test
Expand Down Expand Up @@ -309,4 +315,10 @@ type TestTenantArgs struct {
// heapprofiler. If empty, no heap profiles will be collected during the test.
// If set, this directory should be cleaned up after the test completes.
HeapProfileDirName string

// StartDiagnosticsReporting checks cluster.TelemetryOptOut(), and
// if not disabled starts the asynchronous goroutine that checks for
// CockroachDB upgrades and periodically reports diagnostics to
// Cockroach Labs. Should remain disabled during unit testing.
StartDiagnosticsReporting bool
}
2 changes: 1 addition & 1 deletion pkg/cli/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@ func setServerContextDefaults() {
serverCfg.HeapProfileDirName = ""
serverCfg.CPUProfileDirName = ""
serverCfg.InflightTraceDirName = ""
serverCfg.StartDiagnosticsReporting = false // overridden in flags.go.

serverCfg.AutoInitializeCluster = false
serverCfg.ReadyFn = nil
Expand Down Expand Up @@ -617,7 +618,6 @@ func setDemoContextDefaults() {
demoCtx.RunWorkload = false
demoCtx.Localities = nil
demoCtx.GeoPartitionedReplicas = false
demoCtx.DisableTelemetry = false
demoCtx.DefaultKeySize = defaultKeySize
demoCtx.DefaultCALifetime = defaultCALifetime
demoCtx.DefaultCertLifetime = defaultCertLifetime
Expand Down
4 changes: 1 addition & 3 deletions pkg/cli/demo.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,8 +144,6 @@ func checkDemoConfiguration(
return nil, errors.Newf("--%s cannot be used with --%s", cliflags.Global.Name, cliflags.DemoNodeLocality.Name)
}

demoCtx.DisableTelemetry = cluster.TelemetryOptOut()

// Whether or not we enable enterprise feature is a combination of:
//
// - whether the user wants them (they can disable enterprise
Expand Down Expand Up @@ -293,7 +291,7 @@ func runDemoInternal(

// Only print details about the telemetry configuration if the
// user has control over it.
if demoCtx.DisableTelemetry {
if cluster.TelemetryOptOut() {
cliCtx.PrintlnUnlessEmbedded("#\n# Telemetry disabled by configuration.")
} else {
cliCtx.PrintlnUnlessEmbedded("#\n" +
Expand Down
3 changes: 0 additions & 3 deletions pkg/cli/democluster/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,6 @@ type Context struct {
// CacheSize is the size of the storage cache for each KV server.
CacheSize int64

// DisableTelemetry requests that telemetry be disabled.
DisableTelemetry bool

// NoExampleDatabase prevents the auto-creation of a demo database
// from a workload.
NoExampleDatabase bool
Expand Down
10 changes: 2 additions & 8 deletions pkg/cli/democluster/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -487,14 +487,6 @@ func (c *transientCluster) Start(ctx context.Context) (err error) {
}
}

// Start up the update check loop.
// We don't do this in (*server.Server).Start() because we don't want this
// overhead and possible interference in tests.
if !c.demoCtx.DisableTelemetry {
c.infoLog(ctx, "starting telemetry")
c.firstServer.StartDiagnostics(ctx)
}

return nil
}(phaseCtx); err != nil {
return err
Expand Down Expand Up @@ -530,6 +522,8 @@ func (c *transientCluster) createAndAddNode(
if idx == 0 {
// The first node also auto-inits the cluster.
args.NoAutoInitializeCluster = false
// The first node also runs diagnostics (unless disabled by cluster.TelemetryOptOut).
args.StartDiagnosticsReporting = true
}

serverKnobs := args.Knobs.Server.(*server.TestingKnobs)
Expand Down
3 changes: 3 additions & 0 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -1143,6 +1143,9 @@ func extraServerFlagInit(cmd *cobra.Command) error {
}
serverCfg.LocalityAddresses = localityAdvertiseHosts

// Ensure that diagnostic reporting is enabled for server startup commands.
serverCfg.StartDiagnosticsReporting = true

return nil
}

Expand Down
12 changes: 0 additions & 12 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -330,11 +330,6 @@ type serverStartupInterface interface {
// before the first client is accepted.
PreStart(ctx context.Context) error

// StartDiagnostics starts periodic diagnostics reporting and update checking.
// NOTE: This is not called in PreStart so that it's disabled by default for
// testing.
StartDiagnostics(ctx context.Context)

// AcceptClients starts listening for incoming SQL clients over the network.
AcceptClients(ctx context.Context) error

Expand Down Expand Up @@ -705,13 +700,6 @@ func createAndStartServerAsync(
// all these startup steps to fail. So we do not need to look at
// the "shutdown status" in serverStatusMu any more.

// Start up the diagnostics reporting and update check loops.
// We don't do this in (*server.Server).Start() because we don't
// want this overhead and possible interference in tests.
if !cluster.TelemetryOptOut() {
s.StartDiagnostics(ctx)
}

// Run one-off cluster initialization.
if err := s.RunInitialSQL(ctx, startSingleNode, "" /* adminUser */, "" /* adminPassword */); err != nil {
return err
Expand Down
6 changes: 6 additions & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,12 @@ type BaseConfig struct {

// Stores is specified to enable durable key-value storage.
Stores base.StoreSpecList

// StartDiagnosticsReporting starts the asynchronous goroutine that
// checks for CockroachDB upgrades and periodically reports
// diagnostics to Cockroach Labs.
// Should remain disabled during unit testing.
StartDiagnosticsReporting bool
}

// MakeBaseConfig returns a BaseConfig with default values.
Expand Down
11 changes: 7 additions & 4 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1725,6 +1725,11 @@ func (s *Server) PreStart(ctx context.Context) error {
return err
}

// If enabled, start reporting diagnostics.
if s.sqlServer.cfg.StartDiagnosticsReporting && !cluster.TelemetryOptOut() {
s.startDiagnostics(workersCtx)
}

// Enable the Obs Server.
s.eventsServer.SetResourceInfo(state.clusterID, int32(state.nodeID), build.BinaryVersion())

Expand Down Expand Up @@ -1832,10 +1837,8 @@ func (s *Server) LogicalClusterID() uuid.UUID {
return s.sqlServer.LogicalClusterID()
}

// StartDiagnostics starts periodic diagnostics reporting and update checking.
// NOTE: This is not called in PreStart so that it's disabled by default for
// testing.
func (s *Server) StartDiagnostics(ctx context.Context) {
// startDiagnostics starts periodic diagnostics reporting and update checking.
func (s *Server) startDiagnostics(ctx context.Context) {
s.updates.PeriodicallyCheckForUpdates(ctx, s.stopper)
s.sqlServer.StartDiagnostics(ctx)
}
Expand Down
5 changes: 0 additions & 5 deletions pkg/server/server_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -266,11 +266,6 @@ func (s *Server) startInMemoryTenantServerInternal(
return stopper, tenantServer, err
}

// Start up the diagnostics reporting loop.
if !cluster.TelemetryOptOut() {
tenantServer.StartDiagnostics(startCtx)
}

// Show the tenant details in logs.
// TODO(knz): Remove this once we can use a single listener.
if err := reportTenantInfo(startCtx, baseCfg, sqlCfg); err != nil {
Expand Down
9 changes: 7 additions & 2 deletions pkg/server/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -592,6 +592,11 @@ func (s *SQLServerWrapper) PreStart(ctx context.Context) error {
return err
}

// If enabled, start reporting diagnostics.
if s.sqlServer.cfg.StartDiagnosticsReporting && !cluster.TelemetryOptOut() {
s.startDiagnostics(workersCtx)
}

// Enable the Obs Server.
// There is more logic here than in (*Server).PreStart() because
// we care about the SQL instance ID too.
Expand Down Expand Up @@ -692,9 +697,9 @@ func (s *SQLServerWrapper) LogicalClusterID() uuid.UUID {
return s.sqlServer.LogicalClusterID()
}

// StartDiagnostics begins the diagnostic loop of this tenant server.
// startDiagnostics begins the diagnostic loop of this tenant server.
// Used in cli/mt_start_sql.go.
func (s *SQLServerWrapper) StartDiagnostics(ctx context.Context) {
func (s *SQLServerWrapper) startDiagnostics(ctx context.Context) {
s.sqlServer.StartDiagnostics(ctx)
}

Expand Down
23 changes: 13 additions & 10 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,7 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config {
cfg.SocketFile = params.SocketFile
cfg.RetryOptions = params.RetryOptions
cfg.Locality = params.Locality
cfg.StartDiagnosticsReporting = params.StartDiagnosticsReporting
if params.TraceDir != "" {
if err := initTraceDir(params.TraceDir); err == nil {
cfg.InflightTraceDirName = params.TraceDir
Expand Down Expand Up @@ -533,16 +534,17 @@ func (ts *TestServer) maybeStartDefaultTestTenant(ctx context.Context) error {
params := base.TestTenantArgs{
// Currently, all the servers leverage the same tenant ID. We may
// want to change this down the road, for more elaborate testing.
TenantID: serverutils.TestTenantID(),
MemoryPoolSize: ts.params.SQLMemoryPoolSize,
TempStorageConfig: &tempStorageConfig,
Locality: ts.params.Locality,
ExternalIODir: ts.params.ExternalIODir,
ExternalIODirConfig: ts.params.ExternalIODirConfig,
ForceInsecure: ts.Insecure(),
UseDatabase: ts.params.UseDatabase,
SSLCertsDir: ts.params.SSLCertsDir,
TestingKnobs: ts.params.Knobs,
TenantID: serverutils.TestTenantID(),
MemoryPoolSize: ts.params.SQLMemoryPoolSize,
TempStorageConfig: &tempStorageConfig,
Locality: ts.params.Locality,
ExternalIODir: ts.params.ExternalIODir,
ExternalIODirConfig: ts.params.ExternalIODirConfig,
ForceInsecure: ts.Insecure(),
UseDatabase: ts.params.UseDatabase,
SSLCertsDir: ts.params.SSLCertsDir,
TestingKnobs: ts.params.Knobs,
StartDiagnosticsReporting: ts.params.StartDiagnosticsReporting,
}

// Since we're creating a tenant, it doesn't make sense to pass through the
Expand Down Expand Up @@ -848,6 +850,7 @@ func (ts *TestServer) StartTenant(
baseCfg.HeapProfileDirName = params.HeapProfileDirName
baseCfg.GoroutineDumpDirName = params.GoroutineDumpDirName
baseCfg.ClusterName = ts.Cfg.ClusterName
baseCfg.StartDiagnosticsReporting = params.StartDiagnosticsReporting

// For now, we don't support split RPC/SQL ports for secondary tenants
// in test servers.
Expand Down

0 comments on commit 281aff2

Please sign in to comment.