diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index f677f7e50d9c..8ac25adcb87f 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/envutil" @@ -1192,7 +1191,7 @@ func mtStartSQLFlagsInit(cmd *cobra.Command) error { // We assume that we only need to change top level store as temp dir configs are // initialized when start is executed and temp dirs inherit path from first store. tenantID := fs.Lookup(cliflags.TenantID.Name).Value.String() - serverCfg.Stores.Specs[0].Path = server.DefaultSQLNodeStorePathPrefix + tenantID + serverCfg.Stores.Specs[0].Path += "-tenant-" + tenantID } return nil } diff --git a/pkg/cli/flags_test.go b/pkg/cli/flags_test.go index ec3fffe458db..312b9a8b33e8 100644 --- a/pkg/cli/flags_test.go +++ b/pkg/cli/flags_test.go @@ -1313,10 +1313,15 @@ func TestSQLPodStorageDefaults(t *testing.T) { defer initCLIDefaults() + expectedDefaultDir, err := base.GetAbsoluteStorePath("", "cockroach-data-tenant-9") + if err != nil { + t.Fatal(err) + } + for _, td := range []struct { args []string storePath string - }{{[]string{"mt", "start-sql", "--tenant-id", "9"}, "cockroach-data-tenant-9"}, + }{{[]string{"mt", "start-sql", "--tenant-id", "9"}, expectedDefaultDir}, {[]string{"mt", "start-sql", "--tenant-id", "9", "--store", "/tmp/data"}, "/tmp/data"}, } { t.Run(strings.Join(td.args, ","), func(t *testing.T) { diff --git a/pkg/cli/mt_start_sql.go b/pkg/cli/mt_start_sql.go index 9fb10b1878db..497f05fa86d0 100644 --- a/pkg/cli/mt_start_sql.go +++ b/pkg/cli/mt_start_sql.go @@ -12,17 +12,12 @@ package cli import ( "context" - "fmt" - "os" - "os/signal" "github.com/cockroachdb/cockroach/pkg/cli/clierrorplus" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/sdnotify" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/redact" "github.com/spf13/cobra" ) @@ -61,117 +56,43 @@ 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() + const serverType redact.SafeString = "SQL server" - // 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 - } - - signalCh := make(chan os.Signal, 1) - signal.Notify(signalCh, drainSignals...) - - // 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 - ctx = ambientCtx.AnnotateCtx(ctx) - - stopper, err := setupAndInitializeLoggingAndProfiling(ctx, cmd, false /* isServerCmd */) - if err != nil { - return err - } - defer stopper.Stop(ctx) - stopper.SetTracer(serverCfg.BaseConfig.AmbientCtx.Tracer) - - st := serverCfg.BaseConfig.Settings - - // 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, serverCfg.Settings, stopper, serverCfg.Stores, - ); err != nil { - return err - } - - initGEOS(ctx) - - tenantServer, err := server.NewTenantServer( - ctx, - stopper, - serverCfg.BaseConfig, - serverCfg.SQLConfig, - ) - if err != nil { - return err - } - if err := tenantServer.Start(ctx); err != nil { - return err - } - - // If another process was waiting on the PID (e.g. using a FIFO), - // this is when we can tell them the node has started listening. - if startCtx.pidFile != "" { - log.Ops.Infof(ctx, "PID file: %s", startCtx.pidFile) - if err := os.WriteFile(startCtx.pidFile, []byte(fmt.Sprintf("%d\n", os.Getpid())), 0644); err != nil { - log.Ops.Errorf(ctx, "failed writing the PID: %v", 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 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. + // + // TODO(knz): Check if this special initialization can be removed. + // See: https://github.com/cockroachdb/cockroach/issues/90831 + st := serverCfg.BaseConfig.Settings + return clusterversion.Initialize( + ctx, st.Version.BinaryMinSupportedVersion(), &st.SV, + ) } - // Ensure the configuration logging is written to disk in case a - // process is waiting for the sdnotify readiness to read important - // information from there. - log.Flush() - - // Signal readiness. This unblocks the process when running with - // --background or under systemd. - if err := sdnotify.Ready(); err != nil { - log.Ops.Errorf(ctx, "failed to signal readiness using systemd protocol: %s", err) - } - - // 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) - } - - 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, false /* isHostNode */, false /* initialStart */, tenantClusterID); err != nil { - return err + newServerFn := func(ctx context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverStartupInterface, error) { + // Beware of not writing simply 'return server.NewServer()'. This is + // because it would cause the serverStartupInterface reference to + // always be non-nil, even if NewServer returns a nil pointer (and + // an error). The code below is dependent on the interface + // reference remaining nil in case of error. + s, err := server.NewTenantServer(ctx, stopper, serverCfg.BaseConfig, serverCfg.SQLConfig) + if err != nil { + return nil, err + } + return s, nil } - // 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 33e5bc405d3d..197bc4c1337f 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -311,6 +311,41 @@ func initTempStorageConfig( return tempStorageConfig, nil } +type newServerFn func(ctx context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverStartupInterface, error) + +type serverStartupInterface interface { + serverShutdownInterface + + // ClusterSettings retrieves this server's settings. + ClusterSettings() *cluster.Settings + + // LogicalClusterID retrieves this server's logical cluster ID. + LogicalClusterID() uuid.UUID + + // PreStart starts the server on the specified port(s) and + // initializes subsystems. + // It does not activate the pgwire listener over the network / unix + // socket, which is done by the AcceptClients() method. The separation + // between the two exists so that SQL initialization can take place + // 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 + + // InitialStart returns whether this node is starting for the first time. + // This is (currently) used when displaying the server status report + // on the terminal & in logs. We know that some folk have automation + // that depend on certain strings displayed from this when orchestrating + // KV-only nodes. + InitialStart() bool +} + var errCannotUseJoin = errors.New("cannot use --join with 'cockroach start-single-node' -- use 'cockroach start' instead") func runStartSingleNode(cmd *cobra.Command, args []string) error { @@ -348,7 +383,42 @@ 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) { + // Beware of not writing simply 'return server.NewServer()'. This is + // because it would cause the serverStartupInterface reference to + // always be non-nil, even if NewServer returns a nil pointer (and + // an error). The code below is dependent on the interface + // reference remaining nil in case of error. + s, err := server.NewServer(serverCfg, stopper) + if err != nil { + return nil, err + } + return s, nil + } + + maybeRunInitialSQL := func(ctx context.Context, s serverStartupInterface) error { + // Run SQL for new clusters. + // + // The adminUser/adminPassword fields are for the benefit of 'cockroach demo' + // only and not used here. + return runInitialSQL(ctx, s.(*server.Server), startSingleNode, "" /* adminUser */, "" /* adminPassword */) + } + + return runStartInternal(cmd, serverType, serverCfg.InitNode, newServerFn, maybeRunInitialSQL) +} + +// runStartInternal contains the code common to start a regular server +// or a SQL-only server. +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, @@ -388,11 +458,19 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr 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. @@ -425,12 +503,7 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr if 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()) - } + 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. @@ -445,7 +518,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") @@ -455,28 +528,31 @@ 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. - if serverCfg.Settings.ExternalIODir, err = initExternalIODir(ctx, serverCfg.Stores.Specs[0]); err != nil { + if st.ExternalIODir, err = initExternalIODir(ctx, serverCfg.Stores.Specs[0]); err != nil { return err } - if serverCfg.TempStorageConfig, err = initTempStorageConfig( - ctx, serverCfg.Settings, stopper, serverCfg.Stores, + if serverCfg.SQLConfig.TempStorageConfig, err = initTempStorageConfig( + ctx, st, stopper, serverCfg.Stores, ); err != nil { return err } + // Configure the default storage engine. 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.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 @@ -486,66 +562,7 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr // ReadyFn will be called when the server has started listening on // its network sockets, but perhaps before it has done bootstrapping // and thus before Start() completes. - serverCfg.ReadyFn = func(waitForInit bool) { - // Inform the user if the network settings are suspicious. We need - // to do that after starting to listen because we need to know - // which advertise address NewServer() has decided. - hintServerCmdFlags(ctx, cmd) - - // If another process was waiting on the PID (e.g. using a FIFO), - // this is when we can tell them the node has started listening. - if startCtx.pidFile != "" { - log.Ops.Infof(ctx, "PID file: %s", startCtx.pidFile) - if err := os.WriteFile(startCtx.pidFile, []byte(fmt.Sprintf("%d\n", os.Getpid())), 0644); err != nil { - log.Ops.Errorf(ctx, "failed writing the PID: %v", err) - } - } - - // If the invoker has requested an URL update, do it now that - // the server is ready to accept SQL connections. - // (Note: as stated above, ReadyFn is called after the server - // has started listening on its socket, but possibly before - // the cluster has been initialized and can start processing requests. - // This is OK for SQL clients, as the connection will be accepted - // by the network listener and will just wait/suspend until - // the cluster initializes, at which point it will be picked up - // and let the client go through, transparently.) - if startCtx.listeningURLFile != "" { - log.Ops.Infof(ctx, "listening URL file: %s", startCtx.listeningURLFile) - // (Re-)compute the client connection URL. We cannot do this - // earlier (e.g. above, in the runStart function) because - // at this time the address and port have not been resolved yet. - clientConnOptions, serverParams := makeServerOptionsForURL(&serverCfg) - pgURL, err := clientsecopts.MakeURLForServer(clientConnOptions, serverParams, url.User(username.RootUser)) - if err != nil { - log.Errorf(ctx, "failed computing the URL: %v", err) - return - } - - if err = os.WriteFile(startCtx.listeningURLFile, []byte(fmt.Sprintf("%s\n", pgURL.ToPQ())), 0644); err != nil { - log.Ops.Errorf(ctx, "failed writing the URL: %v", err) - } - } - - if waitForInit { - log.Ops.Shout(ctx, severity.INFO, - "initial startup completed.\n"+ - "Node will now attempt to join a running cluster, or wait for `cockroach init`.\n"+ - "Client connections will be accepted after this completes successfully.\n"+ - "Check the log file(s) for progress. ") - } - - // Ensure the configuration logging is written to disk in case a - // process is waiting for the sdnotify readiness to read important - // information from there. - log.Flush() - - // Signal readiness. This unblocks the process when running with - // --background or under systemd. - if err := sdnotify.Ready(); err != nil { - log.Ops.Errorf(ctx, "failed to signal readiness using systemd protocol: %s", err) - } - } + serverCfg.ReadyFn = func(waitForInit bool) { reportReadinessExternally(ctx, cmd, waitForInit) } // DelayedBootstrapFn will be called if the bootstrap process is // taking a bit long. @@ -571,14 +588,57 @@ If problems persist, please see %s.` // Beyond this point, the configuration is set and the server is // ready to start. - log.Ops.Info(ctx, "starting cockroach node") // Run the rest of the startup process in a goroutine separate from // the main goroutine to avoid preventing proper handling of signals // if we get stuck on something during initialization (#10138). + + srvStatus, serverShutdownReqC := createAndStartServerAsync(ctx, + tBegin, &serverCfg, stopper, startupSpan, newServerFn, maybeRunInitialSQL, serverType) + + return waitForShutdown( + // NB: we delay the access to s, as it is assigned + // asynchronously in a goroutine above. + stopper, serverShutdownReqC, signalCh, + srvStatus) +} + +// createAndStartServerAsync starts an async goroutine which instantiates +// the server and starts it. +// We run it in a separate goroutine because the instantiation&start +// could block, and we want to retain the option to start shutting down +// the process (e.g. via Ctrl+C on the terminal) even in that case. +// The shutdown logic thus starts running asynchronously, via waitForShutdown, +// concurrently with createAndStartServerAsync. +// +// The arguments are as follows: +// - tBegin: time when startup began; used to report statistics at the end of startup. +// - serverCfg: the server configuration. +// - stopper: the stopper used to start all the async tasks. This is the stopper +// used by the shutdown logic. +// - startupSpan: the tracing span for the context that was started earlier +// during startup. It needs to be finalized when the async goroutine completes. +// - newServerFn: a constructor function for the server object. +// - maybeRunInitialSQL: a callback that will be called after the server has +// initialized, but before it starts accepting clients. +// - serverType: a title used for the type of server. This is used +// when reporting the startup messages on the terminal & logs. +func createAndStartServerAsync( + ctx context.Context, + tBegin time.Time, + serverCfg *server.Config, + stopper *stop.Stopper, + startupSpan *tracing.Span, + newServerFn newServerFn, + maybeRunInitialSQL func(context.Context, serverStartupInterface) error, + serverType redact.SafeString, +) (srvStatus *serverStatus, serverShutdownReqC <-chan server.ShutdownRequest) { var serverStatusMu serverStatus - var s *server.Server + var s serverStartupInterface shutdownReqC := make(chan server.ShutdownRequest, 1) + + log.Ops.Infof(ctx, "starting cockroach %s", serverType) + go func() { // Ensure that the log files see the startup messages immediately. defer log.Flush() @@ -594,15 +654,14 @@ If problems persist, please see %s.` logcrash.RecoverAndReportPanic(ctx, &s.ClusterSettings().SV) } }() - // When the start up goroutine completes, so can the start up span - // defined above. + // When the start up goroutine completes, so can the start up span. defer startupSpan.Finish() // Any error beyond this point is reported through shutdownReqC. if err := func() error { // Instantiate the server. var err error - s, err = server.NewServer(serverCfg, stopper) + s, err = newServerFn(ctx, *serverCfg, stopper) if err != nil { return errors.Wrap(err, "failed to start server") } @@ -651,11 +710,10 @@ If problems persist, please see %s.` } initialStart := s.InitialStart() - // Run SQL for new clusters. - // TODO(knz): If/when we want auto-creation of an initial admin user, - // this can be achieved here. - if err := runInitialSQL(ctx, s, startSingleNode, "" /* adminUser */, "" /* adminPassword */); err != nil { - return err + if maybeRunInitialSQL != nil { + if err := maybeRunInitialSQL(ctx, s); err != nil { + return err + } } // Now let SQL clients in. @@ -665,8 +723,8 @@ If problems persist, please see %s.` // Now inform the user that the server is running and tell the // user about its run-time derived parameters. - return reportServerInfo(ctx, tBegin, &serverCfg, s.ClusterSettings(), - true /* isHostNode */, initialStart, uuid.UUID{} /* tenantClusterID */) + return reportServerInfo(ctx, tBegin, serverCfg, s.ClusterSettings(), + serverType, initialStart, s.LogicalClusterID()) }(); err != nil { shutdownReqC <- server.MakeShutdownRequest( server.ShutdownReasonServerStartupError, errors.Wrapf(err, "server startup failed")) @@ -683,7 +741,9 @@ If problems persist, please see %s.` } }() - return waitForShutdown(stopper, shutdownReqC, signalCh, &serverStatusMu) + serverShutdownReqC = shutdownReqC + srvStatus = &serverStatusMu + return srvStatus, serverShutdownReqC } // serverStatus coordinates the async goroutine that starts the server @@ -752,6 +812,7 @@ func (s *serverStatus) startShutdown() (bool, serverShutdownInterface, *stop.Sto type serverShutdownInterface interface { AnnotateCtx(context.Context) context.Context Drain(ctx context.Context, verbose bool) (uint64, redact.RedactableString, error) + ShutdownRequested() <-chan server.ShutdownRequest } // waitForShutdown blocks until interrupted by a shutdown signal, which can come @@ -766,7 +827,7 @@ type serverShutdownInterface interface { func waitForShutdown( stopper *stop.Stopper, shutdownC <-chan server.ShutdownRequest, - signalCh chan os.Signal, + signalCh <-chan os.Signal, serverStatusMu *serverStatus, ) (returnErr error) { shutdownCtx, shutdownSpan := serverCfg.AmbientCtx.AnnotateCtxWithSpan(context.Background(), "server shutdown") @@ -1000,17 +1061,13 @@ func reportServerInfo( startTime time.Time, serverCfg *server.Config, st *cluster.Settings, - isHostNode, initialStart bool, + serverType redact.SafeString, + initialStart bool, tenantClusterID uuid.UUID, ) error { - srvS := redact.SafeString("SQL server") - if isHostNode { - srvS = "node" - } - var buf redact.StringBuilder info := build.GetInfo() - buf.Printf("CockroachDB %s starting at %s (took %0.1fs)\n", srvS, timeutil.Now(), timeutil.Since(startTime).Seconds()) + buf.Printf("CockroachDB %s starting at %s (took %0.1fs)\n", serverType, timeutil.Now(), timeutil.Since(startTime).Seconds()) buf.Printf("build:\t%s %s @ %s (%s)\n", redact.Safe(info.Distribution), redact.Safe(info.Tag), redact.Safe(info.Time), redact.Safe(info.GoVersion)) buf.Printf("webui:\t%s\n", log.SafeManaged(serverCfg.AdminURL())) @@ -1066,14 +1123,14 @@ func reportServerInfo( buf.Printf("cluster name:\t%s\n", log.SafeManaged(baseCfg.ClusterName)) } clusterID := serverCfg.BaseConfig.ClusterIDContainer.Get() - if tenantClusterID.Equal(uuid.Nil) { + if tenantClusterID.Equal(clusterID) { buf.Printf("clusterID:\t%s\n", log.SafeManaged(clusterID)) } else { buf.Printf("storage clusterID:\t%s\n", log.SafeManaged(clusterID)) buf.Printf("tenant clusterID:\t%s\n", log.SafeManaged(tenantClusterID)) } nodeID := serverCfg.BaseConfig.IDContainer.Get() - if isHostNode { + if serverCfg.SQLConfig.TenantID.IsSystem() { if initialStart { if nodeID == kvserver.FirstNodeID { buf.Printf("status:\tinitialized new cluster\n") @@ -1108,7 +1165,7 @@ func reportServerInfo( return err } msgS := msg.ToString() - log.Ops.Infof(ctx, "%s startup completed:\n%s", srvS, msgS) + log.Ops.Infof(ctx, "%s startup completed:\n%s", serverType, msgS) if !startCtx.inBackground && !log.LoggingToStderr(severity.INFO) { fmt.Print(msgS.StripMarkers()) } @@ -1142,8 +1199,13 @@ func hintServerCmdFlags(ctx context.Context, cmd *cobra.Command) { "This feature will be removed in the next version of CockroachDB.") } - listenAddrSpecified := pf.Lookup(cliflags.ListenAddr.Name).Changed || pf.Lookup(cliflags.ServerHost.Name).Changed - advAddrSpecified := pf.Lookup(cliflags.AdvertiseAddr.Name).Changed || pf.Lookup(cliflags.AdvertiseHost.Name).Changed + changed := func(flagName string) bool { + fl := pf.Lookup(cliflags.ListenAddr.Name) + return fl != nil && fl.Changed + } + + listenAddrSpecified := changed(cliflags.ListenAddr.Name) || changed(cliflags.ServerHost.Name) + advAddrSpecified := changed(cliflags.AdvertiseAddr.Name) || changed(cliflags.AdvertiseHost.Name) if !listenAddrSpecified && !advAddrSpecified { host, _, _ := net.SplitHostPort(serverCfg.AdvertiseAddr) log.Ops.Shoutf(ctx, severity.WARNING, @@ -1334,3 +1396,68 @@ func initGEOS(ctx context.Context) { log.Ops.Infof(ctx, "GEOS loaded from directory %s", log.SafeManaged(loc)) } } + +// reportReadinessExternally reports when the server has finished initializing +// and is ready to receive requests. This is useful for other processes on the +// same machine (e.g. a process manager, a test) that are waiting for a signal +// that they can start monitoring or using the server process. +func reportReadinessExternally(ctx context.Context, cmd *cobra.Command, waitForInit bool) { + // Inform the user if the network settings are suspicious. We need + // to do that after starting to listen because we need to know + // which advertise address NewServer() has decided. + hintServerCmdFlags(ctx, cmd) + + // If another process was waiting on the PID (e.g. using a FIFO), + // this is when we can tell them the node has started listening. + if startCtx.pidFile != "" { + log.Ops.Infof(ctx, "PID file: %s", startCtx.pidFile) + if err := os.WriteFile(startCtx.pidFile, []byte(fmt.Sprintf("%d\n", os.Getpid())), 0644); err != nil { + log.Ops.Errorf(ctx, "failed writing the PID: %v", err) + } + } + + // If the invoker has requested an URL update, do it now that + // the server is ready to accept SQL connections. + // (Note: as stated above, ReadyFn is called after the server + // has started listening on its socket, but possibly before + // the cluster has been initialized and can start processing requests. + // This is OK for SQL clients, as the connection will be accepted + // by the network listener and will just wait/suspend until + // the cluster initializes, at which point it will be picked up + // and let the client go through, transparently.) + if startCtx.listeningURLFile != "" { + log.Ops.Infof(ctx, "listening URL file: %s", startCtx.listeningURLFile) + // (Re-)compute the client connection URL. We cannot do this + // earlier (e.g. above, in the runStart function) because + // at this time the address and port have not been resolved yet. + clientConnOptions, serverParams := makeServerOptionsForURL(&serverCfg) + pgURL, err := clientsecopts.MakeURLForServer(clientConnOptions, serverParams, url.User(username.RootUser)) + if err != nil { + log.Errorf(ctx, "failed computing the URL: %v", err) + return + } + + if err = os.WriteFile(startCtx.listeningURLFile, []byte(fmt.Sprintf("%s\n", pgURL.ToPQ())), 0644); err != nil { + log.Ops.Errorf(ctx, "failed writing the URL: %v", err) + } + } + + if waitForInit { + log.Ops.Shout(ctx, severity.INFO, + "initial startup completed.\n"+ + "Node will now attempt to join a running cluster, or wait for `cockroach init`.\n"+ + "Client connections will be accepted after this completes successfully.\n"+ + "Check the log file(s) for progress. ") + } + + // Ensure the configuration logging is written to disk in case a + // process is waiting for the sdnotify readiness to read important + // information from there. + log.Flush() + + // Signal readiness. This unblocks the process when running with + // --background or under systemd. + if err := sdnotify.Ready(); err != nil { + log.Ops.Errorf(ctx, "failed to signal readiness using systemd protocol: %s", err) + } +} diff --git a/pkg/server/config.go b/pkg/server/config.go index ca13a8154004..6412a1781609 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -64,10 +64,6 @@ const ( defaultScanMaxIdleTime = 1 * time.Second DefaultStorePath = "cockroach-data" - // DefaultSQLNodeStorePathPrefix is path prefix that is used by default - // on tenant sql nodes to separate from server node if running on the - // same server without explicit --store location. - DefaultSQLNodeStorePathPrefix = "cockroach-data-tenant-" // TempDirPrefix is the filename prefix of any temporary subdirectory // created. TempDirPrefix = "cockroach-temp" @@ -338,12 +334,6 @@ type KVConfig struct { // The following values can only be set via environment variables and are // for testing only. They are not meant to be set by the end user. - // Enables linearizable behavior of operations on this node by making sure - // that no commit timestamp is reported back to the client until all other - // node clocks have necessarily passed it. - // Environment Variable: COCKROACH_EXPERIMENTAL_LINEARIZABLE - Linearizable bool - // ScanInterval determines a duration during which each range should be // visited approximately once by the range scanner. Set to 0 to disable. // Environment Variable: COCKROACH_SCAN_INTERVAL @@ -441,6 +431,15 @@ type SQLConfig struct { // // Only applies when the SQL server is deployed individually. TenantKVAddrs []string + + // The following values can only be set via environment variables and are + // for testing only. They are not meant to be set by the end user. + + // Enables linearizable behavior of operations on this node by making sure + // that no commit timestamp is reported back to the client until all other + // node clocks have necessarily passed it. + // Environment Variable: COCKROACH_EXPERIMENTAL_LINEARIZABLE + Linearizable bool } // MakeSQLConfig returns a SQLConfig with default values. @@ -754,7 +753,16 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { return enginesCopy, nil } -// InitNode parses node attributes and bootstrap addresses. +// InitSQLServer finalizes the configuration of a SQL-only node. +// It initializes additional configuration flags from the environment. +func (cfg *Config) InitSQLServer(ctx context.Context) error { + cfg.readSQLEnvironmentVariables() + return nil +} + +// InitNode finalizes the configuration of a KV node. +// It parses node attributes and bootstrap addresses and +// initializes additional configuration flags from the environment. func (cfg *Config) InitNode(ctx context.Context) error { cfg.readEnvironmentVariables() @@ -804,12 +812,16 @@ func (cfg *BaseConfig) RequireWebSession() bool { return !cfg.Insecure && cfg.EnableWebSessionAuthentication } +func (cfg *Config) readSQLEnvironmentVariables() { + cfg.SpanConfigsDisabled = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_SPAN_CONFIGS", cfg.SpanConfigsDisabled) + cfg.Linearizable = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_LINEARIZABLE", cfg.Linearizable) +} + // readEnvironmentVariables populates all context values that are environment // variable based. Note that this only happens when initializing a node and not // when NewContext is called. func (cfg *Config) readEnvironmentVariables() { - cfg.SpanConfigsDisabled = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_SPAN_CONFIGS", cfg.SpanConfigsDisabled) - cfg.Linearizable = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_LINEARIZABLE", cfg.Linearizable) + cfg.readSQLEnvironmentVariables() cfg.ScanInterval = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_INTERVAL", cfg.ScanInterval) cfg.ScanMinIdleTime = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_MIN_IDLE_TIME", cfg.ScanMinIdleTime) cfg.ScanMaxIdleTime = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_MAX_IDLE_TIME", cfg.ScanMaxIdleTime) diff --git a/pkg/server/server.go b/pkg/server/server.go index 5493c9f2bb50..ef8a0b3ec02d 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -805,15 +805,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { internalExecutor, ) - // Configure the path to the file that blocks starting background jobs. - var jobAdoptionStopFile string - for _, spec := range cfg.Stores.Specs { - if !spec.InMemory && spec.Path != "" { - jobAdoptionStopFile = filepath.Join(spec.Path, jobs.PreventAdoptionFile) - break - } - } - // Instantiate the KV prober. kvProber := kvprober.NewProber(kvprober.Opts{ Tracer: cfg.AmbientCtx.Tracer, @@ -879,7 +870,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { circularInternalExecutor: internalExecutor, internalExecutorFactory: internalExecutorFactory, circularJobRegistry: jobRegistry, - jobAdoptionStopFile: jobAdoptionStopFile, protectedtsProvider: protectedtsProvider, rangeFeedFactory: rangeFeedFactory, sqlStatusServer: sStatus, @@ -1806,6 +1796,12 @@ func (s *Server) PGServer() *pgwire.Server { return s.sqlServer.pgServer } +// LogicalClusterID implements cli.serverStartupInterface. This +// implementation exports the logical cluster ID of the system tenant. +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. diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index ae83f379e413..643e66bd25f3 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -325,7 +325,6 @@ type sqlServerArgs struct { // pointer to an empty struct in this configuration, which newSQLServer // fills. circularJobRegistry *jobs.Registry - jobAdoptionStopFile string // The executorConfig uses the provider. protectedtsProvider protectedts.Provider @@ -442,6 +441,15 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { codec = keys.MakeSQLCodec(override) } } + + var jobAdoptionStopFile string + for _, spec := range cfg.Stores.Specs { + if !spec.InMemory && spec.Path != "" { + jobAdoptionStopFile = filepath.Join(spec.Path, jobs.PreventAdoptionFile) + break + } + } + // Create blob service for inter-node file sharing. blobService, err := blobs.NewBlobService(cfg.Settings.ExternalIODir) if err != nil { @@ -531,7 +539,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // in sql/jobs/registry.go on planHookMaker. return sql.MakeJobExecContext(opName, user, &sql.MemoryMetrics{}, execCfg) }, - cfg.jobAdoptionStopFile, + jobAdoptionStopFile, td, jobsKnobs, ) diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 857d28e02339..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 { @@ -789,7 +795,7 @@ func makeTenantSQLServerArgs( Clock: clock, Stopper: stopper, HeartbeatInterval: base.DefaultTxnHeartbeatInterval, - Linearizable: false, + Linearizable: sqlCfg.Linearizable, Metrics: txnMetrics, TestingKnobs: clientKnobs, },