From c785f8c7c6548699e9deaa02b172ca4ba118534a Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Thu, 20 Oct 2022 14:52:15 +0200 Subject: [PATCH] cli/start: share code between `cockroach start` and `cockroach mt start-sql` Arguably, this should really have been done from the start. This incidentally introduces the ability to terminate the process if the SQL initialization hangs somehow (e.g. because the remote KV layer is not ready). Release note: None --- pkg/cli/mt_start_sql.go | 230 ++++------------------------------------ pkg/cli/start.go | 55 ++++++---- pkg/server/tenant.go | 6 ++ 3 files changed, 61 insertions(+), 230 deletions(-) diff --git a/pkg/cli/mt_start_sql.go b/pkg/cli/mt_start_sql.go index 9f220d5c2edc..8760a1dec034 100644 --- a/pkg/cli/mt_start_sql.go +++ b/pkg/cli/mt_start_sql.go @@ -12,23 +12,11 @@ package cli import ( "context" - "os" - "os/signal" - "github.com/cockroachdb/cockroach/pkg/cli/clierror" "github.com/cockroachdb/cockroach/pkg/cli/clierrorplus" - "github.com/cockroachdb/cockroach/pkg/cli/exit" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/cgroups" - "github.com/cockroachdb/cockroach/pkg/util/grpcutil" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/log/severity" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" - "github.com/cockroachdb/pebble/vfs" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/redact" "github.com/spf13/cobra" ) @@ -68,205 +56,31 @@ well unless it can be verified using a trusted root certificate store. That is, } func runStartSQL(cmd *cobra.Command, args []string) error { - tBegin := timeutil.Now() - - // First things first: if the user wants background processing, - // relinquish the terminal ASAP by forking and exiting. - // - // If executing in the background, the function returns ok == true in - // the parent process (regardless of err) and the parent exits at - // this point. - if ok, err := maybeRerunBackground(); ok { - return err - } - - // Change the permission mask for all created files. - // - // We're considering everything produced by a cockroach node - // to potentially contain sensitive information, so it should - // not be world-readable. - disableOtherPermissionBits() - - // Set up the signal handlers. This also ensures that any of these - // signals received beyond this point do not interrupt the startup - // sequence until the point signals are checked below. - // We want to set up signal handling before starting logging, because - // logging uses buffering, and we want to be able to sync - // the buffers in the signal handler below. If we started capturing - // signals later, some startup logging might be lost. - signalCh := make(chan os.Signal, 1) - signal.Notify(signalCh, drainSignals...) - - // Check for stores with full disks and exit with an informative exit - // code. This needs to happen early during start, before we perform any - // writes to the filesystem including log rotation. We need to guarantee - // that the process continues to exit with the Disk Full exit code. A - // flapping exit code can affect alerting, including the alerting - // performed within CockroachCloud. - if err := exitIfDiskFull(vfs.Default, serverCfg.Stores.Specs); err != nil { - return err - } - - // If any store has something to say against a server start-up - // (e.g. previously detected corruption), listen to them now. - if err := serverCfg.Stores.PriorCriticalAlertError(); err != nil { - return clierror.NewError(err, exit.FatalError()) - } - - // Set up a cancellable context for the entire start command. - // The context will be canceled at the end. - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - // The context annotation ensures that server identifiers show up - // in the logging metadata as soon as they are known. - ambientCtx := serverCfg.AmbientCtx - - // Annotate the context, and set up a tracing span for the start process. - // - // The context annotation ensures that server identifiers show up - // in the logging metadata as soon as they are known. - // - // The tracing span is because we want any logging happening beyond - // this point to be accounted to this start context, including - // logging related to the initialization of the logging - // infrastructure below. This span concludes when the startup - // goroutine started below has completed. TODO(andrei): we don't - // close the span on the early returns below. - var startupSpan *tracing.Span - ctx, startupSpan = ambientCtx.AnnotateCtxWithSpan(ctx, "server start") - - // Set up the logging and profiling output. - // - // We want to do this as early as possible, because most of the code - // in CockroachDB may use logging, and until logging has been - // initialized log files will be created in $TMPDIR instead of their - // expected location. - // - // This initialization uses the various configuration parameters - // initialized by flag handling (before runStart was called). Any - // additional server configuration tweaks for the startup process - // must be necessarily non-logging-related, as logging parameters - // cannot be picked up beyond this point. - stopper, err := setupAndInitializeLoggingAndProfiling(ctx, cmd, false /* isServerCmd */) - if err != nil { - return err - } - defer stopper.Stop(ctx) // TODO(knz): Move this. - stopper.SetTracer(serverCfg.BaseConfig.AmbientCtx.Tracer) - - // We don't care about GRPCs fairly verbose logs in most client commands, - // but when actually starting a server, we enable them. - grpcutil.LowerSeverity(severity.WARNING) - - // Tweak GOMAXPROCS if we're in a cgroup / container that has cpu limits set. - // The GO default for GOMAXPROCS is NumCPU(), however this is less - // than ideal if the cgroup is limited to a number lower than that. - // - // TODO(bilal): various global settings have already been initialized based on - // GOMAXPROCS(0) by now. - cgroups.AdjustMaxProcs(ctx) - - // Now perform additional configuration tweaks specific to the start - // command. - - st := serverCfg.BaseConfig.Settings - - // Derive temporary/auxiliary directory specifications. - if st.ExternalIODir, err = initExternalIODir(ctx, serverCfg.Stores.Specs[0]); err != nil { - return err - } - - // This value is injected in order to have something populated during startup. - // In the initial 20.2 release of multi-tenant clusters, no version state was - // ever populated in the version cluster setting. A value is populated during - // the activation of 21.1. See the documentation attached to the TenantCluster - // in upgrade/upgradecluster for more details on the tenant upgrade flow. - // Note that a the value of 21.1 is populated when a tenant cluster is created - // during 21.1 in crdb_internal.create_tenant. - // - // Note that the tenant will read the value in the system.settings table - // before accepting SQL connections. - if err := clusterversion.Initialize( - ctx, st.Version.BinaryMinSupportedVersion(), &st.SV, - ); err != nil { - return err - } - - if serverCfg.SQLConfig.TempStorageConfig, err = initTempStorageConfig( - ctx, st, stopper, serverCfg.Stores, - ); err != nil { - return err - } - - // Configure the default storage engine. - // NB: we only support one engine type for now. - if serverCfg.StorageEngine == enginepb.EngineTypeDefault { - serverCfg.StorageEngine = enginepb.EngineTypePebble - } - - // Initialize the node's configuration from startup parameters. - // This also reads the part of the configuration that comes from - // environment variables. - if err := serverCfg.InitSQLServer(ctx); err != nil { - return err - } - - // The configuration is now ready to report to the user and the log - // file. We had to wait after InitNode() so that all configuration - // environment variables, which are reported too, have been read and - // registered. - reportConfiguration(ctx) - - initGEOS(ctx) - - // ReadyFn will be called when the server has started listening on - // its network sockets, but perhaps before it has done bootstrapping - serverCfg.ReadyFn = func(_ bool) { reportReadinessExternally(ctx, cmd, false /* waitForInit */) } - const serverType redact.SafeString = "SQL server" - // Beyond this point, the configuration is set and the server is - // ready to start. - log.Ops.Infof(ctx, "starting cockroach %s", serverType) - - tenantServer, err := server.NewTenantServer( - ctx, - stopper, - serverCfg.BaseConfig, - serverCfg.SQLConfig, - ) - if err != nil { - return err + initConfig := func(ctx context.Context) error { + if err := serverCfg.InitSQLServer(ctx); err != nil { + return err + } + // This value is injected in order to have something populated during startup. + // In the initial 20.2 release of multi-tenant clusters, no version state was + // ever populated in the version cluster setting. A value is populated during + // the activation of 21.1. See the documentation attached to the TenantCluster + // in upgrade/upgradecluster for more details on the tenant upgrade flow. + // Note that a the value of 21.1 is populated when a tenant cluster is created + // during 21.1 in crdb_internal.create_tenant. + // + // Note that the tenant will read the value in the system.settings table + // before accepting SQL connections. + st := serverCfg.BaseConfig.Settings + return clusterversion.Initialize( + ctx, st.Version.BinaryMinSupportedVersion(), &st.SV, + ) } - if err := tenantServer.Start(ctx); err != nil { - return err - } - - // When the start up completes, so can the start up span - // defined above. - defer startupSpan.Finish() - // Start up the diagnostics reporting loop. - // We don't do this in (*server.SQLServer).preStart() because we don't - // want this overhead and possible interference in tests. - if !cluster.TelemetryOptOut() { - tenantServer.StartDiagnostics(ctx) + newServerFn := func(ctx context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverStartupInterface, error) { + return server.NewTenantServer(ctx, stopper, serverCfg.BaseConfig, serverCfg.SQLConfig) } - tenantClusterID := tenantServer.LogicalClusterID() - - // Report the server identifiers and other server details - // in the same format as 'cockroach start'. - if err := reportServerInfo(ctx, tBegin, &serverCfg, st, serverType, false /* initialStart */, tenantClusterID); err != nil { - return err - } - - // TODO(tbg): make the other goodies in `./cockroach start` reusable, such as - // logging to files, periodic memory output, heap and goroutine dumps. - // Then use them here. - - serverStatus := &serverStatus{} - serverStatus.setStarted(tenantServer, stopper) - return waitForShutdown(stopper, tenantServer.ShutdownRequested(), signalCh, serverStatus) + return runStartInternal(cmd, serverType, initConfig, newServerFn, nil /* maybeRunInitialSQL */) } diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 40223978e911..71901e7da774 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -383,7 +383,31 @@ func runStartJoin(cmd *cobra.Command, args []string) error { // // If the argument startSingleNode is set the replication factor // will be set to 1 all zone configs (see initial_sql.go). -func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnErr error) { +func runStart(cmd *cobra.Command, args []string, startSingleNode bool) error { + const serverType redact.SafeString = "node" + + newServerFn := func(_ context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverStartupInterface, error) { + return server.NewServer(serverCfg, stopper) + } + + maybeRunInitialSQL := func(ctx context.Context, s serverStartupInterface) error { + // Run SQL for new clusters. + // + // TODO(knz): If/when we want auto-creation of an initial admin user, + // this can be achieved here. + return runInitialSQL(ctx, s.(*server.Server), startSingleNode, "" /* adminUser */, "" /* adminPassword */) + } + + return runStartInternal(cmd, serverType, serverCfg.InitNode, newServerFn, maybeRunInitialSQL) +} + +func runStartInternal( + cmd *cobra.Command, + serverType redact.SafeString, + initConfigFn func(context.Context) error, + newServerFn newServerFn, + maybeRunInitialSQL func(context.Context, serverStartupInterface) error, +) error { tBegin := timeutil.Now() // First things first: if the user wants background processing, @@ -483,7 +507,7 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr cgroups.AdjustMaxProcs(ctx) // Check the --join flag. - if !cliflagcfg.FlagSetForCmd(cmd).Lookup(cliflags.Join.Name).Changed { + if fl := cliflagcfg.FlagSetForCmd(cmd).Lookup(cliflags.Join.Name); fl != nil && !fl.Changed { err := errors.WithHint( errors.New("no --join flags provided to 'cockroach start'"), "Consider using 'cockroach init' or 'cockroach start-single-node' instead") @@ -493,6 +517,13 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr // Now perform additional configuration tweaks specific to the start // command. + // Initialize the node's configuration from startup parameters. + // This also reads the part of the configuration that comes from + // environment variables. + if err := initConfigFn(ctx); err != nil { + return errors.Wrapf(err, "failed to initialize %s", serverType) + } + st := serverCfg.BaseConfig.Settings // Derive temporary/auxiliary directory specifications. @@ -512,13 +543,6 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr serverCfg.StorageEngine = enginepb.EngineTypePebble } - // Initialize the node's configuration from startup parameters. - // This also reads the part of the configuration that comes from - // environment variables. - if err := serverCfg.InitNode(ctx); err != nil { - return errors.Wrap(err, "failed to initialize node") - } - // The configuration is now ready to report to the user and the log // file. We had to wait after InitNode() so that all configuration // environment variables, which are reported too, have been read and @@ -552,7 +576,6 @@ If problems persist, please see %s.` initGEOS(ctx) - const serverType redact.SafeString = "node" // Beyond this point, the configuration is set and the server is // ready to start. @@ -560,18 +583,6 @@ If problems persist, please see %s.` // the main goroutine to avoid preventing proper handling of signals // if we get stuck on something during initialization (#10138). - newServerFn := func(_ context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverStartupInterface, error) { - return server.NewServer(serverCfg, stopper) - } - - maybeRunInitialSQL := func(ctx context.Context, s serverStartupInterface) error { - // Run SQL for new clusters. - // - // TODO(knz): If/when we want auto-creation of an initial admin user, - // this can be achieved here. - return runInitialSQL(ctx, s.(*server.Server), startSingleNode, "" /* adminUser */, "" /* adminPassword */) - } - srvStatus, serverShutdownReqC := createAndStartServerAsync(ctx, tBegin, &serverCfg, stopper, startupSpan, newServerFn, maybeRunInitialSQL, serverType) diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index e8fb67bf6ffa..a1fc510b647b 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -694,6 +694,12 @@ func (s *SQLServerWrapper) StartDiagnostics(ctx context.Context) { s.sqlServer.StartDiagnostics(ctx) } +// InitialStart implements cli.serverStartupInterface. For SQL-only servers, +// no start is an initial cluster start. +func (s *SQLServerWrapper) InitialStart() bool { + return false +} + // ShutdownRequested returns a channel that is signaled when a subsystem wants // the server to be shut down. func (s *SQLServerWrapper) ShutdownRequested() <-chan ShutdownRequest {