m/test: refactor cluster launch code, use for e2e tests

This is a light dust-off pass for the existing cluster launch code.
Notably, we separate Metropolis-specific code into a subpackage
(allowing us to make the package itself depend on the required
node/kernel images, without introducing dependency loops or unnecessary
dependencies on the Metropolis node image).

We also make the LaunchCluster code return an already authenticated
Management client, and subsequent changes will use this client to add
more nodes to the running cluster.

We then move the E2E test to use LaunchCluster instead of LaunchNode, in
preparation for running a multi-node cluster in the E2E test.

We also add some more log calls and clean up the existing ones to make
it clear which subsystem (launch, launch/cluster or e2e) is respondible
for each message.

Change-Id: I838bdc75073831fe94b9cdcef4fb3ab6bf8cba2c
Reviewed-on: https://review.monogon.dev/c/monogon/+/343
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/ports.go b/metropolis/node/ports.go
index ed1c323..3449f07 100644
--- a/metropolis/node/ports.go
+++ b/metropolis/node/ports.go
@@ -17,12 +17,11 @@
 package node
 
 const (
-	CuratorServicePort  = 7835
-	ConsensusPort       = 7834
-	MasterServicePort   = 7833
-	ExternalServicePort = 7836
-	DebugServicePort    = 7837
-	WireGuardPort       = 7838
-	KubernetesAPIPort   = 6443
-	DebuggerPort        = 2345
+	CuratorServicePort = 7835
+	ConsensusPort      = 7834
+	MasterServicePort  = 7833
+	DebugServicePort   = 7837
+	WireGuardPort      = 7838
+	KubernetesAPIPort  = 6443
+	DebuggerPort       = 2345
 )
diff --git a/metropolis/test/e2e/BUILD.bazel b/metropolis/test/e2e/BUILD.bazel
index 53c0e38..7f8571f 100644
--- a/metropolis/test/e2e/BUILD.bazel
+++ b/metropolis/test/e2e/BUILD.bazel
@@ -33,13 +33,11 @@
     rundir = ".",
     deps = [
         "//metropolis/node:go_default_library",
-        "//metropolis/node/core/rpc:go_default_library",
         "//metropolis/proto/api:go_default_library",
-        "//metropolis/test/launch:go_default_library",
+        "//metropolis/test/launch/cluster:go_default_library",
         "@io_k8s_api//core/v1:go_default_library",
         "@io_k8s_apimachinery//pkg/api/resource:go_default_library",
         "@io_k8s_apimachinery//pkg/apis/meta/v1:go_default_library",
         "@io_k8s_kubernetes//pkg/api/v1/pod:go_default_library",
-        "@org_golang_google_grpc//:go_default_library",
     ],
 )
diff --git a/metropolis/test/e2e/k8s_cts/BUILD.bazel b/metropolis/test/e2e/k8s_cts/BUILD.bazel
index e6e23de..2d8ac79 100644
--- a/metropolis/test/e2e/k8s_cts/BUILD.bazel
+++ b/metropolis/test/e2e/k8s_cts/BUILD.bazel
@@ -33,7 +33,7 @@
     deps = [
         "//metropolis/node:go_default_library",
         "//metropolis/test/e2e:go_default_library",
-        "//metropolis/test/launch:go_default_library",
+        "//metropolis/test/launch/cluster:go_default_library",
         "@io_k8s_api//core/v1:go_default_library",
         "@io_k8s_api//rbac/v1:go_default_library",
         "@io_k8s_apimachinery//pkg/apis/meta/v1:go_default_library",
@@ -42,14 +42,6 @@
 
 go_binary(
     name = "k8s_cts",
-    data = [
-        "//metropolis/node:image",
-        "//metropolis/node:swtpm_data",
-        "//metropolis/test/ktest:linux-testing",
-        "//metropolis/test/nanoswitch:initramfs",
-        "//third_party/edk2:firmware",
-        "@com_github_bonzini_qboot//:qboot-bin",
-    ],
     embed = [":go_default_library"],
     visibility = ["//visibility:private"],
 )
diff --git a/metropolis/test/e2e/k8s_cts/main.go b/metropolis/test/e2e/k8s_cts/main.go
index 026ca8a..46b99b4 100644
--- a/metropolis/test/e2e/k8s_cts/main.go
+++ b/metropolis/test/e2e/k8s_cts/main.go
@@ -35,7 +35,7 @@
 
 	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/test/e2e"
-	"source.monogon.dev/metropolis/test/launch"
+	"source.monogon.dev/metropolis/test/launch/cluster"
 )
 
 // makeCTSPodSpec generates a spec for a standalone pod running the Kubernetes
@@ -101,12 +101,12 @@
 	}()
 
 	// TODO(q3k): bump up number of nodes after multi-node workflow gets reimplemented.
-	debugClient, portMap, err := launch.LaunchCluster(ctx, launch.ClusterOptions{NumNodes: 1})
+	cl, err := cluster.LaunchCluster(ctx, cluster.ClusterOptions{NumNodes: 1})
 	if err != nil {
 		log.Fatalf("Failed to launch cluster: %v", err)
 	}
 	log.Println("Cluster initialized")
-	clientSet, err := e2e.GetKubeClientSet(ctx, debugClient, portMap[common.KubernetesAPIPort])
+	clientSet, err := e2e.GetKubeClientSet(ctx, cl.Debug, cl.Ports[common.KubernetesAPIPort])
 	if err != nil {
 		log.Fatalf("Failed to get clientSet: %v", err)
 	}
diff --git a/metropolis/test/e2e/main_test.go b/metropolis/test/e2e/main_test.go
index 2838af3..4e1c2c5 100644
--- a/metropolis/test/e2e/main_test.go
+++ b/metropolis/test/e2e/main_test.go
@@ -17,7 +17,6 @@
 package e2e
 
 import (
-	"bytes"
 	"context"
 	"crypto/ed25519"
 	"errors"
@@ -32,16 +31,14 @@
 	"testing"
 	"time"
 
-	"google.golang.org/grpc"
 	corev1 "k8s.io/api/core/v1"
 	"k8s.io/apimachinery/pkg/api/resource"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	podv1 "k8s.io/kubernetes/pkg/api/v1/pod"
 
 	common "source.monogon.dev/metropolis/node"
-	"source.monogon.dev/metropolis/node/core/rpc"
 	apb "source.monogon.dev/metropolis/proto/api"
-	"source.monogon.dev/metropolis/test/launch"
+	"source.monogon.dev/metropolis/test/launch/cluster"
 )
 
 const (
@@ -62,83 +59,49 @@
 // subtest.
 func TestE2E(t *testing.T) {
 	// Run pprof server for debugging
+	addr, err := net.ResolveTCPAddr("tcp", "localhost:0")
+	if err != nil {
+		panic(err)
+	}
+
+	pprofListen, err := net.ListenTCP("tcp", addr)
+	if err != nil {
+		log.Fatalf("Failed to listen on pprof port: %s", pprofListen.Addr())
+	}
+
+	log.Printf("E2E: pprof server listening on %s", pprofListen.Addr())
 	go func() {
-		addr, err := net.ResolveTCPAddr("tcp", "localhost:0")
-		if err != nil {
-			panic(err)
-		}
-
-		l, err := net.ListenTCP("tcp", addr)
-		if err != nil {
-			log.Fatalf("Failed to listen on pprof port: %s", l.Addr())
-		}
-		defer l.Close()
-
-		log.Printf("pprof server listening on %s", l.Addr())
-		log.Printf("pprof server returned an error: %v", http.Serve(l, nil))
+		log.Printf("E2E: pprof server returned an error: %v", http.Serve(pprofListen, nil))
+		pprofListen.Close()
 	}()
 
 	// Set a global timeout to make sure this terminates
 	ctx, cancel := context.WithTimeout(context.Background(), globalTestTimeout)
 	defer cancel()
-	portMap, err := launch.ConflictFreePortMap(launch.NodePorts)
+
+	// Launch cluster.
+	cluster, err := cluster.LaunchCluster(ctx, cluster.ClusterOptions{
+		NumNodes: 1,
+	})
 	if err != nil {
-		t.Fatalf("Failed to acquire ports for e2e test: %v", err)
+		t.Fatalf("LaunchCluster failed: %v", err)
 	}
-
-	procExit := make(chan struct{})
-
-	go func() {
-		if err := launch.Launch(ctx, launch.Options{
-			Ports:      portMap,
-			SerialPort: os.Stdout,
-			NodeParameters: &apb.NodeParameters{
-				Cluster: &apb.NodeParameters_ClusterBootstrap_{
-					ClusterBootstrap: launch.InsecureClusterBootstrap,
-				},
-			},
-		}); err != nil {
-			panic(err)
+	defer func() {
+		err := cluster.Close()
+		if err != nil {
+			t.Fatalf("cluster Close failed: %v", err)
 		}
-		close(procExit)
 	}()
 
-	grpcDebug, err := portMap.DialGRPC(common.DebugServicePort, grpc.WithInsecure())
-	if err != nil {
-		log.Printf("Failed to dial debug service (is it running?): %v", err)
-		return
-	}
-	debug := apb.NewNodeDebugServiceClient(grpcDebug)
+	log.Printf("E2E: Cluster running, starting tests...")
 
 	// This exists to keep the parent around while all the children race.
 	// It currently tests both a set of OS-level conditions and Kubernetes
 	// Deployments and StatefulSets
 	t.Run("RunGroup", func(t *testing.T) {
 		t.Run("Connect to Curator", func(t *testing.T) {
-			testEventual(t, "Retrieving owner credentials succesful", ctx, 60*time.Second, func(ctx context.Context) error {
-				remote := fmt.Sprintf("localhost:%v", portMap[common.CuratorServicePort])
-				initClient, err := rpc.NewEphemeralClient(remote, launch.InsecurePrivateKey, nil)
-				if err != nil {
-					return fmt.Errorf("NewInitialClient: %w", err)
-				}
-
-				aaa := apb.NewAAAClient(initClient)
-				cert, err := rpc.RetrieveOwnerCertificate(ctx, aaa, launch.InsecurePrivateKey)
-				if err != nil {
-					return fmt.Errorf("RetrieveOwnerCertificate: %w", err)
-				}
-
-				if !bytes.Equal(cert.PrivateKey.(ed25519.PrivateKey), launch.InsecurePrivateKey) {
-					t.Fatalf("Received certificate for wrong private key")
-				}
-
-				// Connect to management endpoint and retrieve cluster directory.
-				authClient, err := rpc.NewAuthenticatedClient(remote, *cert, nil)
-				if err != nil {
-					return fmt.Errorf("NewAuthenticatedClient: %w", err)
-				}
-				mgmt := apb.NewManagementClient(authClient)
-				res, err := mgmt.GetClusterInfo(ctx, &apb.GetClusterInfoRequest{})
+			testEventual(t, "Retrieving cluster directory sucessful", ctx, 60*time.Second, func(ctx context.Context) error {
+				res, err := cluster.Management.GetClusterInfo(ctx, &apb.GetClusterInfoRequest{})
 				if err != nil {
 					return fmt.Errorf("GetClusterInfo: %w", err)
 				}
@@ -155,7 +118,7 @@
 				if want, got := 1, len(node.Addresses); want != got {
 					return fmt.Errorf("wanted %d node address, got %d", want, got)
 				}
-				if want, got := "10.42.0.10", node.Addresses[0].Host; want != got {
+				if want, got := "10.1.0.2", node.Addresses[0].Host; want != got {
 					return fmt.Errorf("wanted status address %q, got %q", want, got)
 				}
 
@@ -166,7 +129,7 @@
 			t.Parallel()
 			selfCtx, cancel := context.WithTimeout(ctx, largeTestTimeout)
 			defer cancel()
-			clientSet, err := GetKubeClientSet(selfCtx, debug, portMap[common.KubernetesAPIPort])
+			clientSet, err := GetKubeClientSet(selfCtx, cluster.Debug, cluster.Ports[common.KubernetesAPIPort])
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -365,9 +328,4 @@
 			}
 		})
 	})
-
-	// Cancel the main context and wait for our subprocesses to exit
-	// to avoid leaking them and blocking the parent.
-	cancel()
-	<-procExit
 }
diff --git a/metropolis/test/launch/BUILD.bazel b/metropolis/test/launch/BUILD.bazel
index 3f9d8fd..22590fe 100644
--- a/metropolis/test/launch/BUILD.bazel
+++ b/metropolis/test/launch/BUILD.bazel
@@ -2,18 +2,11 @@
 
 go_library(
     name = "go_default_library",
-    srcs = [
-        "insecure_key.go",
-        "launch.go",
-    ],
+    srcs = ["launch.go"],
     importpath = "source.monogon.dev/metropolis/test/launch",
     visibility = ["//metropolis:__subpackages__"],
     deps = [
-        "//metropolis/node:go_default_library",
         "//metropolis/pkg/freeport:go_default_library",
-        "//metropolis/proto/api:go_default_library",
-        "@com_github_golang_protobuf//proto:go_default_library",
-        "@com_github_grpc_ecosystem_go_grpc_middleware//retry:go_default_library",
         "@org_golang_google_grpc//:go_default_library",
         "@org_golang_x_sys//unix:go_default_library",
     ],
diff --git a/metropolis/test/launch/cli/launch/BUILD.bazel b/metropolis/test/launch/cli/launch/BUILD.bazel
index 824b2ff..e7e9271 100644
--- a/metropolis/test/launch/cli/launch/BUILD.bazel
+++ b/metropolis/test/launch/cli/launch/BUILD.bazel
@@ -8,6 +8,7 @@
     deps = [
         "//metropolis/proto/api:go_default_library",
         "//metropolis/test/launch:go_default_library",
+        "//metropolis/test/launch/cluster:go_default_library",
     ],
 )
 
diff --git a/metropolis/test/launch/cli/launch/main.go b/metropolis/test/launch/cli/launch/main.go
index e736d76..aacba06 100644
--- a/metropolis/test/launch/cli/launch/main.go
+++ b/metropolis/test/launch/cli/launch/main.go
@@ -25,6 +25,7 @@
 
 	apb "source.monogon.dev/metropolis/proto/api"
 	"source.monogon.dev/metropolis/test/launch"
+	"source.monogon.dev/metropolis/test/launch/cluster"
 )
 
 func main() {
@@ -35,12 +36,12 @@
 		<-sigs
 		cancel()
 	}()
-	if err := launch.Launch(ctx, launch.Options{
-		Ports:      launch.IdentityPortMap(launch.NodePorts),
+	if err := cluster.LaunchNode(ctx, cluster.NodeOptions{
+		Ports:      launch.IdentityPortMap(cluster.NodePorts),
 		SerialPort: os.Stdout,
 		NodeParameters: &apb.NodeParameters{
 			Cluster: &apb.NodeParameters_ClusterBootstrap_{
-				ClusterBootstrap: launch.InsecureClusterBootstrap,
+				ClusterBootstrap: cluster.InsecureClusterBootstrap,
 			},
 		},
 	}); err != nil {
diff --git a/metropolis/test/launch/cluster/BUILD.bazel b/metropolis/test/launch/cluster/BUILD.bazel
new file mode 100644
index 0000000..6c8d794
--- /dev/null
+++ b/metropolis/test/launch/cluster/BUILD.bazel
@@ -0,0 +1,30 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+    name = "go_default_library",
+    srcs = [
+        "cluster.go",
+        "insecure_key.go",
+    ],
+    data = [
+        "//metropolis/node:image",
+        "//metropolis/node:swtpm_data",
+        "//metropolis/test/ktest:linux-testing",
+        "//metropolis/test/nanoswitch:initramfs",
+        "//third_party/edk2:firmware",
+        "@com_github_bonzini_qboot//:qboot-bin",
+    ],
+    importpath = "source.monogon.dev/metropolis/test/launch/cluster",
+    visibility = ["//visibility:public"],
+    deps = [
+        "//metropolis/node:go_default_library",
+        "//metropolis/node/core/rpc:go_default_library",
+        "//metropolis/proto/api:go_default_library",
+        "//metropolis/test/launch:go_default_library",
+        "@com_github_cenkalti_backoff_v4//:go_default_library",
+        "@com_github_grpc_ecosystem_go_grpc_middleware//retry:go_default_library",
+        "@org_golang_google_grpc//:go_default_library",
+        "@org_golang_google_protobuf//proto:go_default_library",
+        "@org_uber_go_multierr//:go_default_library",
+    ],
+)
diff --git a/metropolis/test/launch/cluster/cluster.go b/metropolis/test/launch/cluster/cluster.go
new file mode 100644
index 0000000..6be0efe
--- /dev/null
+++ b/metropolis/test/launch/cluster/cluster.go
@@ -0,0 +1,430 @@
+// cluster builds on the launch package and implements launching Metropolis
+// nodes and clusters in a virtualized environment using qemu. It's kept in a
+// separate package as it depends on a Metropolis node image, which might not be
+// required for some use of the launch library.
+package cluster
+
+import (
+	"bytes"
+	"context"
+	"crypto/rand"
+	"crypto/tls"
+	"errors"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"log"
+	"net"
+	"os"
+	"os/exec"
+	"path/filepath"
+	"syscall"
+	"time"
+
+	"github.com/cenkalti/backoff/v4"
+	grpcretry "github.com/grpc-ecosystem/go-grpc-middleware/retry"
+	"go.uber.org/multierr"
+	"google.golang.org/grpc"
+	"google.golang.org/protobuf/proto"
+
+	"source.monogon.dev/metropolis/node"
+	"source.monogon.dev/metropolis/node/core/rpc"
+	apb "source.monogon.dev/metropolis/proto/api"
+	"source.monogon.dev/metropolis/test/launch"
+)
+
+// Options contains all options that can be passed to Launch()
+type NodeOptions struct {
+	// Ports contains the port mapping where to expose the internal ports of the VM to
+	// the host. See IdentityPortMap() and ConflictFreePortMap(). Ignored when
+	// ConnectToSocket is set.
+	Ports launch.PortMap
+
+	// If set to true, reboots are honored. Otherwise all reboots exit the Launch()
+	// command. Metropolis nodes generally restarts on almost all errors, so unless you
+	// want to test reboot behavior this should be false.
+	AllowReboot bool
+
+	// By default the VM is connected to the Host via SLIRP. If ConnectToSocket is set,
+	// it is instead connected to the given file descriptor/socket. If this is set, all
+	// port maps from the Ports option are ignored. Intended for networking this
+	// instance together with others for running  more complex network configurations.
+	ConnectToSocket *os.File
+
+	// SerialPort is a io.ReadWriter over which you can communicate with the serial
+	// port of the machine It can be set to an existing file descriptor (like
+	// os.Stdout/os.Stderr) or any Go structure implementing this interface.
+	SerialPort io.ReadWriter
+
+	// NodeParameters is passed into the VM and subsequently used for bootstrapping or
+	// registering into a cluster.
+	NodeParameters *apb.NodeParameters
+}
+
+// NodePorts is the list of ports a fully operational Metropolis node listens on
+var NodePorts = []uint16{
+	node.ConsensusPort,
+
+	node.CuratorServicePort,
+	node.DebugServicePort,
+
+	node.KubernetesAPIPort,
+	node.MasterServicePort,
+	node.CuratorServicePort,
+	node.DebuggerPort,
+}
+
+// LaunchNode launches a single Metropolis node instance with the given options.
+// The instance runs mostly paravirtualized but with some emulated hardware
+// similar to how a cloud provider might set up its VMs. The disk is fully
+// writable but is run in snapshot mode meaning that changes are not kept beyond
+// a single invocation.
+func LaunchNode(ctx context.Context, options NodeOptions) error {
+	// Pin temp directory to /tmp until we can use abstract socket namespace in QEMU
+	// (next release after 5.0,
+	// https://github.com/qemu/qemu/commit/776b97d3605ed0fc94443048fdf988c7725e38a9).
+	// swtpm accepts already-open FDs so we can pass in an abstract socket namespace FD
+	// that we open and pass the name of it to QEMU. Not pinning this crashes both
+	// swtpm and qemu because we run into UNIX socket length limitations (for legacy
+	// reasons 108 chars).
+	tempDir, err := ioutil.TempDir("/tmp", "launch*")
+	if err != nil {
+		return fmt.Errorf("failed to create temporary directory: %w", err)
+	}
+	defer os.RemoveAll(tempDir)
+
+	// Copy TPM state into a temporary directory since it's being modified by the
+	// emulator
+	tpmTargetDir := filepath.Join(tempDir, "tpm")
+	tpmSrcDir := "metropolis/node/tpm"
+	if err := os.Mkdir(tpmTargetDir, 0755); err != nil {
+		return fmt.Errorf("failed to create TPM state directory: %w", err)
+	}
+	tpmFiles, err := ioutil.ReadDir(tpmSrcDir)
+	if err != nil {
+		return fmt.Errorf("failed to read TPM directory: %w", err)
+	}
+	for _, file := range tpmFiles {
+		name := file.Name()
+		src := filepath.Join(tpmSrcDir, name)
+		target := filepath.Join(tpmTargetDir, name)
+		if err := copyFile(src, target); err != nil {
+			return fmt.Errorf("failed to copy TPM directory: file %q to %q: %w", src, target, err)
+		}
+	}
+
+	var qemuNetType string
+	var qemuNetConfig launch.QemuValue
+	if options.ConnectToSocket != nil {
+		qemuNetType = "socket"
+		qemuNetConfig = launch.QemuValue{
+			"id": {"net0"},
+			"fd": {"3"},
+		}
+	} else {
+		qemuNetType = "user"
+		qemuNetConfig = launch.QemuValue{
+			"id":        {"net0"},
+			"net":       {"10.42.0.0/24"},
+			"dhcpstart": {"10.42.0.10"},
+			"hostfwd":   options.Ports.ToQemuForwards(),
+		}
+	}
+
+	tpmSocketPath := filepath.Join(tempDir, "tpm-socket")
+
+	mac, err := generateRandomEthernetMAC()
+	if err != nil {
+		return err
+	}
+
+	qemuArgs := []string{"-machine", "q35", "-accel", "kvm", "-nographic", "-nodefaults", "-m", "4096",
+		"-cpu", "host", "-smp", "sockets=1,cpus=1,cores=2,threads=2,maxcpus=4",
+		"-drive", "if=pflash,format=raw,readonly,file=external/edk2/OVMF_CODE.fd",
+		"-drive", "if=pflash,format=raw,snapshot=on,file=external/edk2/OVMF_VARS.fd",
+		"-drive", "if=virtio,format=raw,snapshot=on,cache=unsafe,file=metropolis/node/node.img",
+		"-netdev", qemuNetConfig.ToOption(qemuNetType),
+		"-device", "virtio-net-pci,netdev=net0,mac=" + mac.String(),
+		"-chardev", "socket,id=chrtpm,path=" + tpmSocketPath,
+		"-tpmdev", "emulator,id=tpm0,chardev=chrtpm",
+		"-device", "tpm-tis,tpmdev=tpm0",
+		"-device", "virtio-rng-pci",
+		"-serial", "stdio"}
+
+	if !options.AllowReboot {
+		qemuArgs = append(qemuArgs, "-no-reboot")
+	}
+
+	if options.NodeParameters != nil {
+		parametersPath := filepath.Join(tempDir, "parameters.pb")
+		parametersRaw, err := proto.Marshal(options.NodeParameters)
+		if err != nil {
+			return fmt.Errorf("failed to encode node paraeters: %w", err)
+		}
+		if err := ioutil.WriteFile(parametersPath, parametersRaw, 0644); err != nil {
+			return fmt.Errorf("failed to write node parameters: %w", err)
+		}
+		qemuArgs = append(qemuArgs, "-fw_cfg", "name=dev.monogon.metropolis/parameters.pb,file="+parametersPath)
+	}
+
+	// Start TPM emulator as a subprocess
+	tpmCtx, tpmCancel := context.WithCancel(ctx)
+	defer tpmCancel()
+
+	tpmEmuCmd := exec.CommandContext(tpmCtx, "swtpm", "socket", "--tpm2", "--tpmstate", "dir="+tpmTargetDir, "--ctrl", "type=unixio,path="+tpmSocketPath)
+	tpmEmuCmd.Stderr = os.Stderr
+	tpmEmuCmd.Stdout = os.Stdout
+
+	err = tpmEmuCmd.Start()
+	if err != nil {
+		return fmt.Errorf("failed to start TPM emulator: %w", err)
+	}
+
+	// Start the main qemu binary
+	systemCmd := exec.CommandContext(ctx, "qemu-system-x86_64", qemuArgs...)
+	if options.ConnectToSocket != nil {
+		systemCmd.ExtraFiles = []*os.File{options.ConnectToSocket}
+	}
+
+	var stdErrBuf bytes.Buffer
+	systemCmd.Stderr = &stdErrBuf
+	systemCmd.Stdout = options.SerialPort
+
+	err = systemCmd.Run()
+
+	// Stop TPM emulator and wait for it to exit to properly reap the child process
+	tpmCancel()
+	log.Print("Node: Waiting for TPM emulator to exit")
+	// Wait returns a SIGKILL error because we just cancelled its context.
+	// We still need to call it to avoid creating zombies.
+	_ = tpmEmuCmd.Wait()
+	log.Print("Node: TPM emulator done")
+
+	var exerr *exec.ExitError
+	if err != nil && errors.As(err, &exerr) {
+		status := exerr.ProcessState.Sys().(syscall.WaitStatus)
+		if status.Signaled() && status.Signal() == syscall.SIGKILL {
+			// Process was killed externally (most likely by our context being canceled).
+			// This is a normal exit for us, so return nil
+			return nil
+		}
+		exerr.Stderr = stdErrBuf.Bytes()
+		newErr := launch.QEMUError(*exerr)
+		return &newErr
+	}
+	return err
+}
+
+func copyFile(src, dst string) error {
+	in, err := os.Open(src)
+	if err != nil {
+		return fmt.Errorf("when opening source: %w", err)
+	}
+	defer in.Close()
+
+	out, err := os.Create(dst)
+	if err != nil {
+		return fmt.Errorf("when creating destination: %w", err)
+	}
+	defer out.Close()
+
+	_, err = io.Copy(out, in)
+	if err != nil {
+		return fmt.Errorf("when copying file: %w", err)
+	}
+	return out.Close()
+}
+
+// Gets a random EUI-48 Ethernet MAC address
+func generateRandomEthernetMAC() (*net.HardwareAddr, error) {
+	macBuf := make([]byte, 6)
+	_, err := rand.Read(macBuf)
+	if err != nil {
+		return nil, fmt.Errorf("failed to read randomness for MAC: %v", err)
+	}
+
+	// Set U/L bit and clear I/G bit (locally administered individual MAC)
+	// Ref IEEE 802-2014 Section 8.2.2
+	macBuf[0] = (macBuf[0] | 2) & 0xfe
+	mac := net.HardwareAddr(macBuf)
+	return &mac, nil
+}
+
+// ClusterPorts contains all ports forwarded by Nanoswitch to the first VM in a
+// launched Metropolis cluster.
+var ClusterPorts = []uint16{
+	node.CuratorServicePort,
+	node.DebugServicePort,
+
+	node.KubernetesAPIPort,
+}
+
+// ClusterOptions contains all options for launching a Metropolis cluster.
+type ClusterOptions struct {
+	// The number of nodes this cluster should be started with.
+	NumNodes int
+}
+
+// Cluster is the running Metropolis cluster launched using the LaunchCluster
+// function.
+type Cluster struct {
+	// Debug is the NodeDebugService gRPC service, allowing for debug
+	// unauthenticated cluster to the access.
+	Debug apb.NodeDebugServiceClient
+	// Management is the Management gRPC service, authenticated as the owner of the
+	// cluster.
+	Management apb.ManagementClient
+	// Owner is the TLS Certificate of the owner of the test cluster. This can be
+	// used to authenticate further clients to the running cluster.
+	Owner tls.Certificate
+	// Ports is the PortMap used to access the first nodes' services (defined in
+	// ClusterPorts).
+	Ports launch.PortMap
+
+	// nodesDone is a list of channels populated with the return codes from all the
+	// nodes' qemu instances. It's used by Close to ensure all nodes have
+	// succesfully been stopped.
+	nodesDone []chan error
+	// ctxC is used by Close to cancel the context under which the nodes are
+	// running.
+	ctxC context.CancelFunc
+}
+
+// LaunchCluster launches a cluster of Metropolis node VMs together with a
+// Nanoswitch instance to network them all together.
+//
+// The given context will be used to run all qemu instances in the cluster, and
+// canceling the context or calling Close() will terminate them.
+func LaunchCluster(ctx context.Context, opts ClusterOptions) (*Cluster, error) {
+	if opts.NumNodes == 0 {
+		return nil, errors.New("refusing to start cluster with zero nodes")
+	}
+	if opts.NumNodes > 1 {
+		return nil, errors.New("unimplemented")
+	}
+
+	ctxT, ctxC := context.WithCancel(ctx)
+
+	// Prepare links between nodes and nanoswitch.
+	var switchPorts []*os.File
+	var vmPorts []*os.File
+	for i := 0; i < opts.NumNodes; i++ {
+		switchPort, vmPort, err := launch.NewSocketPair()
+		if err != nil {
+			ctxC()
+			return nil, fmt.Errorf("failed to get socketpair: %w", err)
+		}
+		switchPorts = append(switchPorts, switchPort)
+		vmPorts = append(vmPorts, vmPort)
+	}
+
+	// Make a list of channels that will be populated and closed by all running node
+	// qemu processes.
+	done := make([]chan error, opts.NumNodes)
+	for i, _ := range done {
+		done[i] = make(chan error, 1)
+	}
+
+	// Start first node.
+	go func() {
+		err := LaunchNode(ctxT, NodeOptions{
+			ConnectToSocket: vmPorts[0],
+			NodeParameters: &apb.NodeParameters{
+				Cluster: &apb.NodeParameters_ClusterBootstrap_{
+					ClusterBootstrap: &apb.NodeParameters_ClusterBootstrap{
+						OwnerPublicKey: InsecurePublicKey,
+					},
+				},
+			},
+		})
+		done[0] <- err
+	}()
+
+	portMap, err := launch.ConflictFreePortMap(ClusterPorts)
+	if err != nil {
+		ctxC()
+		return nil, fmt.Errorf("failed to allocate ephemeral ports: %w", err)
+	}
+
+	go func() {
+		if err := launch.RunMicroVM(ctxT, &launch.MicroVMOptions{
+			KernelPath:             "metropolis/test/ktest/vmlinux",
+			InitramfsPath:          "metropolis/test/nanoswitch/initramfs.lz4",
+			ExtraNetworkInterfaces: switchPorts,
+			PortMap:                portMap,
+		}); err != nil {
+			if !errors.Is(err, ctxT.Err()) {
+				log.Printf("Failed to launch nanoswitch: %v", err)
+			}
+		}
+	}()
+
+	// Dial debug service.
+	copts := []grpcretry.CallOption{
+		grpcretry.WithBackoff(grpcretry.BackoffExponential(100 * time.Millisecond)),
+	}
+	debugConn, err := portMap.DialGRPC(node.DebugServicePort, grpc.WithInsecure(),
+		grpc.WithUnaryInterceptor(grpcretry.UnaryClientInterceptor(copts...)))
+	if err != nil {
+		ctxC()
+		return nil, fmt.Errorf("failed to dial debug service: %w", err)
+	}
+
+	// Dial external service.
+	remote := fmt.Sprintf("localhost:%v", portMap[node.CuratorServicePort])
+	initClient, err := rpc.NewEphemeralClient(remote, InsecurePrivateKey, nil)
+	if err != nil {
+		ctxC()
+		return nil, fmt.Errorf("NewInitialClient: %w", err)
+	}
+
+	// Retrieve owner certificate - this can take a while because the node is still
+	// coming up, so do it in a backoff loop.
+	log.Printf("Cluster: retrieving owner certificate...")
+	aaa := apb.NewAAAClient(initClient)
+	var cert *tls.Certificate
+	err = backoff.Retry(func() error {
+		cert, err = rpc.RetrieveOwnerCertificate(ctxT, aaa, InsecurePrivateKey)
+		return err
+	}, backoff.WithContext(backoff.NewExponentialBackOff(), ctxT))
+	if err != nil {
+		ctxC()
+		return nil, err
+	}
+	log.Printf("Cluster: retrieved owner certificate.")
+
+	authClient, err := rpc.NewAuthenticatedClient(remote, *cert, nil)
+	if err != nil {
+		ctxC()
+		return nil, fmt.Errorf("NewAuthenticatedClient: %w", err)
+	}
+
+	return &Cluster{
+		Debug:      apb.NewNodeDebugServiceClient(debugConn),
+		Management: apb.NewManagementClient(authClient),
+		Owner:      *cert,
+		Ports:      portMap,
+		nodesDone:  done,
+
+		ctxC: ctxC,
+	}, nil
+}
+
+// Close cancels the running clusters' context and waits for all virtualized
+// nodes to stop. It returns an error if stopping the nodes failed, or one of
+// the nodes failed to fully start in the first place.
+func (c *Cluster) Close() error {
+	log.Printf("Cluster: stopping...")
+	c.ctxC()
+
+	var errors []error
+	log.Printf("Cluster: waiting for nodes to exit...")
+	for _, c := range c.nodesDone {
+		err := <-c
+		if err != nil {
+			errors = append(errors, err)
+		}
+	}
+	log.Printf("Cluster: done")
+	return multierr.Combine(errors...)
+}
diff --git a/metropolis/test/launch/insecure_key.go b/metropolis/test/launch/cluster/insecure_key.go
similarity index 98%
rename from metropolis/test/launch/insecure_key.go
rename to metropolis/test/launch/cluster/insecure_key.go
index 72af26f..48cd6d8 100644
--- a/metropolis/test/launch/insecure_key.go
+++ b/metropolis/test/launch/cluster/insecure_key.go
@@ -14,7 +14,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package launch
+package cluster
 
 import (
 	"crypto/ed25519"
diff --git a/metropolis/test/launch/launch.go b/metropolis/test/launch/launch.go
index 0d13792..0b15a58 100644
--- a/metropolis/test/launch/launch.go
+++ b/metropolis/test/launch/launch.go
@@ -14,41 +14,32 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+// launch implements test harnesses for running qemu VMs from tests.
 package launch
 
 import (
 	"bytes"
 	"context"
-	"crypto/rand"
 	"errors"
 	"fmt"
-	"io"
-	"io/ioutil"
-	"log"
 	"net"
 	"os"
 	"os/exec"
-	"path/filepath"
 	"strconv"
 	"strings"
 	"syscall"
-	"time"
 
-	"github.com/golang/protobuf/proto"
-	grpcretry "github.com/grpc-ecosystem/go-grpc-middleware/retry"
 	"golang.org/x/sys/unix"
 	"google.golang.org/grpc"
 
-	"source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/pkg/freeport"
-	apb "source.monogon.dev/metropolis/proto/api"
 )
 
-type qemuValue map[string][]string
+type QemuValue map[string][]string
 
-// toOption encodes structured data into a QEMU option. Example: "test", {"key1":
+// ToOption encodes structured data into a QEMU option. Example: "test", {"key1":
 // {"val1"}, "key2": {"val2", "val3"}} returns "test,key1=val1,key2=val2,key2=val3"
-func (value qemuValue) toOption(name string) string {
+func (value QemuValue) ToOption(name string) string {
 	var optionValues []string
 	if name != "" {
 		optionValues = append(optionValues, name)
@@ -64,33 +55,13 @@
 	return strings.Join(optionValues, ",")
 }
 
-func copyFile(src, dst string) error {
-	in, err := os.Open(src)
-	if err != nil {
-		return fmt.Errorf("when opening source: %w", err)
-	}
-	defer in.Close()
-
-	out, err := os.Create(dst)
-	if err != nil {
-		return fmt.Errorf("when creating destination: %w", err)
-	}
-	defer out.Close()
-
-	_, err = io.Copy(out, in)
-	if err != nil {
-		return fmt.Errorf("when copying file: %w", err)
-	}
-	return out.Close()
-}
-
 // PortMap represents where VM ports are mapped to on the host. It maps from the VM
 // port number to the host port number.
 type PortMap map[uint16]uint16
 
-// toQemuForwards generates QEMU hostfwd values (https://qemu.weilnetz.de/doc/qemu-
+// ToQemuForwards generates QEMU hostfwd values (https://qemu.weilnetz.de/doc/qemu-
 // doc.html#:~:text=hostfwd=) for all mapped ports.
-func (p PortMap) toQemuForwards() []string {
+func (p PortMap) ToQemuForwards() []string {
 	var hostfwdOptions []string
 	for vmPort, hostPort := range p {
 		hostfwdOptions = append(hostfwdOptions, fmt.Sprintf("tcp::%v-:%v", hostPort, vmPort))
@@ -112,38 +83,6 @@
 	return grpcClient, nil
 }
 
-// Options contains all options that can be passed to Launch()
-type Options struct {
-	// Ports contains the port mapping where to expose the internal ports of the VM to
-	// the host. See IdentityPortMap() and ConflictFreePortMap(). Ignored when
-	// ConnectToSocket is set.
-	Ports PortMap
-
-	// If set to true, reboots are honored. Otherwise all reboots exit the Launch()
-	// command. Metropolis nodes generally restarts on almost all errors, so unless you
-	// want to test reboot behavior this should be false.
-	AllowReboot bool
-
-	// By default the VM is connected to the Host via SLIRP. If ConnectToSocket is set,
-	// it is instead connected to the given file descriptor/socket. If this is set, all
-	// port maps from the Ports option are ignored. Intended for networking this
-	// instance together with others for running  more complex network configurations.
-	ConnectToSocket *os.File
-
-	// SerialPort is a io.ReadWriter over which you can communicate with the serial
-	// port of the machine It can be set to an existing file descriptor (like
-	// os.Stdout/os.Stderr) or any Go structure implementing this interface.
-	SerialPort io.ReadWriter
-
-	// NodeParameters is passed into the VM and subsequently used for bootstrapping or
-	// registering into a cluster.
-	NodeParameters *apb.NodeParameters
-}
-
-// NodePorts is the list of ports a fully operational Metropolis node listens on
-var NodePorts = []uint16{node.ConsensusPort, node.CuratorServicePort, node.MasterServicePort,
-	node.ExternalServicePort, node.DebugServicePort, node.KubernetesAPIPort, node.DebuggerPort}
-
 // IdentityPortMap returns a port map where each given port is mapped onto itself
 // on the host. This is mainly useful for development against Metropolis. The dbg
 // command requires this mapping.
@@ -175,160 +114,6 @@
 	return portMap, nil
 }
 
-// Gets a random EUI-48 Ethernet MAC address
-func generateRandomEthernetMAC() (*net.HardwareAddr, error) {
-	macBuf := make([]byte, 6)
-	_, err := rand.Read(macBuf)
-	if err != nil {
-		return nil, fmt.Errorf("failed to read randomness for MAC: %v", err)
-	}
-
-	// Set U/L bit and clear I/G bit (locally administered individual MAC)
-	// Ref IEEE 802-2014 Section 8.2.2
-	macBuf[0] = (macBuf[0] | 2) & 0xfe
-	mac := net.HardwareAddr(macBuf)
-	return &mac, nil
-}
-
-// Launch launches a Metropolis node instance with the given options. The instance
-// runs mostly paravirtualized but with some emulated hardware similar to how a
-// cloud provider might set up its VMs. The disk is fully writable but is run in
-// snapshot mode meaning that changes are not kept beyond a single invocation.
-func Launch(ctx context.Context, options Options) error {
-	// Pin temp directory to /tmp until we can use abstract socket namespace in QEMU
-	// (next release after 5.0,
-	// https://github.com/qemu/qemu/commit/776b97d3605ed0fc94443048fdf988c7725e38a9).
-	// swtpm accepts already-open FDs so we can pass in an abstract socket namespace FD
-	// that we open and pass the name of it to QEMU. Not pinning this crashes both
-	// swtpm and qemu because we run into UNIX socket length limitations (for legacy
-	// reasons 108 chars).
-	tempDir, err := ioutil.TempDir("/tmp", "launch*")
-	if err != nil {
-		return fmt.Errorf("failed to create temporary directory: %w", err)
-	}
-	defer os.RemoveAll(tempDir)
-
-	// Copy TPM state into a temporary directory since it's being modified by the
-	// emulator
-	tpmTargetDir := filepath.Join(tempDir, "tpm")
-	tpmSrcDir := "metropolis/node/tpm"
-	if err := os.Mkdir(tpmTargetDir, 0755); err != nil {
-		return fmt.Errorf("failed to create TPM state directory: %w", err)
-	}
-	tpmFiles, err := ioutil.ReadDir(tpmSrcDir)
-	if err != nil {
-		return fmt.Errorf("failed to read TPM directory: %w", err)
-	}
-	for _, file := range tpmFiles {
-		name := file.Name()
-		src := filepath.Join(tpmSrcDir, name)
-		target := filepath.Join(tpmTargetDir, name)
-		if err := copyFile(src, target); err != nil {
-			return fmt.Errorf("failed to copy TPM directory: file %q to %q: %w", src, target, err)
-		}
-	}
-
-	var qemuNetType string
-	var qemuNetConfig qemuValue
-	if options.ConnectToSocket != nil {
-		qemuNetType = "socket"
-		qemuNetConfig = qemuValue{
-			"id": {"net0"},
-			"fd": {"3"},
-		}
-	} else {
-		qemuNetType = "user"
-		qemuNetConfig = qemuValue{
-			"id":        {"net0"},
-			"net":       {"10.42.0.0/24"},
-			"dhcpstart": {"10.42.0.10"},
-			"hostfwd":   options.Ports.toQemuForwards(),
-		}
-	}
-
-	tpmSocketPath := filepath.Join(tempDir, "tpm-socket")
-
-	mac, err := generateRandomEthernetMAC()
-	if err != nil {
-		return err
-	}
-
-	qemuArgs := []string{"-machine", "q35", "-accel", "kvm", "-nographic", "-nodefaults", "-m", "4096",
-		"-cpu", "host", "-smp", "sockets=1,cpus=1,cores=2,threads=2,maxcpus=4",
-		"-drive", "if=pflash,format=raw,readonly,file=external/edk2/OVMF_CODE.fd",
-		"-drive", "if=pflash,format=raw,snapshot=on,file=external/edk2/OVMF_VARS.fd",
-		"-drive", "if=virtio,format=raw,snapshot=on,cache=unsafe,file=metropolis/node/node.img",
-		"-netdev", qemuNetConfig.toOption(qemuNetType),
-		"-device", "virtio-net-pci,netdev=net0,mac=" + mac.String(),
-		"-chardev", "socket,id=chrtpm,path=" + tpmSocketPath,
-		"-tpmdev", "emulator,id=tpm0,chardev=chrtpm",
-		"-device", "tpm-tis,tpmdev=tpm0",
-		"-device", "virtio-rng-pci",
-		"-serial", "stdio"}
-
-	if !options.AllowReboot {
-		qemuArgs = append(qemuArgs, "-no-reboot")
-	}
-
-	if options.NodeParameters != nil {
-		parametersPath := filepath.Join(tempDir, "parameters.pb")
-		parametersRaw, err := proto.Marshal(options.NodeParameters)
-		if err != nil {
-			return fmt.Errorf("failed to encode node paraeters: %w", err)
-		}
-		if err := ioutil.WriteFile(parametersPath, parametersRaw, 0644); err != nil {
-			return fmt.Errorf("failed to write node parameters: %w", err)
-		}
-		qemuArgs = append(qemuArgs, "-fw_cfg", "name=dev.monogon.metropolis/parameters.pb,file="+parametersPath)
-	}
-
-	// Start TPM emulator as a subprocess
-	tpmCtx, tpmCancel := context.WithCancel(ctx)
-	defer tpmCancel()
-
-	tpmEmuCmd := exec.CommandContext(tpmCtx, "swtpm", "socket", "--tpm2", "--tpmstate", "dir="+tpmTargetDir, "--ctrl", "type=unixio,path="+tpmSocketPath)
-	tpmEmuCmd.Stderr = os.Stderr
-	tpmEmuCmd.Stdout = os.Stdout
-
-	err = tpmEmuCmd.Start()
-	if err != nil {
-		return fmt.Errorf("failed to start TPM emulator: %w", err)
-	}
-
-	// Start the main qemu binary
-	systemCmd := exec.CommandContext(ctx, "qemu-system-x86_64", qemuArgs...)
-	if options.ConnectToSocket != nil {
-		systemCmd.ExtraFiles = []*os.File{options.ConnectToSocket}
-	}
-
-	var stdErrBuf bytes.Buffer
-	systemCmd.Stderr = &stdErrBuf
-	systemCmd.Stdout = options.SerialPort
-
-	err = systemCmd.Run()
-
-	// Stop TPM emulator and wait for it to exit to properly reap the child process
-	tpmCancel()
-	log.Print("Waiting for TPM emulator to exit")
-	// Wait returns a SIGKILL error because we just cancelled its context.
-	// We still need to call it to avoid creating zombies.
-	_ = tpmEmuCmd.Wait()
-
-	var exerr *exec.ExitError
-	if err != nil && errors.As(err, &exerr) {
-		status := exerr.ProcessState.Sys().(syscall.WaitStatus)
-		if status.Signaled() && status.Signal() == syscall.SIGKILL {
-			// Process was killed externally (most likely by our context being canceled).
-			// This is a normal exit for us, so return nil
-			return nil
-		}
-		exerr.Stderr = stdErrBuf.Bytes()
-		newErr := QEMUError(*exerr)
-		return &newErr
-	}
-	return err
-}
-
 // NewSocketPair creates a new socket pair. By connecting both ends to different
 // instances you can connect them with a virtual "network cable". The ends can be
 // passed into the ConnectToSocket option.
@@ -399,28 +184,28 @@
 		// functionality to get a single bidirectional chardev backend backed by a passed-
 		// down RDWR fd. Ref https://lists.gnu.org/archive/html/qemu-devel/2015-
 		// 12/msg01256.html
-		addFdConf := qemuValue{
+		addFdConf := QemuValue{
 			"set": {idxStr},
 			"fd":  {strconv.Itoa(idx + 3)},
 		}
-		chardevConf := qemuValue{
+		chardevConf := QemuValue{
 			"id":   {id},
 			"path": {"/dev/fdset/" + idxStr},
 		}
-		deviceConf := qemuValue{
+		deviceConf := QemuValue{
 			"chardev": {id},
 		}
-		extraArgs = append(extraArgs, "-add-fd", addFdConf.toOption(""),
-			"-chardev", chardevConf.toOption("pipe"), "-device", deviceConf.toOption("virtserialport"))
+		extraArgs = append(extraArgs, "-add-fd", addFdConf.ToOption(""),
+			"-chardev", chardevConf.ToOption("pipe"), "-device", deviceConf.ToOption("virtserialport"))
 	}
 
 	for idx, _ := range opts.ExtraNetworkInterfaces {
 		id := fmt.Sprintf("net%v", idx)
-		netdevConf := qemuValue{
+		netdevConf := QemuValue{
 			"id": {id},
 			"fd": {strconv.Itoa(idx + 3 + len(opts.ExtraChardevs))},
 		}
-		extraArgs = append(extraArgs, "-netdev", netdevConf.toOption("socket"), "-device", "virtio-net-device,netdev="+id)
+		extraArgs = append(extraArgs, "-netdev", netdevConf.ToOption("socket"), "-device", "virtio-net-device,netdev="+id)
 	}
 
 	// This sets up a minimum viable environment for our Linux kernel. It clears all
@@ -455,16 +240,16 @@
 
 	if !opts.DisableHostNetworkInterface {
 		qemuNetType := "user"
-		qemuNetConfig := qemuValue{
+		qemuNetConfig := QemuValue{
 			"id":        {"usernet0"},
 			"net":       {"10.42.0.0/24"},
 			"dhcpstart": {"10.42.0.10"},
 		}
 		if opts.PortMap != nil {
-			qemuNetConfig["hostfwd"] = opts.PortMap.toQemuForwards()
+			qemuNetConfig["hostfwd"] = opts.PortMap.ToQemuForwards()
 		}
 
-		baseArgs = append(baseArgs, "-netdev", qemuNetConfig.toOption(qemuNetType),
+		baseArgs = append(baseArgs, "-netdev", qemuNetConfig.ToOption(qemuNetType),
 			"-device", "virtio-net-device,netdev=usernet0,mac="+HostInterfaceMAC.String())
 	}
 
@@ -477,6 +262,13 @@
 	cmd.ExtraFiles = append(cmd.ExtraFiles, opts.ExtraNetworkInterfaces...)
 
 	err := cmd.Run()
+	// If it's a context error, just quit. There's no way to tell a
+	// killed-due-to-context vs killed-due-to-external-reason error returned by Run,
+	// so we approximate by looking at the context's status.
+	if err != nil && ctx.Err() != nil {
+		return ctx.Err()
+	}
+
 	var exerr *exec.ExitError
 	if err != nil && errors.As(err, &exerr) {
 		exerr.Stderr = stdErrBuf.Bytes()
@@ -493,88 +285,3 @@
 func (e *QEMUError) Error() string {
 	return fmt.Sprintf("%v: %v", e.String(), string(e.Stderr))
 }
-
-// NanoswitchPorts contains all ports forwarded by Nanoswitch to the first VM
-var NanoswitchPorts = []uint16{
-	node.ExternalServicePort,
-	node.DebugServicePort,
-	node.KubernetesAPIPort,
-}
-
-// ClusterOptions contains all options for launching a Metropolis cluster
-type ClusterOptions struct {
-	// The number of nodes this cluster should be started with initially
-	NumNodes int
-}
-
-// LaunchCluster launches a cluster of Metropolis node VMs together with a
-// Nanoswitch instance to network them all together.
-func LaunchCluster(ctx context.Context, opts ClusterOptions) (apb.NodeDebugServiceClient, PortMap, error) {
-	var switchPorts []*os.File
-	var vmPorts []*os.File
-	for i := 0; i < opts.NumNodes; i++ {
-		switchPort, vmPort, err := NewSocketPair()
-		if err != nil {
-			return nil, nil, fmt.Errorf("failed to get socketpair: %w", err)
-		}
-		switchPorts = append(switchPorts, switchPort)
-		vmPorts = append(vmPorts, vmPort)
-	}
-
-	if opts.NumNodes == 0 {
-		return nil, nil, errors.New("refusing to start cluster with zero nodes")
-	}
-
-	if opts.NumNodes > 2 {
-		return nil, nil, errors.New("launching more than 2 nodes is unsupported pending replacement of golden tickets")
-	}
-
-	go func() {
-		if err := Launch(ctx, Options{
-			ConnectToSocket: vmPorts[0],
-			NodeParameters: &apb.NodeParameters{
-				Cluster: &apb.NodeParameters_ClusterBootstrap_{
-					ClusterBootstrap: &apb.NodeParameters_ClusterBootstrap{},
-				},
-			},
-		}); err != nil {
-
-			// Launch() only terminates when QEMU has terminated. At that point our function
-			// probably doesn't run anymore so we have no way of communicating the error back
-			// up, so let's just log it. Also a failure in launching VMs should be very visible
-			// by the unavailability of the clients we return.
-			log.Printf("Failed to launch vm0: %v", err)
-		}
-	}()
-
-	portMap, err := ConflictFreePortMap(NanoswitchPorts)
-	if err != nil {
-		return nil, nil, fmt.Errorf("failed to allocate ephemeral ports: %w", err)
-	}
-
-	go func() {
-		if err := RunMicroVM(ctx, &MicroVMOptions{
-			KernelPath:             "metropolis/test/ktest/vmlinux",
-			InitramfsPath:          "metropolis/test/nanoswitch/initramfs.lz4",
-			ExtraNetworkInterfaces: switchPorts,
-			PortMap:                portMap,
-		}); err != nil {
-			log.Printf("Failed to launch nanoswitch: %v", err)
-		}
-	}()
-	copts := []grpcretry.CallOption{
-		grpcretry.WithBackoff(grpcretry.BackoffExponential(100 * time.Millisecond)),
-	}
-	conn, err := portMap.DialGRPC(node.DebugServicePort, grpc.WithInsecure(),
-		grpc.WithUnaryInterceptor(grpcretry.UnaryClientInterceptor(copts...)))
-	if err != nil {
-		return nil, nil, fmt.Errorf("failed to dial debug service: %w", err)
-	}
-	debug := apb.NewNodeDebugServiceClient(conn)
-
-	if opts.NumNodes == 2 {
-		return nil, nil, fmt.Errorf("multinode unimplemented")
-	}
-
-	return debug, portMap, nil
-}
diff --git a/metropolis/test/nanoswitch/nanoswitch.go b/metropolis/test/nanoswitch/nanoswitch.go
index 21a526e..5c674f5 100644
--- a/metropolis/test/nanoswitch/nanoswitch.go
+++ b/metropolis/test/nanoswitch/nanoswitch.go
@@ -309,7 +309,7 @@
 			logger.Info("No upstream interface detected")
 		}
 		supervisor.Run(ctx, "dhcp-server", runDHCPServer(vmBridgeLink))
-		supervisor.Run(ctx, "proxy-ext1", userspaceProxy(net.IPv4(10, 1, 0, 2), common.ExternalServicePort))
+		supervisor.Run(ctx, "proxy-cur1", userspaceProxy(net.IPv4(10, 1, 0, 2), common.CuratorServicePort))
 		supervisor.Run(ctx, "proxy-dbg1", userspaceProxy(net.IPv4(10, 1, 0, 2), common.DebugServicePort))
 		supervisor.Run(ctx, "proxy-k8s-api1", userspaceProxy(net.IPv4(10, 1, 0, 2), common.KubernetesAPIPort))
 		supervisor.Signal(ctx, supervisor.SignalHealthy)