m/n/core/roleserve: do not store cluster directory, populate resolver earlier

This cleans up the roleserver code slightly, as I've noticed the cluster
directory actually does not need to be stored anymore now that we have
the rpc resolver.

Change-Id: Ibe9f55691602bc937205c5fb54833683e80d4804
Reviewed-on: https://review.monogon.dev/c/monogon/+/1748
Tested-by: Jenkins CI
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/core/cluster/cluster_bootstrap.go b/metropolis/node/core/cluster/cluster_bootstrap.go
index 1306cff..fcc5756 100644
--- a/metropolis/node/core/cluster/cluster_bootstrap.go
+++ b/metropolis/node/core/cluster/cluster_bootstrap.go
@@ -24,10 +24,13 @@
 	"fmt"
 	"time"
 
+	"google.golang.org/protobuf/proto"
+
 	"source.monogon.dev/metropolis/node/core/curator"
 	"source.monogon.dev/metropolis/pkg/supervisor"
 
 	apb "source.monogon.dev/metropolis/proto/api"
+	cpb "source.monogon.dev/metropolis/proto/common"
 	ppb "source.monogon.dev/metropolis/proto/private"
 )
 
@@ -95,6 +98,38 @@
 	}
 	supervisor.Logger(ctx).Infof("Bootstrapping: node public join key: %s", hex.EncodeToString([]byte(jpub)))
 
+	directory := &cpb.ClusterDirectory{
+		Nodes: []*cpb.ClusterDirectory_Node{
+			{
+				PublicKey: pub,
+				Addresses: []*cpb.ClusterDirectory_Node_Address{
+					{
+						Host: "127.0.0.1",
+					},
+				},
+			},
+		},
+	}
+	cdirRaw, err := proto.Marshal(directory)
+	if err != nil {
+		return fmt.Errorf("couldn't marshal ClusterDirectory: %w", err)
+	}
+	if err = m.storageRoot.ESP.Metropolis.ClusterDirectory.Write(cdirRaw, 0644); err != nil {
+		return fmt.Errorf("writing cluster directory failed: %w", err)
+	}
+
+	sc := ppb.SealedConfiguration{
+		NodeUnlockKey: nuk,
+		JoinKey:       jpriv,
+		// No ClusterCA yet, that's added by the roleserver after it finishes curator
+		// bootstrap.
+		ClusterCa: nil,
+	}
+	if err = m.storageRoot.ESP.Metropolis.SealedConfiguration.SealSecureBoot(&sc, tpmUsage); err != nil {
+		return fmt.Errorf("writing sealed configuration failed: %w", err)
+	}
+	supervisor.Logger(ctx).Infof("Saved bootstrapped node's credentials.")
+
 	m.roleServer.ProvideBootstrapData(priv, ownerKey, cuk, nuk, jpriv, cc, tpmUsage)
 
 	supervisor.Signal(ctx, supervisor.SignalHealthy)
diff --git a/metropolis/node/core/roleserve/BUILD.bazel b/metropolis/node/core/roleserve/BUILD.bazel
index 4a79349..2d62ef4 100644
--- a/metropolis/node/core/roleserve/BUILD.bazel
+++ b/metropolis/node/core/roleserve/BUILD.bazel
@@ -41,11 +41,8 @@
         "//metropolis/pkg/pki",
         "//metropolis/pkg/supervisor",
         "//metropolis/proto/common",
-        "//metropolis/proto/private",
         "@org_golang_google_grpc//:go_default_library",
         "@org_golang_google_protobuf//encoding/prototext",
-        "@org_golang_google_protobuf//proto",
-        "@org_golang_x_sys//unix",
     ],
 )
 
diff --git a/metropolis/node/core/roleserve/roleserve.go b/metropolis/node/core/roleserve/roleserve.go
index c1493be..3af5319 100644
--- a/metropolis/node/core/roleserve/roleserve.go
+++ b/metropolis/node/core/roleserve/roleserve.go
@@ -172,6 +172,7 @@
 	// This is the first time we have the node ID, tell the resolver that it's
 	// available on the loopback interface.
 	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.ClusterMembership.Set(&ClusterMembership{
 		pubkey:   pubkey,
@@ -192,12 +193,19 @@
 	// This is the first time we have the node ID, tell the resolver that it's
 	// available on the loopback interface.
 	s.Resolver.AddOverride(credentials.ID(), resolver.NodeByHostPort("127.0.0.1", uint16(common.CuratorServicePort)))
+	// Also tell the resolver about all the existing nodes in the cluster we just
+	// registered into.
+	for _, n := range directory.Nodes {
+		// TODO(q3k): only add control plane nodes.
+		for _, addr := range n.Addresses {
+			s.Resolver.AddEndpoint(resolver.NodeByHostPort(addr.Host, uint16(common.CuratorServicePort)))
+		}
+	}
 
 	s.ClusterMembership.Set(&ClusterMembership{
-		remoteCurators: directory,
-		credentials:    &credentials,
-		pubkey:         credentials.PublicKey(),
-		resolver:       s.Resolver,
+		credentials: &credentials,
+		pubkey:      credentials.PublicKey(),
+		resolver:    s.Resolver,
 	})
 }
 
@@ -205,12 +213,19 @@
 	// This is the first time we have the node ID, tell the resolver that it's
 	// available on the loopback interface.
 	s.Resolver.AddOverride(credentials.ID(), resolver.NodeByHostPort("127.0.0.1", uint16(common.CuratorServicePort)))
+	// Also tell the resolver about all the existing nodes in the cluster we just
+	// joined into.
+	for _, n := range directory.Nodes {
+		// TODO(q3k): only add control plane nodes.
+		for _, addr := range n.Addresses {
+			s.Resolver.AddEndpoint(resolver.NodeByHostPort(addr.Host, uint16(common.CuratorServicePort)))
+		}
+	}
 
 	s.ClusterMembership.Set(&ClusterMembership{
-		remoteCurators: directory,
-		credentials:    &credentials,
-		pubkey:         credentials.PublicKey(),
-		resolver:       s.Resolver,
+		credentials: &credentials,
+		pubkey:      credentials.PublicKey(),
+		resolver:    s.Resolver,
 	})
 	s.clusterDirectorySaved.Set(true)
 }
diff --git a/metropolis/node/core/roleserve/value_clustermembership.go b/metropolis/node/core/roleserve/value_clustermembership.go
index 3044c7c..7801650 100644
--- a/metropolis/node/core/roleserve/value_clustermembership.go
+++ b/metropolis/node/core/roleserve/value_clustermembership.go
@@ -1,19 +1,16 @@
 package roleserve
 
 import (
-	"context"
 	"crypto/ed25519"
 
 	"google.golang.org/grpc"
 
-	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/node/core/consensus"
 	"source.monogon.dev/metropolis/node/core/curator"
 	"source.monogon.dev/metropolis/node/core/identity"
 	"source.monogon.dev/metropolis/node/core/rpc"
 	"source.monogon.dev/metropolis/node/core/rpc/resolver"
 	"source.monogon.dev/metropolis/pkg/event"
-	cpb "source.monogon.dev/metropolis/proto/common"
 )
 
 // ClusterMembership is an Event Value structure used to keep state of the
@@ -35,13 +32,6 @@
 	// node runs control plane services.
 	localConsensus *consensus.Service
 	localCurator   *curator.Service
-	// remoteCurators gets set by Cluster Enrolment code when Registering into a
-	// cluster and gets propagated by the Control Plane Worker to maintain
-	// connectivity to external Curators regardless of local curator health.
-	//
-	// TODO(q3k): also update this based on a live Cluster Directory from the
-	// cluster.
-	remoteCurators *cpb.ClusterDirectory
 	// credentials is set whenever this node has full access to the Cluster and is
 	// the a of credentials which can be used to perform authenticated (as the node)
 	// access to the Curator.
@@ -52,24 +42,7 @@
 	resolver *resolver.Resolver
 }
 
-// GetNodeID returns the Node ID of the locally running node whenever available.
-// NodeIDs are available early on in the node startup process and are guaranteed
-// to never change at runtime. The watcher will then block all further Get calls
-// until new information is available. This method should only be used if
-// GetNodeID is the only method ran on the watcher.
-func GetNodeID(ctx context.Context, watcher event.Watcher[*ClusterMembership]) (string, error) {
-	for {
-		cm, err := watcher.Get(ctx)
-		if err != nil {
-			return "", err
-		}
-		if cm.pubkey != nil {
-			return identity.NodeID(cm.pubkey), nil
-		}
-	}
-}
-
-// GetHome returns a ClusterMembership whenever the local node is HOME to a
+// FilterHome returns a ClusterMembership whenever the local node is HOME to a
 // cluster (ie. whenever the node is fully a member of a cluster and can dial
 // the cluster's Curator). See proto.common.ClusterState for more information
 // about cluster states. The watcher will then block all futher Get calls until
@@ -79,9 +52,6 @@
 		if cm.credentials == nil {
 			return false
 		}
-		if cm.remoteCurators == nil {
-			return false
-		}
 		return true
 	})
 }
@@ -91,19 +61,6 @@
 // forwarded to the current control plane leader, and this gRPC client
 // connection can be used long-term by callers.
 func (m *ClusterMembership) DialCurator() (*grpc.ClientConn, error) {
-	// Always make sure the resolver has fresh data about curators, both local and
-	// remote. This would be better done only when ClusterMembership is set() with
-	// new data, but that would require a bit of a refactor.
-	//
-	// TODO(q3k): take care of the above, possibly when the roleserver is made more generic.
-	if m.localConsensus != nil {
-		m.resolver.AddEndpoint(resolver.NodeByHostPort("127.0.0.1", uint16(common.CuratorServicePort)))
-	}
-	for _, n := range m.remoteCurators.Nodes {
-		for _, addr := range n.Addresses {
-			m.resolver.AddEndpoint(resolver.NodeByHostPort(addr.Host, uint16(common.CuratorServicePort)))
-		}
-	}
 	creds := rpc.NewAuthenticatedCredentials(m.credentials.TLSCredentials(), rpc.WantRemoteCluster(m.credentials.ClusterCA()))
 	return grpc.Dial(resolver.MetropolisControlAddress, grpc.WithTransportCredentials(creds), grpc.WithResolvers(m.resolver))
 }
diff --git a/metropolis/node/core/roleserve/worker_controlplane.go b/metropolis/node/core/roleserve/worker_controlplane.go
index d21df70..eb85f5d 100644
--- a/metropolis/node/core/roleserve/worker_controlplane.go
+++ b/metropolis/node/core/roleserve/worker_controlplane.go
@@ -1,16 +1,12 @@
 package roleserve
 
 import (
-	"bytes"
 	"context"
 	"crypto/ed25519"
 	"crypto/x509"
 	"fmt"
 	"time"
 
-	"golang.org/x/sys/unix"
-	"google.golang.org/protobuf/proto"
-
 	"source.monogon.dev/metropolis/node/core/consensus"
 	"source.monogon.dev/metropolis/node/core/curator"
 	"source.monogon.dev/metropolis/node/core/identity"
@@ -21,7 +17,6 @@
 	"source.monogon.dev/metropolis/pkg/pki"
 	"source.monogon.dev/metropolis/pkg/supervisor"
 	cpb "source.monogon.dev/metropolis/proto/common"
-	ppb "source.monogon.dev/metropolis/proto/private"
 )
 
 // workerControlPlane is the Control Plane Worker, responsible for maintaining a
@@ -253,11 +248,10 @@
 
 			// Prepare curator config, notably performing a bootstrap step if necessary. The
 			// preparation will result in a set of node credentials to run the curator with
-			// and a previously used cluster directory to be passed over to the new
+			// nd a previously used cluster directory to be passed over to the new
 			// ClusterMembership, if any.
 			var creds *identity.NodeCredentials
 			var caCert []byte
-			var directory *cpb.ClusterDirectory
 			if b := startup.bootstrap; b != nil {
 				supervisor.Logger(ctx).Infof("Bootstrapping control plane. Waiting for consensus...")
 
@@ -304,6 +298,18 @@
 				if err != nil {
 					return fmt.Errorf("when creating bootstrap node credentials: %w", err)
 				}
+
+				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()
+				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 {
+					return fmt.Errorf("writing sealed configuration failed: %w", err)
+				}
 				supervisor.Logger(ctx).Infof("Control plane bootstrap complete, starting curator...")
 			} else {
 				// Not bootstrapping, just starting consensus with credentials we already have.
@@ -316,65 +322,11 @@
 				if startup.existingMembership.credentials == nil {
 					panic("no existingMembership.credentials but not bootstrapping either")
 				}
-				if startup.existingMembership.remoteCurators == nil {
-					panic("no existingMembership.remoteCurators but not bootstrapping either")
-				}
 
 				// Use already existing credentials, and pass over already known curators (as
 				// we're not the only node, and we'd like downstream consumers to be able to
 				// keep connecting to existing curators in case the local one fails).
 				creds = startup.existingMembership.credentials
-				directory = startup.existingMembership.remoteCurators
-			}
-
-			// Ensure this node is present in the cluster directory.
-			if directory == nil {
-				directory = &cpb.ClusterDirectory{}
-			}
-			missing := true
-			for _, n := range directory.Nodes {
-				if bytes.Equal(n.PublicKey, creds.PublicKey()) {
-					missing = false
-					break
-				}
-			}
-			if missing {
-				directory.Nodes = append(directory.Nodes, &cpb.ClusterDirectory_Node{
-					PublicKey: creds.PublicKey(),
-					Addresses: []*cpb.ClusterDirectory_Node_Address{
-						{
-							Host: "127.0.0.1",
-						},
-					},
-				})
-			}
-
-			// Save this node's credentials, cluster directory and configuration as
-			// part of the control plane bootstrap process.
-			if b := startup.bootstrap; b != nil && caCert != nil {
-				if err = creds.Save(&s.storageRoot.Data.Node.Credentials); err != nil {
-					return fmt.Errorf("while saving node credentials: %w", err)
-				}
-
-				cdirRaw, err := proto.Marshal(directory)
-				if err != nil {
-					return fmt.Errorf("couldn't marshal ClusterDirectory: %w", err)
-				}
-				if err = s.storageRoot.ESP.Metropolis.ClusterDirectory.Write(cdirRaw, 0644); err != nil {
-					return fmt.Errorf("writing cluster directory failed: %w", err)
-				}
-
-				sc := ppb.SealedConfiguration{
-					NodeUnlockKey: b.nodeUnlockKey,
-					JoinKey:       b.nodePrivateJoinKey,
-					ClusterCa:     caCert,
-				}
-				if err = s.storageRoot.ESP.Metropolis.SealedConfiguration.SealSecureBoot(&sc, b.nodeTPMUsage); err != nil {
-					return fmt.Errorf("writing sealed configuration failed: %w", err)
-				}
-
-				supervisor.Logger(ctx).Infof("Saved bootstrapped node's credentials.")
-				unix.Sync()
 			}
 
 			// Start curator.
@@ -396,7 +348,6 @@
 				localConsensus: con,
 				localCurator:   cur,
 				credentials:    creds,
-				remoteCurators: directory,
 				pubkey:         creds.PublicKey(),
 				resolver:       s.resolver,
 			})