m/node: allow specifying node labels during cluster bootstrap

We also drive-by refactor ProvideBootstrapData to take a structure
instead of a bunch of unnamed arguments.

Change-Id: I8d876fd726fa87420789513540b20f523994d801
Reviewed-on: https://review.monogon.dev/c/monogon/+/3103
Tested-by: Jenkins CI
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/core/cluster/BUILD.bazel b/metropolis/node/core/cluster/BUILD.bazel
index e002a31..4da6c76 100644
--- a/metropolis/node/core/cluster/BUILD.bazel
+++ b/metropolis/node/core/cluster/BUILD.bazel
@@ -11,6 +11,7 @@
     importpath = "source.monogon.dev/metropolis/node/core/cluster",
     visibility = ["//metropolis/node/core:__subpackages__"],
     deps = [
+        "//metropolis/node",
         "//metropolis/node/core/curator",
         "//metropolis/node/core/curator/proto/api",
         "//metropolis/node/core/identity",
diff --git a/metropolis/node/core/cluster/cluster_bootstrap.go b/metropolis/node/core/cluster/cluster_bootstrap.go
index f328815..3e5f745 100644
--- a/metropolis/node/core/cluster/cluster_bootstrap.go
+++ b/metropolis/node/core/cluster/cluster_bootstrap.go
@@ -26,8 +26,10 @@
 
 	"google.golang.org/protobuf/proto"
 
+	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/node/core/curator"
 	"source.monogon.dev/metropolis/node/core/identity"
+	"source.monogon.dev/metropolis/node/core/roleserve"
 	"source.monogon.dev/metropolis/pkg/supervisor"
 
 	apb "source.monogon.dev/metropolis/proto/api"
@@ -132,7 +134,38 @@
 	}
 	supervisor.Logger(ctx).Infof("Saved bootstrapped node's credentials.")
 
-	m.roleServer.ProvideBootstrapData(priv, ownerKey, cuk, nuk, jpriv, cc, tpmUsage)
+	labels := make(map[string]string)
+	if l := bootstrap.Labels; l != nil {
+		if nlabels := len(l.Pairs); nlabels > common.MaxLabelsPerNode {
+			supervisor.Logger(ctx).Warningf("Too many labels (%d, limit %d), truncating...", nlabels, common.MaxLabelsPerNode)
+			l.Pairs = l.Pairs[:common.MaxLabelsPerNode]
+		}
+		for _, pair := range l.Pairs {
+			if err := common.ValidateLabel(pair.Key); err != nil {
+				supervisor.Logger(ctx).Warningf("Skipping label %q/%q: key invalid: %v", pair.Key, pair.Value, err)
+				continue
+			}
+			if err := common.ValidateLabel(pair.Value); err != nil {
+				supervisor.Logger(ctx).Warningf("Skipping label %q/%q: value invalid: %v", pair.Key, pair.Value, err)
+				continue
+			}
+			if _, ok := labels[pair.Key]; ok {
+				supervisor.Logger(ctx).Warningf("Label %q/%q: repeated key, overwriting previous value", pair.Key, pair.Value)
+			}
+			labels[pair.Key] = pair.Value
+		}
+	}
+
+	bd := roleserve.BootstrapData{}
+	bd.Node.PrivateKey = priv
+	bd.Node.ClusterUnlockKey = cuk
+	bd.Node.NodeUnlockKey = nuk
+	bd.Node.JoinKey = jpriv
+	bd.Node.TPMUsage = tpmUsage
+	bd.Node.Labels = labels
+	bd.Cluster.InitialOwnerKey = ownerKey
+	bd.Cluster.Configuration = cc
+	m.roleServer.ProvideBootstrapData(&bd)
 
 	supervisor.Signal(ctx, supervisor.SignalHealthy)
 	supervisor.Signal(ctx, supervisor.SignalDone)
diff --git a/metropolis/node/core/curator/impl_leader_test.go b/metropolis/node/core/curator/impl_leader_test.go
index d3c3de1..5c3c337 100644
--- a/metropolis/node/core/curator/impl_leader_test.go
+++ b/metropolis/node/core/curator/impl_leader_test.go
@@ -107,7 +107,12 @@
 	if err != nil {
 		t.Fatalf("could not generate node keypair: %v", err)
 	}
-	cNode := NewNodeForBootstrap(nil, nodePub, nodeJoinPub, cpb.NodeTPMUsage_NODE_TPM_PRESENT_AND_USED)
+	cNode := NewNodeForBootstrap(&NewNodeData{
+		CUK:      nil,
+		Pubkey:   nodePub,
+		JPub:     nodeJoinPub,
+		TPMUsage: cpb.NodeTPMUsage_NODE_TPM_PRESENT_AND_USED,
+	})
 
 	// Here we would enable the leader node's roles. But for tests, we don't enable
 	// any.
diff --git a/metropolis/node/core/curator/state_node.go b/metropolis/node/core/curator/state_node.go
index 18ea200..5e5ef2e 100644
--- a/metropolis/node/core/curator/state_node.go
+++ b/metropolis/node/core/curator/state_node.go
@@ -106,17 +106,26 @@
 	labels map[string]string
 }
 
+type NewNodeData struct {
+	CUK      []byte
+	Pubkey   []byte
+	JPub     []byte
+	TPMUsage cpb.NodeTPMUsage
+	Labels   map[string]string
+}
+
 // NewNodeForBootstrap creates a brand new node without regard for any other
 // cluster state.
 //
 // This can only be used by the cluster bootstrap logic.
-func NewNodeForBootstrap(cuk, pubkey, jpub []byte, tpmUsage cpb.NodeTPMUsage) Node {
+func NewNodeForBootstrap(n *NewNodeData) Node {
 	return Node{
-		clusterUnlockKey: cuk,
-		pubkey:           pubkey,
-		jkey:             jpub,
+		clusterUnlockKey: n.CUK,
+		pubkey:           n.Pubkey,
+		jkey:             n.JPub,
 		state:            cpb.NodeState_NODE_STATE_UP,
-		tpmUsage:         tpmUsage,
+		tpmUsage:         n.TPMUsage,
+		labels:           n.Labels,
 	}
 }
 
diff --git a/metropolis/node/core/roleserve/roleserve.go b/metropolis/node/core/roleserve/roleserve.go
index 718c394..ad8207b 100644
--- a/metropolis/node/core/roleserve/roleserve.go
+++ b/metropolis/node/core/roleserve/roleserve.go
@@ -81,7 +81,7 @@
 	Config
 
 	KubernetesStatus      memory.Value[*KubernetesStatus]
-	bootstrapData         memory.Value[*bootstrapData]
+	bootstrapData         memory.Value[*BootstrapData]
 	localRoles            memory.Value[*cpb.NodeRoles]
 	podNetwork            memory.Value[*clusternet.Prefixes]
 	clusterDirectorySaved memory.Value[bool]
@@ -178,8 +178,38 @@
 	return s
 }
 
-func (s *Service) ProvideBootstrapData(privkey ed25519.PrivateKey, iok, cuk, nuk, jkey []byte, icc *curator.Cluster, tpmUsage cpb.NodeTPMUsage) {
-	pubkey := privkey.Public().(ed25519.PublicKey)
+// BootstrapData contains all the information needed to be injected into the
+// roleserver by the cluster bootstrap logic via ProvideBootstrapData.
+type BootstrapData struct {
+	// Data about the bootstrapping node.
+	Node struct {
+		PrivateKey ed25519.PrivateKey
+
+		// CUK/NUK for storage, if storage encryption is enabled.
+		ClusterUnlockKey []byte
+		NodeUnlockKey    []byte
+
+		// Join key for subsequent reboots.
+		JoinKey ed25519.PrivateKey
+
+		// Reported TPM usage by the node.
+		TPMUsage cpb.NodeTPMUsage
+
+		// Initial labels for the node.
+		Labels map[string]string
+	}
+	// Cluster-specific data.
+	Cluster struct {
+		// Public keys of initial owner of cluster. Used to escrow real user credentials
+		// during the takeownership metroctl process.
+		InitialOwnerKey []byte
+		// Initial cluster configuration.
+		Configuration *curator.Cluster
+	}
+}
+
+func (s *Service) ProvideBootstrapData(data *BootstrapData) {
+	pubkey := data.Node.PrivateKey.Public().(ed25519.PublicKey)
 	nid := identity.NodeID(pubkey)
 
 	// This is the first time we have the node ID, tell the resolver that it's
@@ -187,15 +217,7 @@
 	s.Resolver.AddOverride(nid, resolver.NodeByHostPort("127.0.0.1", uint16(common.CuratorServicePort)))
 	s.Resolver.AddEndpoint(resolver.NodeByHostPort("127.0.0.1", uint16(common.CuratorServicePort)))
 
-	s.bootstrapData.Set(&bootstrapData{
-		nodePrivateKey:              privkey,
-		initialOwnerKey:             iok,
-		clusterUnlockKey:            cuk,
-		nodeUnlockKey:               nuk,
-		nodePrivateJoinKey:          jkey,
-		initialClusterConfiguration: icc,
-		nodeTPMUsage:                tpmUsage,
-	})
+	s.bootstrapData.Set(data)
 }
 
 func (s *Service) ProvideRegisterData(credentials identity.NodeCredentials, directory *cpb.ClusterDirectory) {
diff --git a/metropolis/node/core/roleserve/values.go b/metropolis/node/core/roleserve/values.go
index 3b7ff25..91510ae 100644
--- a/metropolis/node/core/roleserve/values.go
+++ b/metropolis/node/core/roleserve/values.go
@@ -1,8 +1,6 @@
 package roleserve
 
 import (
-	"crypto/ed25519"
-
 	"google.golang.org/grpc"
 
 	"source.monogon.dev/metropolis/node/core/consensus"
@@ -11,24 +9,8 @@
 	"source.monogon.dev/metropolis/node/core/rpc"
 	"source.monogon.dev/metropolis/node/core/rpc/resolver"
 	"source.monogon.dev/metropolis/node/kubernetes"
-
-	cpb "source.monogon.dev/metropolis/proto/common"
 )
 
-// bootstrapData is an internal EventValue structure which is populated by the
-// Cluster Enrolment logic via ProvideBootstrapData. It contains data needed by
-// the control plane logic to go into bootstrap mode and bring up a control
-// plane from scratch.
-type bootstrapData struct {
-	nodePrivateKey              ed25519.PrivateKey
-	clusterUnlockKey            []byte
-	nodeUnlockKey               []byte
-	initialOwnerKey             []byte
-	nodePrivateJoinKey          ed25519.PrivateKey
-	initialClusterConfiguration *curator.Cluster
-	nodeTPMUsage                cpb.NodeTPMUsage
-}
-
 // localControlPlane is an internal EventValue structure which carries
 // information about whether the node has a locally running consensus and curator
 // service. When it does, the structure pointer inside the EventValue will be
diff --git a/metropolis/node/core/roleserve/worker_controlplane.go b/metropolis/node/core/roleserve/worker_controlplane.go
index d5a60fc..777f887 100644
--- a/metropolis/node/core/roleserve/worker_controlplane.go
+++ b/metropolis/node/core/roleserve/worker_controlplane.go
@@ -23,7 +23,7 @@
 // locally running Control Plane (Consensus and Curator service pair) if needed.
 //
 // The Control Plane will run under the following conditions:
-//   - This node has been started in BOOTSTRAP mode and bootstrapData was provided
+//   - This node has been started in BOOTSTRAP mode and BootstrapData was provided
 //     by the cluster enrolment logic. In this case, the Control Plane Worker will
 //     perform the required bootstrap steps, creating a local node with appropriate
 //     roles, and will start Consensus and the Curator.
@@ -38,8 +38,8 @@
 type workerControlPlane struct {
 	storageRoot *localstorage.Root
 
-	// bootstrapData will be read.
-	bootstrapData *memory.Value[*bootstrapData]
+	// BootstrapData will be read.
+	bootstrapData *memory.Value[*BootstrapData]
 	// localRoles will be read.
 	localRoles *memory.Value[*cpb.NodeRoles]
 	// resolver will be read and used to populate curatorConnection when
@@ -61,7 +61,7 @@
 	consensusConfig *consensus.Config
 	// bootstrap is set if this node should bootstrap consensus. It contains all
 	// data required to perform this bootstrap step.
-	bootstrap *bootstrapData
+	bootstrap *BootstrapData
 	existing  *curatorConnection
 }
 
@@ -94,13 +94,13 @@
 	// Channels are used as intermediaries between map stages and the final reduce,
 	// which is okay as long as the entire tree restarts simultaneously (which we
 	// ensure via RunGroup).
-	bootstrapDataC := make(chan *bootstrapData)
+	bootstrapDataC := make(chan *BootstrapData)
 	curatorConnectionC := make(chan *curatorConnection)
 	rolesC := make(chan *cpb.NodeRoles)
 
 	supervisor.RunGroup(ctx, map[string]supervisor.Runnable{
 		// Plain conversion from Event Value to channel.
-		"map-bootstrap-data": event.Pipe[*bootstrapData](s.bootstrapData, bootstrapDataC),
+		"map-bootstrap-data": event.Pipe[*BootstrapData](s.bootstrapData, bootstrapDataC),
 		// Plain conversion from Event Value to channel.
 		"map-curator-connection": event.Pipe[*curatorConnection](s.curatorConnection, curatorConnectionC),
 		// Plain conversion from Event Value to channel.
@@ -110,7 +110,7 @@
 			supervisor.Signal(ctx, supervisor.SignalHealthy)
 			var lr *cpb.NodeRoles
 			var cc *curatorConnection
-			var bd *bootstrapData
+			var bd *BootstrapData
 			for {
 				select {
 				case <-ctx.Done():
@@ -145,7 +145,7 @@
 						consensusConfig: &consensus.Config{
 							Data:           &s.storageRoot.Data.Etcd,
 							Ephemeral:      &s.storageRoot.Ephemeral.Consensus,
-							NodePrivateKey: bd.nodePrivateKey,
+							NodePrivateKey: bd.Node.PrivateKey,
 						},
 						bootstrap: bd,
 					})
@@ -269,10 +269,16 @@
 				// curator startup.
 				//
 				// TODO(q3k): collapse the curator bootstrap shenanigans into a single function.
-				npub := b.nodePrivateKey.Public().(ed25519.PublicKey)
-				jpub := b.nodePrivateJoinKey.Public().(ed25519.PublicKey)
+				npub := b.Node.PrivateKey.Public().(ed25519.PublicKey)
+				jpub := b.Node.JoinKey.Public().(ed25519.PublicKey)
 
-				n := curator.NewNodeForBootstrap(b.clusterUnlockKey, npub, jpub, b.nodeTPMUsage)
+				n := curator.NewNodeForBootstrap(&curator.NewNodeData{
+					CUK:      b.Node.ClusterUnlockKey,
+					Pubkey:   npub,
+					JPub:     jpub,
+					TPMUsage: b.Node.TPMUsage,
+					Labels:   b.Node.Labels,
+				})
 
 				// The first node always runs consensus.
 				join, err := st.AddNode(ctx, npub)
@@ -284,12 +290,12 @@
 				n.EnableKubernetesController()
 
 				var nodeCert []byte
-				caCert, nodeCert, err = curator.BootstrapNodeFinish(ctx, ckv, &n, b.initialOwnerKey, b.initialClusterConfiguration)
+				caCert, nodeCert, err = curator.BootstrapNodeFinish(ctx, ckv, &n, b.Cluster.InitialOwnerKey, b.Cluster.Configuration)
 				if err != nil {
 					return fmt.Errorf("while bootstrapping node: %w", err)
 				}
 				// ... and build new credentials from bootstrap step.
-				creds, err = identity.NewNodeCredentials(b.nodePrivateKey, nodeCert, caCert)
+				creds, err = identity.NewNodeCredentials(b.Node.PrivateKey, nodeCert, caCert)
 				if err != nil {
 					return fmt.Errorf("when creating bootstrap node credentials: %w", err)
 				}
@@ -297,12 +303,12 @@
 				if err = creds.Save(&s.storageRoot.Data.Node.Credentials); err != nil {
 					return fmt.Errorf("while saving node credentials: %w", err)
 				}
-				sc, err := s.storageRoot.ESP.Metropolis.SealedConfiguration.Unseal(b.nodeTPMUsage)
+				sc, err := s.storageRoot.ESP.Metropolis.SealedConfiguration.Unseal(b.Node.TPMUsage)
 				if err != nil {
 					return fmt.Errorf("reading sealed configuration failed: %w", err)
 				}
 				sc.ClusterCa = caCert
-				if err = s.storageRoot.ESP.Metropolis.SealedConfiguration.SealSecureBoot(sc, b.nodeTPMUsage); err != nil {
+				if err = s.storageRoot.ESP.Metropolis.SealedConfiguration.SealSecureBoot(sc, b.Node.TPMUsage); err != nil {
 					return fmt.Errorf("writing sealed configuration failed: %w", err)
 				}
 				supervisor.Logger(ctx).Infof("Control plane bootstrap complete, starting curator...")
diff --git a/metropolis/proto/api/configuration.proto b/metropolis/proto/api/configuration.proto
index 8c6c454..90e9dbd 100644
--- a/metropolis/proto/api/configuration.proto
+++ b/metropolis/proto/api/configuration.proto
@@ -38,9 +38,16 @@
         // management credentials for the cluster via an AAA.Escrow call.
         bytes owner_public_key = 1;
 
+        reserved 2;
+
         // If not set, defaults to:
         //   - TPM mode: REQUIRED
         common.ClusterConfiguration initial_cluster_configuration = 3;
+
+        // Labels that the first node will start out with. The given labels must
+        // be valid (see NodeLabels for more details). Invalid labels will be
+        // discarded.
+        metropolis.proto.common.NodeLabels labels = 4;
     }
     // ClusterRegister configures the node to attempt to register into an
     // existing cluster, ie. contact an existing running cluster and become
diff --git a/metropolis/test/launch/cluster/cluster.go b/metropolis/test/launch/cluster/cluster.go
index e2f2925..9bce17c 100644
--- a/metropolis/test/launch/cluster/cluster.go
+++ b/metropolis/test/launch/cluster/cluster.go
@@ -774,6 +774,11 @@
 				ClusterBootstrap: &apb.NodeParameters_ClusterBootstrap{
 					OwnerPublicKey:              InsecurePublicKey,
 					InitialClusterConfiguration: opts.InitialClusterConfiguration,
+					Labels: &cpb.NodeLabels{
+						Pairs: []*cpb.NodeLabels_Pair{
+							{Key: "test-node-id", Value: "0"},
+						},
+					},
 				},
 			},
 		},