Switch Metropolis to EROFS-based root filesystem

This gets rid of the old large initramfs and switches to an EROFS-based root
filesystem. It also drops the copy & remount compatibility code. As this filesystem is
properly read-only and not just ephemeral, this also brings various changes to the code
to make systems compatible with that.

Test Plan: Covered by E2E tests, also manually smoke-tested.

X-Origin-Diff: phab/D696
GitOrigin-RevId: 037f2b8253e7cff8435cc79771fad05f53670ff0
diff --git a/metropolis/node/BUILD.bazel b/metropolis/node/BUILD.bazel
index 5c0934f..9d6957e 100644
--- a/metropolis/node/BUILD.bazel
+++ b/metropolis/node/BUILD.bazel
@@ -1,5 +1,5 @@
 load("@io_bazel_rules_go//go:def.bzl", "go_library")
-load("//metropolis/node/build:def.bzl", "node_initramfs")
+load("//metropolis/node/build:def.bzl", "erofs_image")
 
 go_library(
     name = "go_default_library",
@@ -16,18 +16,29 @@
     },
 )
 
-node_initramfs(
-    name = "initramfs",
+erofs_image(
+    name = "rootfs",
     extra_dirs = [
         "/kubernetes/conf/flexvolume-plugins",
         "/containerd/plugins",
+        "/sys",
+        "/proc",
+        "/dev",
+        "/esp",
+        "/tmp",
+        "/run",
+        "/ephemeral",
+        "/data",
     ],
     files = {
         "//metropolis/node/core": "/init",
         "//third_party/xfsprogs:mkfs.xfs": "/bin/mkfs.xfs",
 
-        # CA Certificate bundle & os-release
+        # CA Certificate bundle & os-release & resolv.conf
+        # These should not be explicitly used by Metropolis code and are only here for compatibility with
+        # paths hardcoded by standard libraries (like Go's).
         "@cacerts//file": "/etc/ssl/cert.pem",
+        "//metropolis/node/core/network/dns:resolv.conf": "/etc/resolv.conf",
         ":os-release-info": "/etc/os-release",
 
         # Hyperkube
@@ -67,13 +78,17 @@
         # runc runtime, with cgo
         "@com_github_opencontainers_runc//:runc": "/containerd/bin/runc",
     },
+    symlinks = {
+        "/ephemeral/machine-id": "/etc/machine-id",
+        "/ephemeral/hosts": "/etc/hosts",
+    },
 )
 
 genrule(
     name = "image",
     srcs = [
         "//third_party/linux:bzImage",
-        ":initramfs",
+        ":rootfs",
     ],
     outs = [
         "node.img",
@@ -81,7 +96,7 @@
     cmd = """
     $(location //metropolis/node/build/mkimage) \
         -efi $(location //third_party/linux:bzImage) \
-        -initramfs $(location :initramfs) \
+        -system $(location :rootfs) \
         -out $@
     """,
     tools = [
diff --git a/metropolis/node/build/mkimage/main.go b/metropolis/node/build/mkimage/main.go
index 94b2ecb..c2f7d5a 100644
--- a/metropolis/node/build/mkimage/main.go
+++ b/metropolis/node/build/mkimage/main.go
@@ -23,6 +23,7 @@
 import (
 	"flag"
 	"fmt"
+	"io"
 	"io/ioutil"
 	"log"
 	"os"
@@ -34,27 +35,42 @@
 )
 
 var NodeDataPartition gpt.Type = gpt.Type("9eeec464-6885-414a-b278-4305c51f7966")
+var NodeSystemPartition gpt.Type = gpt.Type("ee96055b-f6d0-4267-8bbb-724b2afea74c")
 
 var (
 	flagEFI                  string
 	flagOut                  string
-	flagInitramfs            string
+	flagSystemPath           string
 	flagEnrolmentCredentials string
 	flagDataPartitionSize    uint64
 	flagESPPartitionSize     uint64
+	flagSystemPartitionSize  uint64
 )
 
 func mibToSectors(size uint64) uint64 {
 	return (size * 1024 * 1024) / 512
 }
 
+type devZeroReader struct{}
+
+func (_ devZeroReader) Read(b []byte) (n int, err error) {
+	for i := range b {
+		b[i] = 0
+	}
+	return len(b), nil
+}
+
+// devZero is a /dev/zero-like reader which reads an infinite number of zeroes
+var devZero = devZeroReader{}
+
 func main() {
 	flag.StringVar(&flagEFI, "efi", "", "UEFI payload")
 	flag.StringVar(&flagOut, "out", "", "Output disk image")
-	flag.StringVar(&flagInitramfs, "initramfs", "", "External initramfs [optional]")
+	flag.StringVar(&flagSystemPath, "system", "", "System partition [optional]")
 	flag.StringVar(&flagEnrolmentCredentials, "enrolment_credentials", "", "Enrolment credentials [optional]")
 	flag.Uint64Var(&flagDataPartitionSize, "data_partition_size", 2048, "Override the data partition size (default 2048 MiB)")
-	flag.Uint64Var(&flagESPPartitionSize, "esp_partition_size", 512, "Override the ESP partition size (default: 512MiB)")
+	flag.Uint64Var(&flagESPPartitionSize, "esp_partition_size", 128, "Override the ESP partition size (default: 128MiB)")
+	flag.Uint64Var(&flagSystemPartitionSize, "system_partition_size", 1024, "Override the System partition size (default: 1024MiB)")
 	flag.Parse()
 
 	if flagEFI == "" || flagOut == "" {
@@ -62,7 +78,7 @@
 	}
 
 	_ = os.Remove(flagOut)
-	diskImg, err := diskfs.Create(flagOut, 3*1024*1024*1024, diskfs.Raw)
+	diskImg, err := diskfs.Create(flagOut, 4*1024*1024*1024, diskfs.Raw)
 	if err != nil {
 		log.Fatalf("diskfs.Create(%q): %v", flagOut, err)
 	}
@@ -80,10 +96,16 @@
 				End:   mibToSectors(flagESPPartitionSize) - 1,
 			},
 			{
+				Type:  NodeSystemPartition,
+				Name:  "METROPOLIS-SYSTEM",
+				Start: mibToSectors(flagESPPartitionSize),
+				End:   mibToSectors(flagESPPartitionSize+flagSystemPartitionSize) - 1,
+			},
+			{
 				Type:  NodeDataPartition,
 				Name:  "METROPOLIS-NODE-DATA",
-				Start: mibToSectors(flagESPPartitionSize),
-				End:   mibToSectors(flagESPPartitionSize+flagDataPartitionSize) - 1,
+				Start: mibToSectors(flagESPPartitionSize + flagSystemPartitionSize),
+				End:   mibToSectors(flagESPPartitionSize+flagSystemPartitionSize+flagDataPartitionSize) - 1,
 			},
 		},
 	}
@@ -91,6 +113,23 @@
 		log.Fatalf("Failed to apply partition table: %v", err)
 	}
 
+	if flagSystemPath != "" {
+		systemPart, err := os.Open(flagSystemPath)
+		if err != nil {
+			log.Fatalf("Failed to open system partition: %v", err)
+		}
+		defer systemPart.Close()
+		systemPartMeta, err := systemPart.Stat()
+		if err != nil {
+			log.Fatalf("Failed to stat system partition: %v", err)
+		}
+		padding := int64(flagSystemPartitionSize*1024*1024) - systemPartMeta.Size()
+		systemPartMulti := io.MultiReader(systemPart, io.LimitReader(devZero, padding))
+		if _, err := diskImg.WritePartitionContents(2, systemPartMulti); err != nil {
+			log.Fatalf("Failed to write system partition: %v", err)
+		}
+	}
+
 	fs, err := diskImg.CreateFilesystem(disk.FilesystemSpec{Partition: 1, FSType: filesystem.TypeFat32, VolumeLabel: "ESP"})
 	if err != nil {
 		log.Fatalf("Failed to create filesystem: %v", err)
@@ -105,10 +144,6 @@
 
 	put(fs, flagEFI, "/EFI/BOOT/BOOTX64.EFI")
 
-	if flagInitramfs != "" {
-		put(fs, flagInitramfs, "/EFI/metropolis/initramfs.cpio.lz4")
-	}
-
 	if flagEnrolmentCredentials != "" {
 		put(fs, flagEnrolmentCredentials, "/EFI/metropolis/enrolment.pb")
 	}
diff --git a/metropolis/node/core/BUILD.bazel b/metropolis/node/core/BUILD.bazel
index 95344e8..28f4d2a 100644
--- a/metropolis/node/core/BUILD.bazel
+++ b/metropolis/node/core/BUILD.bazel
@@ -6,7 +6,7 @@
     srcs = [
         "debug_service.go",
         "main.go",
-        "switchroot.go",
+        "mounts.go",
     ] + select({
         "//metropolis/node:debug_build": ["delve_enabled.go"],
         "//conditions:default": ["delve_disabled.go"],
diff --git a/metropolis/node/core/cluster/node.go b/metropolis/node/core/cluster/node.go
index 92ff072..7d26213 100644
--- a/metropolis/node/core/cluster/node.go
+++ b/metropolis/node/core/cluster/node.go
@@ -205,7 +205,7 @@
 
 // ConfigureLocalHostname uses the node's ID as a hostname, and sets the current hostname, and local files like hosts
 // and machine-id accordingly.
-func (n *Node) ConfigureLocalHostname(etc *localstorage.EtcDirectory) error {
+func (n *Node) ConfigureLocalHostname(etc *localstorage.EphemeralDirectory) error {
 	if err := unix.Sethostname([]byte(n.ID())); err != nil {
 		return fmt.Errorf("failed to set runtime hostname: %w", err)
 	}
diff --git a/metropolis/node/core/localstorage/crypt/blockdev.go b/metropolis/node/core/localstorage/crypt/blockdev.go
index 86d5381..7f874b7 100644
--- a/metropolis/node/core/localstorage/crypt/blockdev.go
+++ b/metropolis/node/core/localstorage/crypt/blockdev.go
@@ -79,12 +79,14 @@
 			}
 			for partNumber, part := range table.Partitions {
 				if part.Type == EFIPartitionType {
-					if err := unix.Mknod(ESPDevicePath, 0600|unix.S_IFBLK, int(unix.Mkdev(uint32(majorDev), uint32(partNumber+1)))); err != nil {
+					err := unix.Mknod(ESPDevicePath, 0600|unix.S_IFBLK, int(unix.Mkdev(uint32(majorDev), uint32(partNumber+1))))
+					if err != nil && !os.IsExist(err) {
 						return fmt.Errorf("failed to create device node for ESP partition: %w", err)
 					}
 				}
 				if part.Type == NodeDataPartitionType {
-					if err := unix.Mknod(NodeDataCryptPath, 0600|unix.S_IFBLK, int(unix.Mkdev(uint32(majorDev), uint32(partNumber+1)))); err != nil {
+					err := unix.Mknod(NodeDataCryptPath, 0600|unix.S_IFBLK, int(unix.Mkdev(uint32(majorDev), uint32(partNumber+1))))
+					if err != nil && !os.IsExist(err) {
 						return fmt.Errorf("failed to create device node for Metropolis node encrypted data partition: %w", err)
 					}
 				}
diff --git a/metropolis/node/core/localstorage/directory_data.go b/metropolis/node/core/localstorage/directory_data.go
index 9be4fc1..01e2e07 100644
--- a/metropolis/node/core/localstorage/directory_data.go
+++ b/metropolis/node/core/localstorage/directory_data.go
@@ -18,7 +18,6 @@
 
 import (
 	"fmt"
-	"os"
 	"os/exec"
 
 	"golang.org/x/sys/unix"
@@ -139,10 +138,6 @@
 }
 
 func (d *DataDirectory) mount() error {
-	if err := os.Mkdir(d.FullPath(), 0755); err != nil {
-		return fmt.Errorf("making data directory: %w", err)
-	}
-
 	if err := unix.Mount("/dev/data", d.FullPath(), "xfs", unix.MS_NOEXEC|unix.MS_NODEV, "pquota"); err != nil {
 		return fmt.Errorf("mounting data directory: %w", err)
 	}
diff --git a/metropolis/node/core/localstorage/directory_root.go b/metropolis/node/core/localstorage/directory_root.go
index ac1a453..c3a49cb 100644
--- a/metropolis/node/core/localstorage/directory_root.go
+++ b/metropolis/node/core/localstorage/directory_root.go
@@ -39,28 +39,26 @@
 		return fmt.Errorf("MakeBlockDevices: %w", err)
 	}
 
-	if err := os.Mkdir(r.ESP.FullPath(), 0755); err != nil {
-		return fmt.Errorf("making ESP directory: %w", err)
-	}
-
 	if err := unix.Mount(crypt.ESPDevicePath, r.ESP.FullPath(), "vfat", unix.MS_NOEXEC|unix.MS_NODEV|unix.MS_SYNC, ""); err != nil {
 		return fmt.Errorf("mounting ESP partition: %w", err)
 	}
 
 	r.Data.canMount = true
 
-	if err := os.Mkdir(r.Tmp.FullPath(), 0777); err != nil {
-		return fmt.Errorf("making /tmp directory: %w", err)
-	}
-
 	if err := unix.Mount("tmpfs", r.Tmp.FullPath(), "tmpfs", unix.MS_NOEXEC|unix.MS_NODEV, ""); err != nil {
 		return fmt.Errorf("mounting /tmp: %w", err)
 	}
 
+	if err := unix.Mount("tmpfs", r.Ephemeral.FullPath(), "tmpfs", unix.MS_NODEV, ""); err != nil {
+		return fmt.Errorf("mounting /ephemeral: %v", err)
+	}
+
+	if err := unix.Mount("tmpfs", r.Run.FullPath(), "tmpfs", unix.MS_NOEXEC|unix.MS_NODEV, ""); err != nil {
+		return fmt.Errorf("mounting /run: %w", err)
+	}
+
 	// TODO(q3k): do this automatically?
 	for _, d := range []declarative.DirectoryPlacement{
-		r.Etc,
-		r.Ephemeral,
 		r.Ephemeral.Consensus,
 		r.Ephemeral.Containerd, r.Ephemeral.Containerd.Tmp, r.Ephemeral.Containerd.RunSC, r.Ephemeral.Containerd.IPAM,
 		r.Ephemeral.FlexvolumePlugins,
diff --git a/metropolis/node/core/localstorage/storage.go b/metropolis/node/core/localstorage/storage.go
index 110513c..73d33e1 100644
--- a/metropolis/node/core/localstorage/storage.go
+++ b/metropolis/node/core/localstorage/storage.go
@@ -142,8 +142,8 @@
 
 type EtcDirectory struct {
 	declarative.Directory
-	Hosts     declarative.File `file:"hosts"`
-	MachineID declarative.File `file:"machine-id"`
+	Hosts     declarative.File `file:"hosts"`      // Symlinked to /ephemeral/hosts, baked into the erofs system image
+	MachineID declarative.File `file:"machine-id"` // Symlinked to /ephemeral/machine-id, baked into the erofs system image
 }
 
 type EphemeralDirectory struct {
@@ -151,6 +151,8 @@
 	Consensus         EphemeralConsensusDirectory  `dir:"consensus"`
 	Containerd        EphemeralContainerdDirectory `dir:"containerd"`
 	FlexvolumePlugins declarative.Directory        `dir:"flexvolume_plugins"`
+	Hosts             declarative.File             `file:"hosts"`
+	MachineID         declarative.File             `file:"machine-id"`
 }
 
 type EphemeralConsensusDirectory struct {
diff --git a/metropolis/node/core/main.go b/metropolis/node/core/main.go
index e72c8b5..3cf75ce 100644
--- a/metropolis/node/core/main.go
+++ b/metropolis/node/core/main.go
@@ -92,10 +92,10 @@
 	// Initial logger. Used until we get to a supervisor.
 	logger := lt.MustLeveledFor("init")
 
-	// Remount onto a tmpfs and re-exec if needed. Otherwise, keep running.
-	err = switchRoot(logger)
+	// Set up basic mounts
+	err = setupMounts(logger)
 	if err != nil {
-		panic(fmt.Errorf("could not remount root: %w", err))
+		panic(fmt.Errorf("could not set up basic mounts: %w", err))
 	}
 
 	// Linux kernel default is 4096 which is far too low. Raise it to 1M which is what gVisor suggests.
@@ -170,7 +170,7 @@
 		// we should be running.
 
 		node := m.Node()
-		if err := node.ConfigureLocalHostname(&root.Etc); err != nil {
+		if err := node.ConfigureLocalHostname(&root.Ephemeral); err != nil {
 			close(trapdoor)
 			return fmt.Errorf("failed to set local hostname: %w", err)
 		}
diff --git a/metropolis/node/core/mounts.go b/metropolis/node/core/mounts.go
new file mode 100644
index 0000000..c6a626f
--- /dev/null
+++ b/metropolis/node/core/mounts.go
@@ -0,0 +1,86 @@
+// Copyright 2020 The Monogon Project Authors.
+//
+// SPDX-License-Identifier: Apache-2.0
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package main
+
+import (
+	"fmt"
+	"io/ioutil"
+	"os"
+	"strings"
+
+	"golang.org/x/sys/unix"
+
+	"source.monogon.dev/metropolis/pkg/logtree"
+)
+
+// setupMounts sets up basic mounts like sysfs, procfs, devtmpfs and cgroups. This should be called early during init
+// as a lot of processes depend on this being available.
+func setupMounts(log logtree.LeveledLogger) error {
+	// Set up target filesystems.
+	for _, el := range []struct {
+		dir   string
+		fs    string
+		flags uintptr
+	}{
+		{"/sys", "sysfs", unix.MS_NOEXEC | unix.MS_NOSUID | unix.MS_NODEV},
+		{"/proc", "proc", unix.MS_NOEXEC | unix.MS_NOSUID | unix.MS_NODEV},
+		{"/dev", "devtmpfs", unix.MS_NOEXEC | unix.MS_NOSUID},
+		{"/dev/pts", "devpts", unix.MS_NOEXEC | unix.MS_NOSUID},
+	} {
+		if err := os.MkdirAll(el.dir, 0755); err != nil {
+			return fmt.Errorf("could not make %s: %w", el.dir, err)
+		}
+		if err := unix.Mount(el.fs, el.dir, el.fs, el.flags, ""); err != nil {
+			return fmt.Errorf("could not mount %s on %s: %w", el.fs, el.dir, err)
+		}
+	}
+
+	// Mount all available CGroups for v1 (v2 uses a single unified hierarchy and is not supported by our runtimes yet)
+	if err := unix.Mount("tmpfs", "/sys/fs/cgroup", "tmpfs", unix.MS_NOEXEC|unix.MS_NOSUID|unix.MS_NODEV, ""); err != nil {
+		panic(err)
+	}
+	cgroupsRaw, err := ioutil.ReadFile("/proc/cgroups")
+	if err != nil {
+		panic(err)
+	}
+
+	cgroupLines := strings.Split(string(cgroupsRaw), "\n")
+	for _, cgroupLine := range cgroupLines {
+		if cgroupLine == "" || strings.HasPrefix(cgroupLine, "#") {
+			continue
+		}
+		cgroupParts := strings.Split(cgroupLine, "\t")
+		cgroupName := cgroupParts[0]
+		if err := os.Mkdir("/sys/fs/cgroup/"+cgroupName, 0755); err != nil {
+			panic(err)
+		}
+		if err := unix.Mount("cgroup", "/sys/fs/cgroup/"+cgroupName, "cgroup", unix.MS_NOEXEC|unix.MS_NOSUID|unix.MS_NODEV, cgroupName); err != nil {
+			panic(err)
+		}
+	}
+
+	// Enable hierarchical memory accounting
+	useMemoryHierarchy, err := os.OpenFile("/sys/fs/cgroup/memory/memory.use_hierarchy", os.O_RDWR, 0)
+	if err != nil {
+		panic(err)
+	}
+	if _, err := useMemoryHierarchy.WriteString("1"); err != nil {
+		panic(err)
+	}
+	useMemoryHierarchy.Close()
+	return nil
+}
diff --git a/metropolis/node/core/network/BUILD.bazel b/metropolis/node/core/network/BUILD.bazel
index e7ca16e..d24e07c 100644
--- a/metropolis/node/core/network/BUILD.bazel
+++ b/metropolis/node/core/network/BUILD.bazel
@@ -15,6 +15,5 @@
         "@com_github_google_nftables//expr:go_default_library",
         "@com_github_insomniacslk_dhcp//dhcpv4:go_default_library",
         "@com_github_vishvananda_netlink//:go_default_library",
-        "@org_golang_x_sys//unix:go_default_library",
     ],
 )
diff --git a/metropolis/node/core/network/dns/BUILD.bazel b/metropolis/node/core/network/dns/BUILD.bazel
index 4579d54..1ef31f1 100644
--- a/metropolis/node/core/network/dns/BUILD.bazel
+++ b/metropolis/node/core/network/dns/BUILD.bazel
@@ -13,3 +13,5 @@
         "//metropolis/pkg/supervisor:go_default_library",
     ],
 )
+
+exports_files(["resolv.conf"])
diff --git a/metropolis/node/core/network/dns/resolv.conf b/metropolis/node/core/network/dns/resolv.conf
new file mode 100644
index 0000000..eed7161
--- /dev/null
+++ b/metropolis/node/core/network/dns/resolv.conf
@@ -0,0 +1,2 @@
+# NOTE: This is baked into the rootfs. All DNS-related settings are in CoreDNS at //metropolis/node/core/network/dns.
+nameserver 127.0.0.1
\ No newline at end of file
diff --git a/metropolis/node/core/network/main.go b/metropolis/node/core/network/main.go
index 94bca0a..25afe8e 100644
--- a/metropolis/node/core/network/main.go
+++ b/metropolis/node/core/network/main.go
@@ -22,7 +22,6 @@
 	"fmt"
 	"io/ioutil"
 	"net"
-	"os"
 	"sync"
 	"time"
 
@@ -30,7 +29,6 @@
 	"github.com/google/nftables/expr"
 	"github.com/insomniacslk/dhcp/dhcpv4"
 	"github.com/vishvananda/netlink"
-	"golang.org/x/sys/unix"
 
 	"source.monogon.dev/metropolis/node/core/network/dhcp4c"
 	dhcpcb "source.monogon.dev/metropolis/node/core/network/dhcp4c/callback"
@@ -70,29 +68,6 @@
 	}
 }
 
-func setResolvconf(nameservers []net.IP, searchDomains []string) error {
-	_ = os.Mkdir("/etc", 0755)
-	newResolvConf, err := os.Create(resolvConfSwapPath)
-	if err != nil {
-		return err
-	}
-	defer newResolvConf.Close()
-	defer os.Remove(resolvConfSwapPath)
-	for _, ns := range nameservers {
-		if _, err := newResolvConf.WriteString(fmt.Sprintf("nameserver %v\n", ns)); err != nil {
-			return err
-		}
-	}
-	for _, searchDomain := range searchDomains {
-		if _, err := newResolvConf.WriteString(fmt.Sprintf("search %v", searchDomain)); err != nil {
-			return err
-		}
-	}
-	newResolvConf.Close()
-	// Atomically swap in new config
-	return unix.Rename(resolvConfSwapPath, resolvConfPath)
-}
-
 // nfifname converts an interface name into 16 bytes padded with zeroes (for nftables)
 func nfifname(n string) []byte {
 	b := make([]byte, 16)
@@ -208,11 +183,6 @@
 		logger.Fatalf("Failed to enable IPv4 forwarding: %v", err)
 	}
 
-	// We're handling all DNS requests with CoreDNS, including local ones
-	if err := setResolvconf([]net.IP{{127, 0, 0, 1}}, []string{}); err != nil {
-		logger.Fatalf("Failed to set resolv.conf: %v", err)
-	}
-
 	supervisor.Signal(ctx, supervisor.SignalHealthy)
 	supervisor.Signal(ctx, supervisor.SignalDone)
 	return nil
diff --git a/metropolis/node/core/switchroot.go b/metropolis/node/core/switchroot.go
deleted file mode 100644
index 7391e75..0000000
--- a/metropolis/node/core/switchroot.go
+++ /dev/null
@@ -1,213 +0,0 @@
-// Copyright 2020 The Monogon Project Authors.
-//
-// SPDX-License-Identifier: Apache-2.0
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package main
-
-import (
-	"fmt"
-	"io"
-	"io/ioutil"
-	"os"
-	"path/filepath"
-	"strings"
-	"syscall"
-
-	"golang.org/x/sys/unix"
-
-	"source.monogon.dev/metropolis/pkg/logtree"
-)
-
-// switchRoot moves the root from initramfs into a tmpfs
-// This is necessary because you cannot pivot_root from a initramfs (and runsc wants to do that).
-// In the future, we should instead use something like squashfs instead of an initramfs and just nuke this.
-func switchRoot(log logtree.LeveledLogger) error {
-	// We detect the need to remount to tmpfs over env vars.
-	// The first run of /init (from initramfs) will not have this var, and will be re-exec'd from a new tmpfs root with
-	// that variable set.
-	witness := "METROPOLIS_REMOUNTED"
-
-	// If the witness env var is found in the environment, it means we are ready to go.
-	environ := os.Environ()
-	for _, env := range environ {
-		if strings.HasPrefix(env, witness+"=") {
-			log.Info("Metropolis node running in tmpfs root")
-			return nil
-		}
-	}
-
-	// Otherwise, we need to remount to a tmpfs.
-	environ = append(environ, witness+"=yes")
-	log.Info("Metropolis node running in initramfs, remounting to tmpfs...")
-
-	// Make note of all directories we have to make and files that we have to copy.
-	paths := []string{}
-	dirs := []string{}
-	err := filepath.Walk("/", func(path string, info os.FileInfo, err error) error {
-		if err != nil {
-			return err
-		}
-		if path == "/" {
-			return nil
-		}
-		// /dev is prepopulated by the initramfs, skip that. The target root uses devtmpfs.
-		if path == "/dev" || strings.HasPrefix(path, "/dev/") {
-			return nil
-		}
-
-		if info.IsDir() {
-			dirs = append(dirs, path)
-		} else {
-			paths = append(paths, path)
-		}
-
-		return nil
-	})
-	if err != nil {
-		return fmt.Errorf("could not list root files: %w", err)
-	}
-
-	log.Info("Copying paths to tmpfs:")
-	for _, p := range paths {
-		log.Infof(" - %s", p)
-	}
-
-	// Make new root at /mnt
-	if err := os.Mkdir("/mnt", 0755); err != nil {
-		return fmt.Errorf("could not make /mnt: %w", err)
-	}
-	// And mount a tmpfs on it
-	if err := unix.Mount("tmpfs", "/mnt", "tmpfs", 0, ""); err != nil {
-		return fmt.Errorf("could not mount tmpfs on /mnt: %w", err)
-	}
-
-	// Make all directories. Since filepath.Walk is lexicographically ordered, we don't need to ensure that the parent
-	// exists.
-	for _, src := range dirs {
-		stat, err := os.Stat(src)
-		if err != nil {
-			return fmt.Errorf("Stat(%q): %w", src, err)
-		}
-		dst := "/mnt" + src
-		err = os.Mkdir(dst, stat.Mode())
-		if err != nil {
-			return fmt.Errorf("Mkdir(%q): %w", dst, err)
-		}
-	}
-
-	// Move all files over. Parent directories will exist by now.
-	for _, src := range paths {
-		stat, err := os.Stat(src)
-		if err != nil {
-			return fmt.Errorf("Stat(%q): %w", src, err)
-		}
-		dst := "/mnt" + src
-
-		// Copy file.
-		sfd, err := os.Open(src)
-		if err != nil {
-			return fmt.Errorf("Open(%q): %w", src, err)
-		}
-		dfd, err := os.OpenFile(dst, os.O_WRONLY|os.O_CREATE, stat.Mode())
-		if err != nil {
-			sfd.Close()
-			return fmt.Errorf("OpenFile(%q): %w", dst, err)
-		}
-		_, err = io.Copy(dfd, sfd)
-
-		sfd.Close()
-		dfd.Close()
-		if err != nil {
-			return fmt.Errorf("Copying %q failed: %w", src, err)
-		}
-
-		// Remove the old file.
-		err = unix.Unlink(src)
-		if err != nil {
-			return fmt.Errorf("Unlink(%q): %w", src, err)
-		}
-	}
-
-	// Set up target filesystems.
-	for _, el := range []struct {
-		dir   string
-		fs    string
-		flags uintptr
-	}{
-		{"/sys", "sysfs", unix.MS_NOEXEC | unix.MS_NOSUID | unix.MS_NODEV},
-		{"/proc", "proc", unix.MS_NOEXEC | unix.MS_NOSUID | unix.MS_NODEV},
-		{"/dev", "devtmpfs", unix.MS_NOEXEC | unix.MS_NOSUID},
-		{"/dev/pts", "devpts", unix.MS_NOEXEC | unix.MS_NOSUID},
-	} {
-		if err := os.Mkdir("/mnt"+el.dir, 0755); err != nil {
-			return fmt.Errorf("could not make /mnt%s: %w", el.dir, err)
-		}
-		if err := unix.Mount(el.fs, "/mnt"+el.dir, el.fs, el.flags, ""); err != nil {
-			return fmt.Errorf("could not mount %s on /mnt%s: %w", el.fs, el.dir, err)
-		}
-	}
-
-	// Mount all available CGroups for v1 (v2 uses a single unified hierarchy and is not supported by our runtimes yet)
-	if unix.Mount("tmpfs", "/mnt/sys/fs/cgroup", "tmpfs", unix.MS_NOEXEC|unix.MS_NOSUID|unix.MS_NODEV, ""); err != nil {
-		panic(err)
-	}
-	cgroupsRaw, err := ioutil.ReadFile("/mnt/proc/cgroups")
-	if err != nil {
-		panic(err)
-	}
-
-	cgroupLines := strings.Split(string(cgroupsRaw), "\n")
-	for _, cgroupLine := range cgroupLines {
-		if cgroupLine == "" || strings.HasPrefix(cgroupLine, "#") {
-			continue
-		}
-		cgroupParts := strings.Split(cgroupLine, "\t")
-		cgroupName := cgroupParts[0]
-		if err := os.Mkdir("/mnt/sys/fs/cgroup/"+cgroupName, 0755); err != nil {
-			panic(err)
-		}
-		if err := unix.Mount("cgroup", "/mnt/sys/fs/cgroup/"+cgroupName, "cgroup", unix.MS_NOEXEC|unix.MS_NOSUID|unix.MS_NODEV, cgroupName); err != nil {
-			panic(err)
-		}
-	}
-
-	// Enable hierarchical memory accounting
-	useMemoryHierarchy, err := os.OpenFile("/mnt/sys/fs/cgroup/memory/memory.use_hierarchy", os.O_RDWR, 0)
-	if err != nil {
-		panic(err)
-	}
-	if _, err := useMemoryHierarchy.WriteString("1"); err != nil {
-		panic(err)
-	}
-	useMemoryHierarchy.Close()
-
-	// Chroot to new root.
-	// This is adapted from util-linux's switch_root.
-	err = os.Chdir("/mnt")
-	if err != nil {
-		return fmt.Errorf("could not chdir to /mnt: %w", err)
-	}
-	err = syscall.Mount("/mnt", "/", "", syscall.MS_MOVE, "")
-	if err != nil {
-		return fmt.Errorf("could not remount /mnt to /: %w", err)
-	}
-	err = syscall.Chroot(".")
-	if err != nil {
-		return fmt.Errorf("could not chroot to new root: %w", err)
-	}
-
-	// Re-exec into new init with new environment
-	return unix.Exec("/init", os.Args, environ)
-}
diff --git a/third_party/linux/linux-metropolis.config b/third_party/linux/linux-metropolis.config
index b2f7e7a..75f60ea 100644
--- a/third_party/linux/linux-metropolis.config
+++ b/third_party/linux/linux-metropolis.config
@@ -434,7 +434,7 @@
 # CONFIG_LEGACY_VSYSCALL_XONLY is not set
 CONFIG_LEGACY_VSYSCALL_NONE=y
 CONFIG_CMDLINE_BOOL=y
-CONFIG_CMDLINE="console=ttyS0 initrd=\\EFI\\metropolis\\initramfs.cpio.lz4"
+CONFIG_CMDLINE="console=ttyS0 root=PARTLABEL=METROPOLIS-SYSTEM rootfstype=erofs init=/init"
 CONFIG_CMDLINE_OVERRIDE=y
 CONFIG_MODIFY_LDT_SYSCALL=y
 CONFIG_HAVE_LIVEPATCH=y