metropolis/node/core: migrate main to osbase/bringup

This migrates our existing startup code to use our bringup library

Change-Id: Ic78c80bc4631fa2a67cd6a3db1ea19642b20aea9
Reviewed-on: https://review.monogon.dev/c/monogon/+/3706
Tested-by: Jenkins CI
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
Reviewed-by: Jan Schär <jan@monogon.tech>
diff --git a/metropolis/node/core/BUILD.bazel b/metropolis/node/core/BUILD.bazel
index 4f15dea..4e0b3b8 100644
--- a/metropolis/node/core/BUILD.bazel
+++ b/metropolis/node/core/BUILD.bazel
@@ -7,8 +7,6 @@
         "main.go",
         "mounts.go",
         "nodeparams.go",
-        "panichandler.go",
-        "pstore.go",
         "sysctl.go",
     ] + select({
         "//metropolis/node:debug_build": [
@@ -39,9 +37,9 @@
         "//metropolis/node/core/update",
         "//metropolis/proto/api",
         "//metropolis/version",
+        "//osbase/bringup",
         "//osbase/logtree",
         "//osbase/net/dns",
-        "//osbase/pstore",
         "//osbase/supervisor",
         "//osbase/sysctl",
         "//osbase/tpm",
diff --git a/metropolis/node/core/main.go b/metropolis/node/core/main.go
index 142242b..527030a 100644
--- a/metropolis/node/core/main.go
+++ b/metropolis/node/core/main.go
@@ -19,9 +19,7 @@
 import (
 	"context"
 	"fmt"
-	"os"
 	"strings"
-	"time"
 
 	"golang.org/x/sys/unix"
 
@@ -38,6 +36,7 @@
 	timesvc "source.monogon.dev/metropolis/node/core/time"
 	"source.monogon.dev/metropolis/node/core/update"
 	mversion "source.monogon.dev/metropolis/version"
+	"source.monogon.dev/osbase/bringup"
 	"source.monogon.dev/osbase/logtree"
 	"source.monogon.dev/osbase/net/dns"
 	"source.monogon.dev/osbase/supervisor"
@@ -47,243 +46,17 @@
 )
 
 func main() {
-	// Set up basic mounts (like /dev, /sys...).
-	if err := setupMounts(); err != nil {
-		panic(fmt.Errorf("could not set up basic mounts: %w", err))
-	}
-
-	// Root system logtree.
-	lt := logtree.New()
-
-	// Set up logger for Metropolis. Currently logs everything to /dev/ttyS{0,1}.
-	serialConsoles := []*console{
-		{
-			path:     "/dev/ttyS0",
-			maxWidth: 120,
+	bringup.Runnable(root).RunWith(bringup.Config{
+		Console: bringup.ConsoleConfig{
+			ShortenDictionary: logtree.MetropolisShortenDict,
+			Filter:            consoleFilter,
 		},
-		{
-			path:     "/dev/ttyS1",
-			maxWidth: 120,
+		Supervisor: bringup.SupervisorConfig{
+			Metrics: []supervisor.Metrics{
+				supervisor.NewMetricsPrometheus(metrics.CoreRegistry),
+			},
 		},
-	}
-	// Alternative channel that crash handling writes to, and which gets distributed
-	// to the consoles.
-	crash := make(chan string)
-
-	// Open up consoles and set up logging from logtree and crash channel.
-	for _, c := range serialConsoles {
-		f, err := os.OpenFile(c.path, os.O_WRONLY, 0)
-		if err != nil {
-			continue
-		}
-		reader, err := lt.Read("", logtree.WithChildren(), logtree.WithStream())
-		if err != nil {
-			panic(fmt.Sprintf("could not set up root log reader: %v", err))
-		}
-		c.reader = reader
-		go func() {
-			fmt.Fprintf(f, "\nMetropolis: this is %s. Verbose node logs follow.\n\n", c.path)
-			for {
-				select {
-				case p := <-reader.Stream:
-					if consoleFilter(p) {
-						fmt.Fprintf(f, "%s\n", p.ConciseString(logtree.MetropolisShortenDict, c.maxWidth))
-					}
-				case s := <-crash:
-					fmt.Fprintf(f, "%s\n", s)
-				}
-			}
-		}()
-	}
-
-	// Initialize persistent panic handler early
-	initPanicHandler(lt, serialConsoles)
-
-	// Initial logger. Used until we get to a supervisor.
-	logger := lt.MustLeveledFor("init")
-
-	// Linux kernel default is 4096 which is far too low. Raise it to 1M which
-	// is what gVisor suggests.
-	if err := unix.Setrlimit(unix.RLIMIT_NOFILE, &unix.Rlimit{Cur: 1048576, Max: 1048576}); err != nil {
-		logger.Fatalf("Failed to raise rlimits: %v", err)
-	}
-
-	logger.Info("Starting Metropolis node init")
-	logger.Infof("Version: %s", version.Semver(mversion.Version))
-
-	haveTPM := true
-	if err := tpm.Initialize(logger); err != nil {
-		logger.Warningf("Failed to initialize TPM 2.0: %v", err)
-		haveTPM = false
-	}
-
-	metrics.CoreRegistry.MustRegister(dns.MetricsRegistry)
-	networkSvc := network.New(nil, []string{"hosts", "kubernetes"})
-	networkSvc.DHCPVendorClassID = "dev.monogon.metropolis.node.v1"
-	timeSvc := timesvc.New()
-	devmgrSvc := devmgr.New()
-
-	// This function initializes a headless Delve if this is a debug build or
-	// does nothing if it's not
-	initializeDebugger(networkSvc)
-
-	// Prepare local storage.
-	root := &localstorage.Root{}
-	if err := declarative.PlaceFS(root, "/"); err != nil {
-		panic(fmt.Errorf("when placing root FS: %w", err))
-	}
-
-	updateSvc := &update.Service{
-		Logger: lt.MustLeveledFor("update"),
-	}
-
-	// Make context for supervisor. We cancel it when we reach the trapdoor.
-	ctxS, ctxC := context.WithCancel(context.Background())
-
-	// Make node-wide cluster resolver.
-	res := resolver.New(ctxS, resolver.WithLogger(lt.MustLeveledFor("resolver")))
-
-	// Function which performs core, one-way initialization of the node. This means
-	// waiting for the network, starting the cluster manager, and then starting all
-	// services related to the node's roles.
-	init := func(ctx context.Context) error {
-		// Start storage and network - we need this to get anything else done.
-		if err := root.Start(ctx, updateSvc); err != nil {
-			return fmt.Errorf("cannot start root FS: %w", err)
-		}
-		localNodeParams, err := getLocalNodeParams(ctx, root)
-		if err != nil {
-			return fmt.Errorf("cannot get local node parameters: %w", err)
-		}
-		if localNodeParams.NetworkConfig != nil {
-			networkSvc.StaticConfig = localNodeParams.NetworkConfig
-			if err := root.ESP.Metropolis.NetworkConfiguration.Marshal(localNodeParams.NetworkConfig); err != nil {
-				logger.Errorf("Error writing back network_config from NodeParameters: %v", err)
-			}
-		}
-		if networkSvc.StaticConfig == nil {
-			staticConfig, err := root.ESP.Metropolis.NetworkConfiguration.Unmarshal()
-			if err == nil {
-				networkSvc.StaticConfig = staticConfig
-			} else {
-				logger.Errorf("Unable to load static config, proceeding without it: %v", err)
-			}
-		}
-		if err := supervisor.Run(ctx, "devmgr", devmgrSvc.Run); err != nil {
-			return fmt.Errorf("when starting devmgr: %w", err)
-		}
-		if err := supervisor.Run(ctx, "network", networkSvc.Run); err != nil {
-			return fmt.Errorf("when starting network: %w", err)
-		}
-		if err := supervisor.Run(ctx, "time", timeSvc.Run); err != nil {
-			return fmt.Errorf("when starting time: %w", err)
-		}
-		if err := supervisor.Run(ctx, "pstore", dumpAndCleanPstore); err != nil {
-			return fmt.Errorf("when starting pstore: %w", err)
-		}
-		if err := supervisor.Run(ctx, "sysctl", nodeSysctls); err != nil {
-			return fmt.Errorf("when applying sysctls: %w", err)
-		}
-
-		// The kernel does of course not run in this runnable, only the log pipe
-		// runs in it.
-		if err := supervisor.Run(ctx, "kernel", func(ctx context.Context) error {
-			return logtree.KmsgPipe(ctx, supervisor.Logger(ctx))
-		}); err != nil {
-			return fmt.Errorf("when starting kernel log pipe: %w", err)
-		}
-
-		// Start the role service. The role service connects to the curator and runs
-		// all node-specific role code (eg. Kubernetes services).
-		//   supervisor.Logger(ctx).Infof("Starting role service...")
-		rs := roleserve.New(roleserve.Config{
-			StorageRoot: root,
-			Network:     networkSvc,
-			Resolver:    res,
-			LogTree:     lt,
-			Update:      updateSvc,
-		})
-		if err := supervisor.Run(ctx, "role", rs.Run); err != nil {
-			return fmt.Errorf("failed to start role service: %w", err)
-		}
-
-		if err := runDebugService(ctx, rs, lt, root); err != nil {
-			return fmt.Errorf("when starting debug service: %w", err)
-		}
-
-		// Initialize interactive consoles.
-		interactiveConsoles := []string{"/dev/tty0"}
-		for _, c := range interactiveConsoles {
-			console, err := tconsole.New(tconsole.TerminalLinux, c, lt, &networkSvc.Status, &rs.LocalRoles, &rs.CuratorConnection)
-			if err != nil {
-				logger.Infof("Failed to initialize interactive console at %s: %v", c, err)
-				// TODO: fall back to logger
-			} else {
-				logger.Infof("Started interactive console at %s", c)
-				supervisor.Run(ctx, "console-"+c, console.Run)
-			}
-		}
-
-		// Now that we have consoles, set console logging level to 1 (KERNEL_EMERG,
-		// minimum possible). This prevents the TUI console from being polluted by
-		// random printks.
-		opts := sysctl.Options{
-			"kernel.printk": "1",
-		}
-		if err := opts.Apply(); err != nil {
-			logger.Errorf("Failed to configure printk logging: %v", err)
-		}
-
-		nodeParams, err := getNodeParams(ctx, root)
-		if err != nil {
-			return fmt.Errorf("cannot get node parameters: %w", err)
-		}
-
-		// Start cluster manager. This kicks off cluster membership machinery,
-		// which will either start a new cluster, enroll into one or join one.
-		m := cluster.NewManager(root, networkSvc, rs, updateSvc, nodeParams, haveTPM)
-		return m.Run(ctx)
-	}
-
-	// Start the init function in a one-shot runnable. Smuggle out any errors from
-	// the init function and stuff them into the fatal channel. This is where the
-	// system supervisor takes over as the main process management system.
-	fatal := make(chan error)
-	supervisor.New(ctxS, func(ctx context.Context) error {
-		err := init(ctx)
-		if err != nil {
-			fatal <- err
-			select {}
-		}
-		return nil
-	}, supervisor.WithExistingLogtree(lt), supervisor.WithMetrics(supervisor.NewMetricsPrometheus(metrics.CoreRegistry)))
-
-	// Meanwhile, wait for any fatal error from the init process, and handle it
-	// accordingly.
-	err := <-fatal
-	// Log error with primary logging mechanism still active.
-	logger.Infof("Node startup failed: %v", err)
-	// Start shutting down the supervision tree...
-	ctxC()
-	time.Sleep(time.Second)
-	// After a bit, kill all console log readers.
-	for _, c := range serialConsoles {
-		if c.reader == nil {
-			continue
-		}
-		c.reader.Close()
-		c.reader.Stream = nil
-	}
-	// Wait for final logs to flush to console...
-	time.Sleep(time.Second)
-	// Present final message to the console.
-	crash <- ""
-	crash <- ""
-	crash <- fmt.Sprintf(" Fatal error: %v", err)
-	crash <- " This node could not be started. Rebooting..."
-	time.Sleep(time.Second)
-	// Return to minit, which will reboot this node.
-	os.Exit(0)
+	})
 }
 
 // consoleFilter is used to filter out some uselessly verbose logs from the
@@ -329,8 +102,148 @@
 	return true
 }
 
-type console struct {
-	path     string
-	maxWidth int
-	reader   *logtree.LogReader
+// Function which performs core, one-way initialization of the node. This means
+// waiting for the network, starting the cluster manager, and then starting all
+// services related to the node's roles.
+func root(ctx context.Context) error {
+	logger := supervisor.Logger(ctx)
+
+	logger.Info("Starting Metropolis node init")
+	logger.Infof("Version: %s", version.Semver(mversion.Version))
+
+	// Linux kernel default is 4096 which is far too low. Raise it to 1M which
+	// is what gVisor suggests.
+	if err := unix.Setrlimit(unix.RLIMIT_NOFILE, &unix.Rlimit{Cur: 1048576, Max: 1048576}); err != nil {
+		logger.Fatalf("Failed to raise rlimits: %v", err)
+	}
+
+	haveTPM := true
+	if err := tpm.Initialize(logger); err != nil {
+		logger.Warningf("Failed to initialize TPM 2.0: %v", err)
+		haveTPM = false
+	}
+
+	metrics.CoreRegistry.MustRegister(dns.MetricsRegistry)
+	networkSvc := network.New(nil, []string{"hosts", "kubernetes"})
+	networkSvc.DHCPVendorClassID = "dev.monogon.metropolis.node.v1"
+	timeSvc := timesvc.New()
+	devmgrSvc := devmgr.New()
+
+	// This function initializes a headless Delve if this is a debug build or
+	// does nothing if it's not
+	initializeDebugger(networkSvc)
+
+	// Prepare local storage.
+	root := &localstorage.Root{}
+	if err := declarative.PlaceFS(root, "/"); err != nil {
+		panic(fmt.Errorf("when placing root FS: %w", err))
+	}
+
+	updateSvc := &update.Service{
+		Logger: supervisor.MustSubLogger(ctx, "update"),
+	}
+	// Make node-wide cluster resolver.
+	res := resolver.New(ctx, resolver.WithLogger(supervisor.MustSubLogger(ctx, "resolver")))
+
+	// Start storage and network - we need this to get anything else done.
+	if err := root.Start(ctx, updateSvc); err != nil {
+		return fmt.Errorf("cannot start root FS: %w", err)
+	}
+
+	localNodeParams, err := getLocalNodeParams(ctx, root)
+	if err != nil {
+		return fmt.Errorf("cannot get local node parameters: %w", err)
+	}
+
+	if localNodeParams.NetworkConfig != nil {
+		networkSvc.StaticConfig = localNodeParams.NetworkConfig
+		if err := root.ESP.Metropolis.NetworkConfiguration.Marshal(localNodeParams.NetworkConfig); err != nil {
+			logger.Errorf("Error writing back network_config from NodeParameters: %v", err)
+		}
+	}
+	if networkSvc.StaticConfig == nil {
+		staticConfig, err := root.ESP.Metropolis.NetworkConfiguration.Unmarshal()
+		if err == nil {
+			networkSvc.StaticConfig = staticConfig
+		} else {
+			logger.Errorf("Unable to load static config, proceeding without it: %v", err)
+		}
+	}
+
+	if err := supervisor.Run(ctx, "devmgr", devmgrSvc.Run); err != nil {
+		return fmt.Errorf("when starting devmgr: %w", err)
+	}
+	if err := supervisor.Run(ctx, "network", networkSvc.Run); err != nil {
+		return fmt.Errorf("when starting network: %w", err)
+	}
+	if err := supervisor.Run(ctx, "time", timeSvc.Run); err != nil {
+		return fmt.Errorf("when starting time: %w", err)
+	}
+	if err := supervisor.Run(ctx, "sysctl", nodeSysctls); err != nil {
+		return fmt.Errorf("when applying sysctls: %w", err)
+	}
+
+	// The kernel does of course not run in this runnable, only the log pipe
+	// runs in it.
+	if err := supervisor.Run(ctx, "kernel", func(ctx context.Context) error {
+		return logtree.KmsgPipe(ctx, supervisor.Logger(ctx))
+	}); err != nil {
+		return fmt.Errorf("when starting kernel log pipe: %w", err)
+	}
+
+	// Start the role service. The role service connects to the curator and runs
+	// all node-specific role code (eg. Kubernetes services).
+	logger.Infof("Starting role service...")
+	rs := roleserve.New(roleserve.Config{
+		StorageRoot: root,
+		Network:     networkSvc,
+		Resolver:    res,
+		LogTree:     supervisor.LogTree(ctx),
+		Update:      updateSvc,
+	})
+	if err := supervisor.Run(ctx, "role", rs.Run); err != nil {
+		return fmt.Errorf("failed to start role service: %w", err)
+	}
+
+	if err := runDebugService(ctx, rs, supervisor.LogTree(ctx), root); err != nil {
+		return fmt.Errorf("when starting debug service: %w", err)
+	}
+
+	// Initialize interactive consoles.
+	interactiveConsoles := []string{"/dev/tty0"}
+	for _, c := range interactiveConsoles {
+		console, err := tconsole.New(tconsole.TerminalLinux, c, supervisor.LogTree(ctx), &networkSvc.Status, &rs.LocalRoles, &rs.CuratorConnection)
+		if err != nil {
+			logger.Infof("Failed to initialize interactive console at %s: %v", c, err)
+		} else {
+			logger.Infof("Started interactive console at %s", c)
+			supervisor.Run(ctx, "console-"+c, console.Run)
+		}
+	}
+
+	// Now that we have consoles, set console logging level to 1 (KERNEL_EMERG,
+	// minimum possible). This prevents the TUI console from being polluted by
+	// random printks.
+	opts := sysctl.Options{
+		"kernel.printk": "1",
+	}
+	if err := opts.Apply(); err != nil {
+		logger.Errorf("Failed to configure printk logging: %v", err)
+	}
+
+	nodeParams, err := getNodeParams(ctx, root)
+	if err != nil {
+		return fmt.Errorf("cannot get node parameters: %w", err)
+	}
+
+	// Start cluster manager. This kicks off cluster membership machinery,
+	// which will either start a new cluster, enroll into one or join one.
+	m := cluster.NewManager(root, networkSvc, rs, updateSvc, nodeParams, haveTPM)
+	if err := supervisor.Run(ctx, "cluster-manager", m.Run); err != nil {
+		return fmt.Errorf("when starting cluster manager: %w", err)
+	}
+
+	supervisor.Signal(ctx, supervisor.SignalHealthy)
+	supervisor.Signal(ctx, supervisor.SignalDone)
+	return nil
 }
diff --git a/metropolis/node/core/panichandler.go b/metropolis/node/core/panichandler.go
deleted file mode 100644
index 8a6ef6c..0000000
--- a/metropolis/node/core/panichandler.go
+++ /dev/null
@@ -1,81 +0,0 @@
-// This uses the unstable overrideWrite interface to also emit all runtime
-// writes to a dedicated runtime file descriptor to catch and debug crash dumps.
-// See https://go-review.googlesource.com/c/go/+/278792 for details about the
-// interface. This interface is relatively special, refrain from using most Go
-// features in here as it might cause unexpected behavior. Especially yielding
-// is a bad idea as the scheduler might be in an inconsistent state. But using
-// this interface was judged to be vastly more maintenance-friendly than
-// attempting to parse out this information from a combined stderr.
-package main
-
-import (
-	"os"
-	"unsafe"
-
-	"golang.org/x/sys/unix"
-
-	"source.monogon.dev/osbase/logtree"
-)
-
-// This hooks into a global variable which is checked by runtime.write and used
-// instead of runtime.write1 if populated.
-//
-//go:linkname overrideWrite runtime.overrideWrite
-var overrideWrite func(fd uintptr, p unsafe.Pointer, n int32) int32
-
-// Contains the files into which runtime logs and crashes are written.
-var runtimeFds []int
-
-// This is essentially a reimplementation of the assembly function
-// runtime.write1, just with a hardcoded file descriptor and using the assembly
-// function unix.RawSyscall to not get a dependency on Go's calling convention
-// and needing an implementation for every architecture.
-//
-//go:nosplit
-func runtimeWrite(fd uintptr, p unsafe.Pointer, n int32) int32 {
-	// Only redirect writes to stderr.
-	if fd != 2 {
-		a, _, err := unix.RawSyscall(unix.SYS_WRITE, fd, uintptr(p), uintptr(n))
-		if err == 0 {
-			return int32(a)
-		}
-		return int32(err)
-	}
-	// Write to the runtime panic FDs.
-	for _, f := range runtimeFds {
-		_, _, _ = unix.RawSyscall(unix.SYS_WRITE, uintptr(f), uintptr(p), uintptr(n))
-	}
-
-	// Finally, write to original FD
-	a, _, err := unix.RawSyscall(unix.SYS_WRITE, fd, uintptr(p), uintptr(n))
-	if err == 0 {
-		return int32(a)
-	}
-	return int32(err)
-}
-
-func initPanicHandler(lt *logtree.LogTree, consoles []*console) {
-	l := lt.MustLeveledFor("panichandler")
-
-	// Setup pstore userspace message buffer
-	fd, err := unix.Open("/dev/pmsg0", os.O_WRONLY, 0)
-	if err != nil {
-		l.Errorf("Failed to open pstore userspace device (pstore probably unavailable): %v", err)
-		l.Warningf("Continuing without persistent panic storage.")
-	} else {
-		runtimeFds = append(runtimeFds, fd)
-	}
-
-	for _, c := range consoles {
-		fd, err := unix.Open(c.path, os.O_WRONLY, 0)
-		if err == nil {
-			runtimeFds = append(runtimeFds, fd)
-			l.Infof("Panic console: %s", c.path)
-		}
-	}
-
-	// This could cause a data race if the runtime crashed while we're
-	// initializing the crash handler, but there is no locking infrastructure
-	// for this so we have to take that risk.
-	overrideWrite = runtimeWrite
-}
diff --git a/metropolis/node/core/pstore.go b/metropolis/node/core/pstore.go
deleted file mode 100644
index dfa30f0..0000000
--- a/metropolis/node/core/pstore.go
+++ /dev/null
@@ -1,43 +0,0 @@
-package main
-
-import (
-	"context"
-
-	"source.monogon.dev/osbase/pstore"
-	"source.monogon.dev/osbase/supervisor"
-)
-
-// dumpAndCleanPstore dumps all files accumulated in the pstore into the log
-// and clears them from the pstore. This allows looking at these logs and also
-// keeps the pstore from overflowing the generally limited storage it has.
-func dumpAndCleanPstore(ctx context.Context) error {
-	logger := supervisor.Logger(ctx)
-	dumps, err := pstore.GetKmsgDumps()
-	if err != nil {
-		logger.Errorf("Failed to recover logs from pstore: %v", err)
-		return nil
-	}
-	for _, dump := range dumps {
-		logger.Errorf("Recovered log from %v at %v. Reconstructed log follows.", dump.Reason, dump.OccurredAt)
-		for _, line := range dump.Lines {
-			logger.Warning(line)
-		}
-	}
-	userspaceLines, err := pstore.GetPmsgDump()
-	if err != nil {
-		logger.Errorf("Failed to recover userspace logs from pstore: %v", err)
-	}
-	for _, line := range userspaceLines {
-		logger.Warning(line)
-	}
-	cleanErr := pstore.ClearAll()
-	if cleanErr != nil {
-		logger.Errorf("Failed to clear pstore: %v", err)
-	}
-	// Retrying this is extremely unlikely to result in any change and is most
-	// likely just going to generate large amounts of useless logs obscuring
-	// errors.
-	supervisor.Signal(ctx, supervisor.SignalHealthy)
-	supervisor.Signal(ctx, supervisor.SignalDone)
-	return nil
-}