metropolis/proto: EnrolmentConfig -> NodeParameters

This starts off the move to a node configuration API conforming to
the lifecycle management design document.

Instead of an Enrolment Config used only to join an existing cluster, we
move to a NodeParameters proto that must always be given to a node if
it's supposed to either bootstrap a new cluster or join an existing one.

This links the existing cluster management code (and its state machine)
to work with this file. However, that state machine will be removed very
soon, anyway.

We also remove everything related to golden tickets.

This breaks multi-node tests.

X-Origin-Diff: phab/D710
GitOrigin-RevId: f22615fbccab975f2d5e6928bdc7387ab3aa5714
diff --git a/metropolis/cli/dbg/main.go b/metropolis/cli/dbg/main.go
index c0baad9..eb9070f 100644
--- a/metropolis/cli/dbg/main.go
+++ b/metropolis/cli/dbg/main.go
@@ -61,7 +61,6 @@
 
 		fmt.Fprintf(os.Stderr, "Example:\n  %s %s --tail 5 --follow init\n", os.Args[0], os.Args[1])
 	}
-	goldenticketCmd := flag.NewFlagSet("goldenticket", flag.ExitOnError)
 	conditionCmd := flag.NewFlagSet("condition", flag.ExitOnError)
 	conditionCmd.Usage = func() {
 		fmt.Fprintf(os.Stderr, "Usage: %s %s [options] component_path\n", os.Args[0], os.Args[1])
@@ -123,15 +122,6 @@
 				fmt.Println(entry.String())
 			}
 		}
-	case "goldenticket":
-		goldenticketCmd.Parse(os.Args[2:])
-		ip := goldenticketCmd.Arg(0)
-		res, err := debugClient.GetGoldenTicket(ctx, &apb.GetGoldenTicketRequest{ExternalIp: ip})
-		if err != nil {
-			fmt.Fprintf(os.Stderr, "Failed to get golden ticket: %v\n", err)
-			os.Exit(1)
-		}
-		fmt.Println(res.Ticket)
 	case "kubectl":
 		// Always get a kubeconfig with cluster-admin (group system:masters), kubectl itself can impersonate
 		kubeconfigFile, err := ioutil.TempFile("", "dbg_kubeconfig")
diff --git a/metropolis/node/build/mkimage/main.go b/metropolis/node/build/mkimage/main.go
index c2f7d5a..5546055 100644
--- a/metropolis/node/build/mkimage/main.go
+++ b/metropolis/node/build/mkimage/main.go
@@ -18,7 +18,7 @@
 
 // mkimage is a tool to generate a Metropolis node disk image containing the
 // given EFI payload, and optionally, a given external initramfs image and
-// enrolment credentials.
+// node parameters
 
 import (
 	"flag"
@@ -38,13 +38,13 @@
 var NodeSystemPartition gpt.Type = gpt.Type("ee96055b-f6d0-4267-8bbb-724b2afea74c")
 
 var (
-	flagEFI                  string
-	flagOut                  string
-	flagSystemPath           string
-	flagEnrolmentCredentials string
-	flagDataPartitionSize    uint64
-	flagESPPartitionSize     uint64
-	flagSystemPartitionSize  uint64
+	flagEFI                 string
+	flagOut                 string
+	flagSystemPath          string
+	flagNodeParameters      string
+	flagDataPartitionSize   uint64
+	flagESPPartitionSize    uint64
+	flagSystemPartitionSize uint64
 )
 
 func mibToSectors(size uint64) uint64 {
@@ -67,7 +67,7 @@
 	flag.StringVar(&flagEFI, "efi", "", "UEFI payload")
 	flag.StringVar(&flagOut, "out", "", "Output disk image")
 	flag.StringVar(&flagSystemPath, "system", "", "System partition [optional]")
-	flag.StringVar(&flagEnrolmentCredentials, "enrolment_credentials", "", "Enrolment credentials [optional]")
+	flag.StringVar(&flagNodeParameters, "node_parameters", "", "Node parameters [optional]")
 	flag.Uint64Var(&flagDataPartitionSize, "data_partition_size", 2048, "Override the data partition size (default 2048 MiB)")
 	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)")
@@ -144,8 +144,8 @@
 
 	put(fs, flagEFI, "/EFI/BOOT/BOOTX64.EFI")
 
-	if flagEnrolmentCredentials != "" {
-		put(fs, flagEnrolmentCredentials, "/EFI/metropolis/enrolment.pb")
+	if flagNodeParameters != "" {
+		put(fs, flagNodeParameters, "/EFI/metropolis/parameters.pb")
 	}
 
 	if err := diskImg.File.Close(); err != nil {
diff --git a/metropolis/node/core/BUILD.bazel b/metropolis/node/core/BUILD.bazel
index 28f4d2a..e7e6179 100644
--- a/metropolis/node/core/BUILD.bazel
+++ b/metropolis/node/core/BUILD.bazel
@@ -16,7 +16,6 @@
     deps = [
         "//metropolis/node:go_default_library",
         "//metropolis/node/core/cluster:go_default_library",
-        "//metropolis/node/core/consensus/ca:go_default_library",
         "//metropolis/node/core/localstorage:go_default_library",
         "//metropolis/node/core/localstorage/declarative:go_default_library",
         "//metropolis/node/core/network:go_default_library",
diff --git a/metropolis/node/core/cluster/BUILD.bazel b/metropolis/node/core/cluster/BUILD.bazel
index ad2620c..4490e1f 100644
--- a/metropolis/node/core/cluster/BUILD.bazel
+++ b/metropolis/node/core/cluster/BUILD.bazel
@@ -9,10 +9,8 @@
     importpath = "source.monogon.dev/metropolis/node/core/cluster",
     visibility = ["//metropolis/node/core:__subpackages__"],
     deps = [
-        "//metropolis/node:go_default_library",
         "//metropolis/node/core/consensus:go_default_library",
         "//metropolis/node/core/localstorage:go_default_library",
-        "//metropolis/node/core/localstorage/declarative:go_default_library",
         "//metropolis/node/core/network:go_default_library",
         "//metropolis/pkg/supervisor:go_default_library",
         "//metropolis/proto/api:go_default_library",
@@ -20,6 +18,7 @@
         "@com_github_cenkalti_backoff_v4//:go_default_library",
         "@com_github_golang_protobuf//proto:go_default_library",
         "@io_etcd_go_etcd//clientv3:go_default_library",
+        "@org_golang_google_protobuf//proto:go_default_library",
         "@org_golang_x_sys//unix:go_default_library",
     ],
 )
diff --git a/metropolis/node/core/cluster/manager.go b/metropolis/node/core/cluster/manager.go
index 1df9bac..6d6d592 100644
--- a/metropolis/node/core/cluster/manager.go
+++ b/metropolis/node/core/cluster/manager.go
@@ -18,23 +18,18 @@
 
 import (
 	"context"
-	"crypto/x509"
-	"encoding/pem"
 	"fmt"
 	"io/ioutil"
 	"os"
-	"strings"
 	"sync"
 	"time"
 
 	"github.com/cenkalti/backoff/v4"
-	"github.com/golang/protobuf/proto"
 	"go.etcd.io/etcd/clientv3"
+	"google.golang.org/protobuf/proto"
 
-	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/node/core/consensus"
 	"source.monogon.dev/metropolis/node/core/localstorage"
-	"source.monogon.dev/metropolis/node/core/localstorage/declarative"
 	"source.monogon.dev/metropolis/node/core/network"
 	"source.monogon.dev/metropolis/pkg/supervisor"
 	apb "source.monogon.dev/metropolis/proto/api"
@@ -73,9 +68,6 @@
 	// reaches a final state (Running or Failed respectively).
 	stateWaiters []chan bool
 
-	// goldenTicket is the Golden Ticket present in the enrolment config, if any.
-	goldenTicket *apb.GoldenTicket
-
 	// consensus is the spawned etcd/consensus service, if the Manager brought up a Node that should run one.
 	consensus *consensus.Service
 }
@@ -98,9 +90,6 @@
 	// StateCreatingCluster is when the Manager attempts to create a new cluster - this happens when a node is started
 	// with no EnrolmentConfig.
 	StateCreatingCluster
-	// StateCharlie is when the Manager uses the Golden Ticket debug/stopgap system to join an already
-	// existing cluster. This mechanism will be removed before the first Metropolis release.
-	StateCharlie
 	// StateRunning is when the Manager successfully got the node to be part of a cluster. stateRunningNode is valid.
 	StateRunning
 	// StateFailed is when the Manager failed to ge the node to be part of a cluster.
@@ -113,8 +102,6 @@
 		return "New"
 	case StateCreatingCluster:
 		return "CreatingCluster"
-	case StateCharlie:
-		return "Charlie"
 	case StateRunning:
 		return "Running"
 	case StateFailed:
@@ -126,9 +113,8 @@
 
 // allowedTransition describes all allowed state transitions (map[From][]To).
 var allowedTransitions = map[State][]State{
-	StateNew:             {StateCreatingCluster, StateCharlie},
+	StateNew:             {StateCreatingCluster},
 	StateCreatingCluster: {StateRunning, StateFailed},
-	StateCharlie:         {StateRunning, StateFailed},
 }
 
 // allowed returns whether a transition from a state to another state is allowed (ie. is defined in allowedTransitions).
@@ -221,8 +207,6 @@
 			err = m.stateNew(ctx)
 		case StateCreatingCluster:
 			err = m.stateCreatingCluster(ctx)
-		case StateCharlie:
-			err = m.stateCharlie(ctx)
 		default:
 			done = true
 			break
@@ -261,40 +245,37 @@
 func (m *Manager) stateNew(ctx context.Context) error {
 	supervisor.Logger(ctx).Info("Starting enrolment process...")
 
+	// STOPGAP when migrating to enrolment config and cluster lifecycle: always
+	// expect NodeParameters with ClusterBootstrap.
+
 	// Check for presence of EnrolmentConfig on ESP or in qemu firmware variables.
 	var configRaw []byte
-	configRaw, err := m.storageRoot.ESP.Enrolment.Read()
+	configRaw, err := m.storageRoot.ESP.NodeParameters.Read()
 	if err != nil && !os.IsNotExist(err) {
 		return fmt.Errorf("could not read local enrolment file: %w", err)
 	} else if err != nil {
-		configRaw, err = ioutil.ReadFile("/sys/firmware/qemu_fw_cfg/by_name/dev.monogon.metropolis/enrolment.pb/raw")
+		configRaw, err = ioutil.ReadFile("/sys/firmware/qemu_fw_cfg/by_name/dev.monogon.metropolis/parameters.pb/raw")
 		if err != nil && !os.IsNotExist(err) {
 			return fmt.Errorf("could not read firmware enrolment file: %w", err)
 		}
 	}
 
-	// If no enrolment file exists, we create a new cluster.
 	if configRaw == nil {
-		m.next(ctx, StateCreatingCluster)
-		return nil
+		return fmt.Errorf("no enrolment config present")
 	}
 
-	// Enrolment file exists, parse it.
-
-	enrolmentConfig := apb.EnrolmentConfig{}
-	if err := proto.Unmarshal(configRaw, &enrolmentConfig); err != nil {
-		return fmt.Errorf("could not unmarshal local enrolment file: %w", err)
+	parameters := &apb.NodeParameters{}
+	if err := proto.Unmarshal(configRaw, parameters); err != nil {
+		return fmt.Errorf("enrolment config could not get unmarshaled: %w", err)
 	}
 
-	// If no join ticket exists, we can't do anything yet.
-	if enrolmentConfig.GoldenTicket == nil {
-		return fmt.Errorf("joining a cluster without a golden ticket not yet implemented")
+	switch parameters.Cluster.(type) {
+	case *apb.NodeParameters_ClusterBootstrap_:
+	default:
+		return fmt.Errorf("enrolment config has no ClusterBootstrap: %w", err)
 	}
 
-	m.goldenTicket = enrolmentConfig.GoldenTicket
-
-	// Otherwise, we begin enrolling with the Golden Ticket.
-	m.next(ctx, StateCharlie)
+	m.next(ctx, StateCreatingCluster)
 	return nil
 }
 
@@ -380,121 +361,6 @@
 	return nil
 }
 
-// stateCharlie is used to join an existing cluster via the GoldenTicket mechanism. This mechanism is temporarily
-// implemented in Metropolis in order to allow for testing multi-node clusters without a TPM attestation flow implemented.
-// The Golden Ticket contains a pregenerated node certificate, etcd certificate, and other data that any node can
-// use to join the cluster.
-// Since this flow is temporary, it has a slight impedance mismatch with methods exposed by localstorage, node, etc.,
-// and the resulting sequencing is a bit odd:
-//  - the {node,etcd} certificates/keys are loaded (this already dictates the new node name, as the node name is based
-//    off of the node public key)
-//  - local storage is initialized, a local/cluster unlock keypair is generated
-//  - etcd keys are manually saved to localstorage (vs. being generated locally by CA)
-//  - an etcd/consensus member is started, knowing that the remote member was already generated when the golden ticket
-//    was generated (vs. being created now by an RPC call, via an promote-node-to-etcd-member flow)
-//  - the node is then promoted to a consensus member and kubernetes worker, its clusterunlock key is set, and then it
-//    is saved to etcd.
-// As such, in this flow, we first create an etcd member (on goldenticket generation), and then only create a new
-// Metropolis node (when the goldenticket is used).
-func (m *Manager) stateCharlie(ctx context.Context) error {
-	t := m.goldenTicket
-	nodeCert, err := x509.ParseCertificate(t.NodeCert)
-	if err != nil {
-		return fmt.Errorf("parsing node certificate from ticket: %w", err)
-	}
-
-	supervisor.Logger(ctx).Info("Joining cluster: waiting for IP address...")
-	ip, err := m.networkService.GetIP(ctx, true)
-	if err != nil {
-		return fmt.Errorf("when getting IP address: %w", err)
-	}
-	supervisor.Logger(ctx).Info("Joining cluster: got IP address %s", ip.String())
-
-	supervisor.Logger(ctx).Info("Joining cluster: initializing storage...")
-	cuk, err := m.storageRoot.Data.MountNew(&m.storageRoot.ESP.LocalUnlock)
-	if err != nil {
-		return fmt.Errorf("when making new data partition: %w", err)
-	}
-	supervisor.Logger(ctx).Info("Joining cluster: storage initialized")
-	node := NewNode(cuk, *ip, *nodeCert)
-
-	// Save etcd PKI to disk.
-	for _, f := range []struct {
-		target    declarative.FilePlacement
-		data      []byte
-		blockType string
-	}{
-		{m.storageRoot.Data.Etcd.PeerPKI.Key, t.EtcdClientKey, "PRIVATE KEY"},
-		{m.storageRoot.Data.Etcd.PeerPKI.Certificate, t.EtcdClientCert, "CERTIFICATE"},
-		{m.storageRoot.Data.Etcd.PeerPKI.CACertificate, t.EtcdCaCert, "CERTIFICATE"},
-	} {
-		if err := f.target.Write(pem.EncodeToMemory(&pem.Block{Type: f.blockType, Bytes: f.data}), 0600); err != nil {
-			return fmt.Errorf("when writing etcd PKI data: %w", err)
-		}
-	}
-	if err := m.storageRoot.Data.Etcd.PeerCRL.Write(t.EtcdCrl, 0600); err != nil {
-		return fmt.Errorf("when writing etcd CRL: %w", err)
-	}
-
-	https := func(p *apb.GoldenTicket_EtcdPeer) string {
-		return fmt.Sprintf("%s=https://%s:%d", p.Name, p.Address, common.ConsensusPort)
-	}
-	var initialCluster []string
-	for _, p := range t.Peers {
-		initialCluster = append(initialCluster, https(p))
-	}
-	initialCluster = append(initialCluster, https(t.This))
-
-	supervisor.Logger(ctx).Infof("Joining cluster: starting etcd join, name: %s, initial_cluster: %s", node.ID(), strings.Join(initialCluster, ","))
-	m.consensus = consensus.New(consensus.Config{
-		Data:           &m.storageRoot.Data.Etcd,
-		Ephemeral:      &m.storageRoot.Ephemeral.Consensus,
-		Name:           node.ID(),
-		InitialCluster: strings.Join(initialCluster, ","),
-		ExternalHost:   ip.String(),
-		ListenHost:     ip.String(),
-	})
-
-	if err := supervisor.Run(ctx, "consensus", m.consensus.Run); err != nil {
-		return fmt.Errorf("when starting consensus: %w", err)
-	}
-
-	// TODO(q3k): make timeout configurable?
-	ctxT, ctxC := context.WithTimeout(ctx, 5*time.Second)
-	defer ctxC()
-
-	supervisor.Logger(ctx).Info("Joining cluster: waiting for consensus...")
-	if err := m.consensus.WaitReady(ctxT); err != nil {
-		return fmt.Errorf("consensus service failed to become ready: %w", err)
-	}
-
-	// Configure node to be a consensus member and kubernetes worker. In the future, different nodes will have
-	// different roles, but for now they're all symmetrical.
-	_, consensusName, err := m.consensus.MemberInfo(ctx)
-	if err != nil {
-		return fmt.Errorf("could not get consensus MemberInfo: %w", err)
-	}
-	if err := node.MakeConsensusMember(consensusName); err != nil {
-		return fmt.Errorf("could not make new node into consensus member: %w", err)
-	}
-	if err := node.MakeKubernetesWorker(node.ID()); err != nil {
-		return fmt.Errorf("could not make new node into kubernetes worker: %w", err)
-	}
-
-	// Save node into etcd.
-	supervisor.Logger(ctx).Info("Creating new cluster: storing first node...")
-	if err := node.Store(ctx, m.consensus.KV("cluster", "enrolment")); err != nil {
-		return fmt.Errorf("could not save new node: %w", err)
-	}
-
-	m.stateLock.Lock()
-	m.stateRunningNode = node
-	m.stateLock.Unlock()
-
-	m.next(ctx, StateRunning)
-	return nil
-}
-
 // Node returns the Node that the Manager brought into a cluster, or nil if the Manager is not Running.
 // This is safe to call from any goroutine.
 func (m *Manager) Node() *Node {
diff --git a/metropolis/node/core/debug_service.go b/metropolis/node/core/debug_service.go
index 62a9a8a..6dee3d9 100644
--- a/metropolis/node/core/debug_service.go
+++ b/metropolis/node/core/debug_service.go
@@ -18,16 +18,11 @@
 
 import (
 	"context"
-	"crypto/x509"
-	"fmt"
-	"net"
 
 	"google.golang.org/grpc/codes"
 	"google.golang.org/grpc/status"
 
-	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/node/core/cluster"
-	"source.monogon.dev/metropolis/node/core/consensus/ca"
 	"source.monogon.dev/metropolis/node/kubernetes"
 	"source.monogon.dev/metropolis/pkg/logtree"
 	apb "source.monogon.dev/metropolis/proto/api"
@@ -44,61 +39,6 @@
 	logtree    *logtree.LogTree
 }
 
-func (s *debugService) GetGoldenTicket(ctx context.Context, req *apb.GetGoldenTicketRequest) (*apb.GetGoldenTicketResponse, error) {
-	ip := net.ParseIP(req.ExternalIp)
-	if ip == nil {
-		return nil, status.Errorf(codes.InvalidArgument, "could not parse IP %q", req.ExternalIp)
-	}
-	this := s.cluster.Node()
-
-	certRaw, key, err := s.nodeCertificate()
-	if err != nil {
-		return nil, status.Errorf(codes.Unavailable, "failed to generate node certificate: %v", err)
-	}
-	cert, err := x509.ParseCertificate(certRaw)
-	if err != nil {
-		panic(err)
-	}
-	kv := s.cluster.ConsensusKVRoot()
-	ca, err := ca.Load(ctx, kv)
-	if err != nil {
-		return nil, status.Errorf(codes.Unavailable, "could not load CA: %v", err)
-	}
-	etcdCert, etcdKey, err := ca.Issue(ctx, kv, cert.Subject.CommonName, ip)
-	if err != nil {
-		return nil, status.Errorf(codes.Unavailable, "could not generate etcd peer certificate: %v", err)
-	}
-	etcdCRL, err := ca.GetCurrentCRL(ctx, kv)
-	if err != nil {
-		return nil, status.Errorf(codes.Unavailable, "could not get etcd CRL: %v", err)
-	}
-
-	// Add new etcd member to etcd cluster.
-	etcd := s.cluster.ConsensusCluster()
-	etcdAddr := fmt.Sprintf("https://%s:%d", ip.String(), common.ConsensusPort)
-	_, err = etcd.MemberAddAsLearner(ctx, []string{etcdAddr})
-	if err != nil {
-		return nil, status.Errorf(codes.Unavailable, "could not add as new etcd consensus member: %v", err)
-	}
-
-	return &apb.GetGoldenTicketResponse{
-		Ticket: &apb.GoldenTicket{
-			EtcdCaCert:     ca.CACertRaw,
-			EtcdClientCert: etcdCert,
-			EtcdClientKey:  etcdKey,
-			EtcdCrl:        etcdCRL,
-			Peers: []*apb.GoldenTicket_EtcdPeer{
-				{Name: this.ID(), Address: this.Address().String()},
-			},
-			This: &apb.GoldenTicket_EtcdPeer{Name: cert.Subject.CommonName, Address: ip.String()},
-
-			NodeId:   cert.Subject.CommonName,
-			NodeCert: certRaw,
-			NodeKey:  key,
-		},
-	}, nil
-}
-
 func (s *debugService) GetDebugKubeconfig(ctx context.Context, req *apb.GetDebugKubeconfigRequest) (*apb.GetDebugKubeconfigResponse, error) {
 	return s.kubernetes.GetDebugKubeconfig(ctx, req)
 }
diff --git a/metropolis/node/core/localstorage/storage.go b/metropolis/node/core/localstorage/storage.go
index 73d33e1..a1f567b 100644
--- a/metropolis/node/core/localstorage/storage.go
+++ b/metropolis/node/core/localstorage/storage.go
@@ -63,10 +63,8 @@
 // ESPDirectory is the EFI System Partition.
 type ESPDirectory struct {
 	declarative.Directory
-	LocalUnlock ESPLocalUnlockFile `file:"local_unlock.bin"`
-	// Enrolment is the configuration/provisioning file for this node, containing information required to begin
-	// joining the cluster.
-	Enrolment declarative.File `file:"enrolment.pb"`
+	LocalUnlock    ESPLocalUnlockFile `file:"local_unlock.bin"`
+	NodeParameters ESPNodeParameters  `file:"parameters.pb"`
 }
 
 // ESPLocalUnlockFile is the localUnlock file, encrypted by the TPM of this node. After decrypting by the TPM it is used
@@ -75,6 +73,13 @@
 	declarative.File
 }
 
+// ESPNodeParameters is the configuration for this node when first
+// bootstrapping a cluster or registering into an existing one. It's a
+// api.NodeParameters protobuf message.
+type ESPNodeParameters struct {
+	declarative.File
+}
+
 // DataDirectory is an xfs partition mounted via cryptsetup/LUKS, with a key derived from {global,local}Unlock keys.
 type DataDirectory struct {
 	declarative.Directory
diff --git a/metropolis/proto/api/BUILD.bazel b/metropolis/proto/api/BUILD.bazel
index e7b4cc7..5004440 100644
--- a/metropolis/proto/api/BUILD.bazel
+++ b/metropolis/proto/api/BUILD.bazel
@@ -6,8 +6,8 @@
     name = "api_proto",
     srcs = [
         "aaa.proto",
+        "configuration.proto",
         "debug.proto",
-        "enrolment.proto",
     ],
     visibility = ["//visibility:public"],
 )
diff --git a/metropolis/proto/api/configuration.proto b/metropolis/proto/api/configuration.proto
new file mode 100644
index 0000000..8c953ed
--- /dev/null
+++ b/metropolis/proto/api/configuration.proto
@@ -0,0 +1,36 @@
+// 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.
+
+syntax = "proto3";
+package metropolis.proto.api;
+option go_package = "source.monogon.dev/metropolis/proto/api";
+
+// NodeParameters is the data with which a Node is set booted. It contains the
+// configuration required for a node to either bootstrap a new cluster, or
+// register into an existing one.
+// It is serialized into a proto message and supplied to Metropolis in an
+// implementation-specific way (currently: either on ESP partition or via qemu
+// fw_cfg).
+message NodeParameters {
+    message ClusterBootstrap {
+    }
+    message ClusterRegister {
+    }
+    oneof cluster {
+        ClusterBootstrap cluster_bootstrap = 1;
+        ClusterRegister cluster_register = 2;
+    }
+}
diff --git a/metropolis/proto/api/debug.proto b/metropolis/proto/api/debug.proto
index fddd750..25e369e 100644
--- a/metropolis/proto/api/debug.proto
+++ b/metropolis/proto/api/debug.proto
@@ -18,8 +18,6 @@
 package metropolis.proto.api;
 option go_package = "source.monogon.dev/metropolis/proto/api";
 
-import "metropolis/proto/api/enrolment.proto";
-
 // NodeDebugService exposes debug and testing endpoints that allow introspection into a running Metropolis node.
 // It is not authenticated and will be disabled in production. It is currently consumed by metropolis/cli/dbg and
 // by tests.
@@ -41,10 +39,6 @@
     //
     // TODO(q3k): move method and its related messages to the non-debug node endpoint once we have one.
     rpc GetLogs(GetLogsRequest) returns (stream GetLogsResponse);
-
-    // GetGoldenTicket requests a 'golden ticket' which can be used to enroll any node into the cluster.
-    // This bypasses integrity checks.
-    rpc GetGoldenTicket(GetGoldenTicketRequest) returns (GetGoldenTicketResponse);
 }
 
 
@@ -152,13 +146,3 @@
         Raw raw = 3;
     }
 }
-
-message GetGoldenTicketRequest {
-    // IP address at which the new node will run.
-    string external_ip = 1;
-}
-
-message GetGoldenTicketResponse {
-    // Ticket to use in the new node's EnrolmentConfig.
-    GoldenTicket ticket = 1;
-}
diff --git a/metropolis/proto/api/enrolment.proto b/metropolis/proto/api/enrolment.proto
deleted file mode 100644
index cf66adb..0000000
--- a/metropolis/proto/api/enrolment.proto
+++ /dev/null
@@ -1,64 +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.
-
-syntax = "proto3";
-package metropolis.proto.api;
-option go_package = "source.monogon.dev/metropolis/proto/api";
-
-// EnrolmentConfig is the single Metropolis node boot configuration file
-// contained in the ESP. It configures the way the node will start up (what
-// cluster it will join/enroll into/create).
-message EnrolmentConfig {
-    // Debug/temporary cluster enrolment method. If set, the node will attempt to enroll into the
-    // cluster that this ticket was generated for. Otherwise, a new cluster will be created.
-    GoldenTicket golden_ticket = 1;
-
-    // Filled in by node after it is enrolled
-    string node_id = 2;
-}
-
-// GoldenTicket is a ticket that allows any node to enroll into a cluster, bypassing any integrity
-// checks.
-//
-// Currently, enrolling into a cluster does not use a TPM-based workflow, and instead
-// bases on a simplified workflow of joining consensus by being started with a
-// TLS client certificate. This is a short-circuit fix to allow multi-node
-// clusters for testing before we design the final cluster node lifecycle system.
-message GoldenTicket {
-    // Etcd peer CA certificate.
-    bytes etcd_ca_cert = 1;
-    // Etcd peer client certificate.
-    bytes etcd_client_cert = 2;
-    // Etcd peer client key.
-    bytes etcd_client_key = 3;
-    // Initial etcd peer CRL.
-    bytes etcd_crl = 4;
-
-    message EtcdPeer {
-        string name = 1;
-        string address = 2;
-    }
-    // All other current etcd peers in the cluster.
-    repeated EtcdPeer peers = 5;
-    // The peer that this node should start running.
-    EtcdPeer this = 6;
-
-    // Node configuration. Currently unused (in the future, this will be used to run a node
-    // management service separate from etcd clustering).
-    string node_id = 7;
-    bytes node_cert = 8;
-    bytes node_key = 9;
-}
diff --git a/metropolis/test/e2e/main_test.go b/metropolis/test/e2e/main_test.go
index 71713cc..3e4ffb0 100644
--- a/metropolis/test/e2e/main_test.go
+++ b/metropolis/test/e2e/main_test.go
@@ -82,7 +82,15 @@
 	procExit := make(chan struct{})
 
 	go func() {
-		if err := launch.Launch(ctx, launch.Options{Ports: portMap, SerialPort: os.Stdout}); err != nil {
+		if err := launch.Launch(ctx, launch.Options{
+			Ports:      portMap,
+			SerialPort: os.Stdout,
+			NodeParameters: &apb.NodeParameters{
+				Cluster: &apb.NodeParameters_ClusterBootstrap_{
+					ClusterBootstrap: &apb.NodeParameters_ClusterBootstrap{},
+				},
+			},
+		}); err != nil {
 			panic(err)
 		}
 		close(procExit)
diff --git a/metropolis/test/launch/cli/launch-multi2/BUILD.bazel b/metropolis/test/launch/cli/launch-multi2/BUILD.bazel
index 91efdd7..51118cf 100644
--- a/metropolis/test/launch/cli/launch-multi2/BUILD.bazel
+++ b/metropolis/test/launch/cli/launch-multi2/BUILD.bazel
@@ -5,14 +5,7 @@
     srcs = ["main.go"],
     importpath = "source.monogon.dev/metropolis/test/launch/cli/launch-multi2",
     visibility = ["//visibility:private"],
-    deps = [
-        "//metropolis/node:go_default_library",
-        "//metropolis/pkg/logbuffer:go_default_library",
-        "//metropolis/proto/api:go_default_library",
-        "//metropolis/test/launch:go_default_library",
-        "@com_github_grpc_ecosystem_go_grpc_middleware//retry:go_default_library",
-        "@org_golang_google_grpc//:go_default_library",
-    ],
+    deps = ["//metropolis/pkg/logbuffer:go_default_library"],
 )
 
 go_binary(
diff --git a/metropolis/test/launch/cli/launch-multi2/main.go b/metropolis/test/launch/cli/launch-multi2/main.go
index a2e00bc..d6c5f05 100644
--- a/metropolis/test/launch/cli/launch-multi2/main.go
+++ b/metropolis/test/launch/cli/launch-multi2/main.go
@@ -17,22 +17,12 @@
 package main
 
 import (
-	"context"
 	"fmt"
 	"io"
 	"log"
 	"os"
-	"os/signal"
-	"syscall"
-	"time"
 
-	grpcretry "github.com/grpc-ecosystem/go-grpc-middleware/retry"
-	"google.golang.org/grpc"
-
-	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/pkg/logbuffer"
-	apb "source.monogon.dev/metropolis/proto/api"
-	"source.monogon.dev/metropolis/test/launch"
 )
 
 // prefixedStdout is a os.Stdout proxy that prefixes every line with a constant
@@ -57,77 +47,5 @@
 }
 
 func main() {
-	sigs := make(chan os.Signal, 1)
-	signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM)
-	ctx, cancel := context.WithCancel(context.Background())
-	go func() {
-		<-sigs
-		cancel()
-	}()
-	sw0, vm0, err := launch.NewSocketPair()
-	if err != nil {
-		log.Fatalf("Failed to create network pipe: %v\n", err)
-	}
-	sw1, vm1, err := launch.NewSocketPair()
-	if err != nil {
-		log.Fatalf("Failed to create network pipe: %v\n", err)
-	}
-
-	go func() {
-		if err := launch.Launch(ctx, launch.Options{
-			ConnectToSocket: vm0,
-			SerialPort:      prefixedStdout("1| "),
-		}); err != nil {
-			log.Fatalf("Failed to launch vm0: %v", err)
-		}
-	}()
-	nanoswitchPortMap := make(launch.PortMap)
-	identityPorts := []uint16{
-		common.ExternalServicePort,
-		common.DebugServicePort,
-		common.KubernetesAPIPort,
-	}
-	for _, port := range identityPorts {
-		nanoswitchPortMap[port] = port
-	}
-	go func() {
-		opts := []grpcretry.CallOption{
-			grpcretry.WithBackoff(grpcretry.BackoffExponential(100 * time.Millisecond)),
-		}
-		conn, err := nanoswitchPortMap.DialGRPC(common.DebugServicePort, grpc.WithInsecure(),
-			grpc.WithUnaryInterceptor(grpcretry.UnaryClientInterceptor(opts...)))
-		if err != nil {
-			panic(err)
-		}
-		defer conn.Close()
-		debug := apb.NewNodeDebugServiceClient(conn)
-		res, err := debug.GetGoldenTicket(ctx, &apb.GetGoldenTicketRequest{
-			// HACK: this is assigned by DHCP, and we assume that everything goes well.
-			ExternalIp: "10.1.0.3",
-		}, grpcretry.WithMax(10))
-		if err != nil {
-			log.Fatalf("Failed to get golden ticket: %v", err)
-		}
-
-		ec := &apb.EnrolmentConfig{
-			GoldenTicket: res.Ticket,
-		}
-
-		if err := launch.Launch(ctx, launch.Options{
-			ConnectToSocket: vm1,
-			EnrolmentConfig: ec,
-			SerialPort:      prefixedStdout("2| "),
-		}); err != nil {
-			log.Fatalf("Failed to launch vm1: %v", err)
-		}
-	}()
-	if err := launch.RunMicroVM(ctx, &launch.MicroVMOptions{
-		SerialPort:             os.Stdout,
-		KernelPath:             "metropolis/test/ktest/linux-testing.elf",
-		InitramfsPath:          "metropolis/test/nanoswitch/initramfs.lz4",
-		ExtraNetworkInterfaces: []*os.File{sw0, sw1},
-		PortMap:                nanoswitchPortMap,
-	}); err != nil {
-		log.Fatalf("Failed to launch nanoswitch: %v", err)
-	}
+	log.Fatal("unimplemented")
 }
diff --git a/metropolis/test/launch/cli/launch/BUILD.bazel b/metropolis/test/launch/cli/launch/BUILD.bazel
index 43c02b6..824b2ff 100644
--- a/metropolis/test/launch/cli/launch/BUILD.bazel
+++ b/metropolis/test/launch/cli/launch/BUILD.bazel
@@ -5,7 +5,10 @@
     srcs = ["main.go"],
     importpath = "source.monogon.dev/metropolis/test/launch/cli/launch",
     visibility = ["//visibility:private"],
-    deps = ["//metropolis/test/launch:go_default_library"],
+    deps = [
+        "//metropolis/proto/api:go_default_library",
+        "//metropolis/test/launch:go_default_library",
+    ],
 )
 
 go_binary(
diff --git a/metropolis/test/launch/cli/launch/main.go b/metropolis/test/launch/cli/launch/main.go
index a855f73..cb85c88 100644
--- a/metropolis/test/launch/cli/launch/main.go
+++ b/metropolis/test/launch/cli/launch/main.go
@@ -23,6 +23,7 @@
 	"os/signal"
 	"syscall"
 
+	apb "source.monogon.dev/metropolis/proto/api"
 	"source.monogon.dev/metropolis/test/launch"
 )
 
@@ -34,7 +35,15 @@
 		<-sigs
 		cancel()
 	}()
-	if err := launch.Launch(ctx, launch.Options{Ports: launch.IdentityPortMap(launch.NodePorts), SerialPort: os.Stdout}); err != nil {
+	if err := launch.Launch(ctx, launch.Options{
+		Ports:      launch.IdentityPortMap(launch.NodePorts),
+		SerialPort: os.Stdout,
+		NodeParameters: &apb.NodeParameters{
+			Cluster: &apb.NodeParameters_ClusterBootstrap_{
+				ClusterBootstrap: &apb.NodeParameters_ClusterBootstrap{},
+			},
+		},
+	}); err != nil {
 		if err == ctx.Err() {
 			return
 		}
diff --git a/metropolis/test/launch/launch.go b/metropolis/test/launch/launch.go
index 6e6891a..3a444ef 100644
--- a/metropolis/test/launch/launch.go
+++ b/metropolis/test/launch/launch.go
@@ -130,8 +130,8 @@
 	// It can be set to an existing file descriptor (like os.Stdout/os.Stderr) or any Go structure implementing this interface.
 	SerialPort io.ReadWriter
 
-	// EnrolmentConfig is passed into the VM and subsequently used for bootstrapping if no enrolment config is built-in
-	EnrolmentConfig *apb.EnrolmentConfig
+	// 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
@@ -254,16 +254,16 @@
 		qemuArgs = append(qemuArgs, "-no-reboot")
 	}
 
-	if options.EnrolmentConfig != nil {
-		enrolmentConfigPath := filepath.Join(tempDir, "enrolment.pb")
-		enrolmentConfigRaw, err := proto.Marshal(options.EnrolmentConfig)
+	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 enrolment config: %w", err)
+			return fmt.Errorf("failed to encode node paraeters: %w", err)
 		}
-		if err := ioutil.WriteFile(enrolmentConfigPath, enrolmentConfigRaw, 0644); err != nil {
-			return fmt.Errorf("failed to write enrolment config: %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/enrolment.pb,file="+enrolmentConfigPath)
+		qemuArgs = append(qemuArgs, "-fw_cfg", "name=dev.monogon.metropolis/parameters.pb,file="+parametersPath)
 	}
 
 	// Start TPM emulator as a subprocess
@@ -498,7 +498,15 @@
 	}
 
 	go func() {
-		if err := Launch(ctx, Options{ConnectToSocket: vmPorts[0]}); err != nil {
+		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.
@@ -532,23 +540,7 @@
 	debug := apb.NewNodeDebugServiceClient(conn)
 
 	if opts.NumNodes == 2 {
-		res, err := debug.GetGoldenTicket(ctx, &apb.GetGoldenTicketRequest{
-			// HACK: this is assigned by DHCP, and we assume that everything goes well.
-			ExternalIp: "10.1.0.3",
-		}, grpcretry.WithMax(10))
-		if err != nil {
-			return nil, nil, fmt.Errorf("failed to get golden ticket: %w", err)
-		}
-
-		ec := &apb.EnrolmentConfig{
-			GoldenTicket: res.Ticket,
-		}
-
-		go func() {
-			if err := Launch(ctx, Options{ConnectToSocket: vmPorts[1], EnrolmentConfig: ec}); err != nil {
-				log.Printf("Failed to launch vm1: %v", err)
-			}
-		}()
+		return nil, nil, fmt.Errorf("multinode unimplemented")
 	}
 
 	return debug, portMap, nil