metropolis: implement cluster configuration

This adds a cluster configuration to Metropolis. We'll be keeping any
non-node-specific options there. The config is stored in etcd by the
curator.

An initial cluster configuration can be specified when bootstrapping a
cluster. By design the configuration is then immutable by default, but
we might add some purpose-specific management API calls to change some
values if needed.

We initialize the cluster configuration with a setting for node TPM
policy, 'TPMMode'. It's currently populated on cluster bootstrap, but
not used otherwise. That will come in a follow-up CR.

Change-Id: I44ddcd099c9ae68c20519c77e3fa77c894cf5a20
Reviewed-on: https://review.monogon.dev/c/monogon/+/1494
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
Tested-by: Jenkins CI
diff --git a/metropolis/node/core/cluster/BUILD.bazel b/metropolis/node/core/cluster/BUILD.bazel
index 1a724c1..93cad93 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/core/curator",
         "//metropolis/node/core/curator/proto/api",
         "//metropolis/node/core/identity",
         "//metropolis/node/core/localstorage",
diff --git a/metropolis/node/core/cluster/cluster.go b/metropolis/node/core/cluster/cluster.go
index adfc632..8a1a3ae 100644
--- a/metropolis/node/core/cluster/cluster.go
+++ b/metropolis/node/core/cluster/cluster.go
@@ -44,6 +44,7 @@
 	networkService *network.Service
 	roleServer     *roleserve.Service
 	nodeParams     *apb.NodeParameters
+	haveTPM        bool
 
 	oneway chan struct{}
 }
@@ -51,12 +52,13 @@
 // NewManager creates a new cluster Manager. The given localstorage Root must
 // be places, but not yet started (and will be started as the Manager makes
 // progress). The given network Service must already be running.
-func NewManager(storageRoot *localstorage.Root, networkService *network.Service, rs *roleserve.Service, nodeParams *apb.NodeParameters) *Manager {
+func NewManager(storageRoot *localstorage.Root, networkService *network.Service, rs *roleserve.Service, nodeParams *apb.NodeParameters, haveTPM bool) *Manager {
 	return &Manager{
 		storageRoot:    storageRoot,
 		networkService: networkService,
 		roleServer:     rs,
 		nodeParams:     nodeParams,
+		haveTPM:        haveTPM,
 		oneway:         make(chan struct{}),
 	}
 }
diff --git a/metropolis/node/core/cluster/cluster_bootstrap.go b/metropolis/node/core/cluster/cluster_bootstrap.go
index 0b51e1a..4f85fdc 100644
--- a/metropolis/node/core/cluster/cluster_bootstrap.go
+++ b/metropolis/node/core/cluster/cluster_bootstrap.go
@@ -24,7 +24,9 @@
 	"fmt"
 	"time"
 
+	"source.monogon.dev/metropolis/node/core/curator"
 	"source.monogon.dev/metropolis/pkg/supervisor"
+
 	apb "source.monogon.dev/metropolis/proto/api"
 	ppb "source.monogon.dev/metropolis/proto/private"
 )
@@ -32,6 +34,26 @@
 func (m *Manager) bootstrap(ctx context.Context, bootstrap *apb.NodeParameters_ClusterBootstrap) error {
 	supervisor.Logger(ctx).Infof("Bootstrapping new cluster, owner public key: %s", hex.EncodeToString(bootstrap.OwnerPublicKey))
 
+	var cc *curator.Cluster
+
+	if bootstrap.InitialClusterConfiguration == nil {
+		supervisor.Logger(ctx).Infof("No initial cluster configuration provided, using defaults.")
+		cc = curator.DefaultClusterConfiguration()
+	} else {
+		var err error
+		cc, err = curator.ClusterConfigurationFromInitial(bootstrap.InitialClusterConfiguration)
+		return fmt.Errorf("invalid initial cluster configuration: %w", err)
+	}
+
+	useTPM, err := cc.UseTPM(m.haveTPM)
+	if err != nil {
+		return fmt.Errorf("cannot join cluster: %w", err)
+	}
+
+	supervisor.Logger(ctx).Infof("TPM: cluster TPM mode: %s", cc.TPMMode)
+	supervisor.Logger(ctx).Infof("TPM: present in this node: %v", m.haveTPM)
+	supervisor.Logger(ctx).Infof("TPM: used by this node: %v", useTPM)
+
 	ownerKey := bootstrap.OwnerPublicKey
 	configuration := ppb.SealedConfiguration{}
 
@@ -67,7 +89,7 @@
 	}
 	supervisor.Logger(ctx).Infof("Bootstrapping: node public join key: %s", hex.EncodeToString([]byte(jpub)))
 
-	m.roleServer.ProvideBootstrapData(priv, ownerKey, cuk, nuk, jpriv)
+	m.roleServer.ProvideBootstrapData(priv, ownerKey, cuk, nuk, jpriv, cc)
 
 	supervisor.Signal(ctx, supervisor.SignalHealthy)
 	supervisor.Signal(ctx, supervisor.SignalDone)
diff --git a/metropolis/node/core/curator/BUILD.bazel b/metropolis/node/core/curator/BUILD.bazel
index dc39f38..83ffbe6 100644
--- a/metropolis/node/core/curator/BUILD.bazel
+++ b/metropolis/node/core/curator/BUILD.bazel
@@ -15,6 +15,7 @@
         "impl_leader_management.go",
         "listener.go",
         "state.go",
+        "state_cluster.go",
         "state_node.go",
         "state_pki.go",
         "state_registerticket.go",
diff --git a/metropolis/node/core/curator/bootstrap.go b/metropolis/node/core/curator/bootstrap.go
index 7566a43..39c1100 100644
--- a/metropolis/node/core/curator/bootstrap.go
+++ b/metropolis/node/core/curator/bootstrap.go
@@ -8,9 +8,10 @@
 	"google.golang.org/protobuf/proto"
 
 	"source.monogon.dev/metropolis/node/core/consensus/client"
-	ppb "source.monogon.dev/metropolis/node/core/curator/proto/private"
 	"source.monogon.dev/metropolis/node/core/identity"
 	"source.monogon.dev/metropolis/pkg/pki"
+
+	ppb "source.monogon.dev/metropolis/node/core/curator/proto/private"
 )
 
 // bootstrap.go contains functions specific for integration between the curator
@@ -33,7 +34,7 @@
 // This must only be used by the cluster bootstrap logic. It is idempotent, thus
 // can be called repeatedly in case of intermittent failures in the bootstrap
 // logic.
-func BootstrapNodeFinish(ctx context.Context, etcd client.Namespaced, node *Node, ownerKey []byte) (caCertBytes, nodeCertBytes []byte, err error) {
+func BootstrapNodeFinish(ctx context.Context, etcd client.Namespaced, node *Node, ownerKey []byte, cluster *Cluster) (caCertBytes, nodeCertBytes []byte, err error) {
 	// Workaround for pkiCA being a global, but BootstrapNodeFinish being called for
 	// multiple, different etcd instances in tests. Doing this ensures that we
 	// always resynchronize with etcd, ie. not keep certificates loaded in memory
@@ -81,15 +82,26 @@
 		return nil, nil, fmt.Errorf("failed to get join key: %w", err)
 	}
 
+	clusterProto, err := cluster.proto()
+	if err != nil {
+		return nil, nil, fmt.Errorf("failed to serialize initial cluster configuration: %w", err)
+	}
+	clusterRaw, err := proto.Marshal(clusterProto)
+	if err != nil {
+		return nil, nil, fmt.Errorf("failed to marshal initial cluster configuration: %w", err)
+	}
+
 	// We don't care about the result's success - this is idempotent.
 	_, err = etcd.Txn(ctx).If(
 		clientv3.Compare(clientv3.CreateRevision(nodePath), "=", 0),
 		clientv3.Compare(clientv3.CreateRevision(initialOwnerEtcdPath), "=", 0),
 		clientv3.Compare(clientv3.CreateRevision(joinKeyPath), "=", 0),
+		clientv3.Compare(clientv3.CreateRevision(clusterConfigurationKey), "=", 0),
 	).Then(
 		clientv3.OpPut(nodePath, string(nodeRaw)),
 		clientv3.OpPut(initialOwnerEtcdPath, string(ownerRaw)),
 		clientv3.OpPut(joinKeyPath, node.ID()),
+		clientv3.OpPut(clusterConfigurationKey, string(clusterRaw)),
 	).Commit()
 	if err != nil {
 		return nil, nil, fmt.Errorf("failed to store initial cluster state: %w", err)
diff --git a/metropolis/node/core/curator/impl_leader_management.go b/metropolis/node/core/curator/impl_leader_management.go
index a46c9e3..461df02 100644
--- a/metropolis/node/core/curator/impl_leader_management.go
+++ b/metropolis/node/core/curator/impl_leader_management.go
@@ -116,10 +116,17 @@
 		}
 	}
 
-	return &apb.GetClusterInfoResponse{
+	resp := &apb.GetClusterInfoResponse{
 		ClusterDirectory: directory,
 		CaCertificate:    l.node.ClusterCA().Raw,
-	}, nil
+	}
+
+	cl, err := clusterLoad(ctx, l.leadership)
+	if err == nil {
+		resp.ClusterConfiguration, _ = cl.proto()
+	}
+
+	return resp, nil
 }
 
 // nodeHeartbeatTimestamp returns the node nid's last heartbeat timestamp, as
diff --git a/metropolis/node/core/curator/impl_leader_test.go b/metropolis/node/core/curator/impl_leader_test.go
index d245f6f..b40ad54 100644
--- a/metropolis/node/core/curator/impl_leader_test.go
+++ b/metropolis/node/core/curator/impl_leader_test.go
@@ -99,7 +99,7 @@
 	// Here we would enable the leader node's roles. But for tests, we don't enable
 	// any.
 
-	caCertBytes, nodeCertBytes, err := BootstrapNodeFinish(ctx, curEtcd, &cNode, nil)
+	caCertBytes, nodeCertBytes, err := BootstrapNodeFinish(ctx, curEtcd, &cNode, nil, DefaultClusterConfiguration())
 	if err != nil {
 		t.Fatalf("could not finish node bootstrap: %v", err)
 	}
diff --git a/metropolis/node/core/curator/state_cluster.go b/metropolis/node/core/curator/state_cluster.go
new file mode 100644
index 0000000..b62fe07
--- /dev/null
+++ b/metropolis/node/core/curator/state_cluster.go
@@ -0,0 +1,144 @@
+package curator
+
+import (
+	"context"
+	"fmt"
+
+	clientv3 "go.etcd.io/etcd/client/v3"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"google.golang.org/protobuf/proto"
+
+	"source.monogon.dev/metropolis/node/core/rpc"
+	cpb "source.monogon.dev/metropolis/proto/common"
+)
+
+var (
+	clusterConfigurationKey = "/cluster/configuration"
+)
+
+// Cluster is the cluster's configuration, as (un)marshaled to/from
+// common.ClusterConfiguration.
+type Cluster struct {
+	TPMMode cpb.ClusterConfiguration_TPMMode
+}
+
+// DefaultClusterConfiguration is the default cluster configuration for a newly
+// bootstrapped cluster if no initial cluster configuration was specified by the
+// user.
+func DefaultClusterConfiguration() *Cluster {
+	return &Cluster{
+		TPMMode: cpb.ClusterConfiguration_TPM_MODE_REQUIRED,
+	}
+}
+
+// ClusterConfigurationFromInitial converts a user-provided initial cluster
+// configuration proto into a Cluster, checking that the provided values are
+// valid.
+func ClusterConfigurationFromInitial(icc *cpb.ClusterConfiguration) (*Cluster, error) {
+	// clusterFromProto performs type checks.
+	return clusterFromProto(icc)
+}
+
+func (c *Cluster) UseTPM(available bool) (bool, error) {
+	switch c.TPMMode {
+	case cpb.ClusterConfiguration_TPM_MODE_DISABLED:
+		return false, nil
+	case cpb.ClusterConfiguration_TPM_MODE_REQUIRED:
+		if !available {
+			return false, fmt.Errorf("TPM required but not available")
+		}
+		return true, nil
+	case cpb.ClusterConfiguration_TPM_MODE_BEST_EFFORT:
+		return available, nil
+	default:
+		return false, fmt.Errorf("invalid TPM mode")
+	}
+}
+
+func clusterUnmarshal(data []byte) (*Cluster, error) {
+	msg := cpb.ClusterConfiguration{}
+	if err := proto.Unmarshal(data, &msg); err != nil {
+		return nil, fmt.Errorf("could not unmarshal proto: %w", err)
+	}
+	return clusterFromProto(&msg)
+}
+
+func clusterFromProto(cc *cpb.ClusterConfiguration) (*Cluster, error) {
+	switch cc.TpmMode {
+	case cpb.ClusterConfiguration_TPM_MODE_REQUIRED:
+	case cpb.ClusterConfiguration_TPM_MODE_BEST_EFFORT:
+	case cpb.ClusterConfiguration_TPM_MODE_DISABLED:
+	default:
+		return nil, fmt.Errorf("invalid TpmMode: %v", cc.TpmMode)
+	}
+
+	c := &Cluster{
+		TPMMode: cc.TpmMode,
+	}
+
+	return c, nil
+}
+
+func (c *Cluster) proto() (*cpb.ClusterConfiguration, error) {
+	switch c.TPMMode {
+	case cpb.ClusterConfiguration_TPM_MODE_REQUIRED:
+	case cpb.ClusterConfiguration_TPM_MODE_BEST_EFFORT:
+	case cpb.ClusterConfiguration_TPM_MODE_DISABLED:
+	default:
+		return nil, fmt.Errorf("invalid TPMMode %d", c.TPMMode)
+	}
+	return &cpb.ClusterConfiguration{
+		TpmMode: c.TPMMode,
+	}, nil
+}
+
+func clusterLoad(ctx context.Context, l *leadership) (*Cluster, error) {
+	rpc.Trace(ctx).Printf("loadCluster...")
+	res, err := l.txnAsLeader(ctx, clientv3.OpGet(clusterConfigurationKey))
+	if err != nil {
+		if rpcErr, ok := rpcError(err); ok {
+			return nil, rpcErr
+		}
+		rpc.Trace(ctx).Printf("could not retrieve cluster configuartion: %v", err)
+		return nil, status.Errorf(codes.Unavailable, "could not retrieve cluster configuration: %v", err)
+	}
+	kvs := res.Responses[0].GetResponseRange().Kvs
+	rpc.Trace(ctx).Printf("loadCluster: %d KVs", len(kvs))
+	if len(kvs) != 1 {
+		return nil, errNodeNotFound
+	}
+	node, err := clusterUnmarshal(kvs[0].Value)
+	if err != nil {
+		rpc.Trace(ctx).Printf("could not unmarshal cluster: %v", err)
+		return nil, status.Errorf(codes.Unavailable, "could not unmarshal cluster")
+	}
+	rpc.Trace(ctx).Printf("loadCluster: unmarshal ok")
+	return node, nil
+}
+
+func clusterSave(ctx context.Context, l *leadership, c *Cluster) error {
+	rpc.Trace(ctx).Printf("clusterSave...")
+	clusterProto, err := c.proto()
+	if err != nil {
+		rpc.Trace(ctx).Printf("could not convert updated cluster configuration: %v", err)
+		return status.Errorf(codes.Unavailable, "could not convert updated cluster")
+	}
+	clusterBytes, err := proto.Marshal(clusterProto)
+	if err != nil {
+		rpc.Trace(ctx).Printf("could not marshal updated cluster configuration: %v", err)
+		return status.Errorf(codes.Unavailable, "could not marshal updated cluster")
+	}
+
+	ocs := clientv3.OpPut(clusterConfigurationKey, string(clusterBytes))
+	_, err = l.txnAsLeader(ctx, ocs)
+	if err != nil {
+		if rpcErr, ok := rpcError(err); ok {
+			return rpcErr
+		}
+		rpc.Trace(ctx).Printf("could not save updated cluster configuration: %v", err)
+		return status.Error(codes.Unavailable, "could not save updated cluster configuration")
+	}
+	rpc.Trace(ctx).Printf("clusterSave: write ok")
+	return nil
+}
diff --git a/metropolis/node/core/main.go b/metropolis/node/core/main.go
index a9bdc98..08eb39a 100644
--- a/metropolis/node/core/main.go
+++ b/metropolis/node/core/main.go
@@ -103,8 +103,10 @@
 
 	logger.Info("Starting Metropolis node init")
 
+	haveTPM := true
 	if err := tpm.Initialize(logger); err != nil {
-		logger.Warningf("Failed to initialize TPM 2.0, attempting fallback to untrusted: %v", err)
+		logger.Warningf("Failed to initialize TPM 2.0: %v", err)
+		haveTPM = false
 	}
 
 	networkSvc := network.New(nil)
@@ -184,7 +186,7 @@
 
 		// Start cluster manager. This kicks off cluster membership machinery,
 		// which will either start a new cluster, enroll into one or join one.
-		m := cluster.NewManager(root, networkSvc, rs, nodeParams)
+		m := cluster.NewManager(root, networkSvc, rs, nodeParams, haveTPM)
 		return m.Run(ctx)
 	}
 
diff --git a/metropolis/node/core/roleserve/roleserve.go b/metropolis/node/core/roleserve/roleserve.go
index d2e5bf0..f97e9c9 100644
--- a/metropolis/node/core/roleserve/roleserve.go
+++ b/metropolis/node/core/roleserve/roleserve.go
@@ -1,4 +1,4 @@
-// package roleserve implements the roleserver/“Role Server”.
+// Package roleserve implements the roleserver/“Role Server”.
 //
 // The Role Server runs on every node and is responsible for running all of the
 // node's role dependant services, like the control plane (Consensus/etcd and
@@ -45,6 +45,7 @@
 
 	common "source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/node/core/clusternet"
+	"source.monogon.dev/metropolis/node/core/curator"
 	"source.monogon.dev/metropolis/node/core/identity"
 	"source.monogon.dev/metropolis/node/core/localstorage"
 	"source.monogon.dev/metropolis/node/core/network"
@@ -164,7 +165,7 @@
 	return s
 }
 
-func (s *Service) ProvideBootstrapData(privkey ed25519.PrivateKey, iok, cuk, nuk, jkey []byte) {
+func (s *Service) ProvideBootstrapData(privkey ed25519.PrivateKey, iok, cuk, nuk, jkey []byte, icc *curator.Cluster) {
 	pubkey := privkey.Public().(ed25519.PublicKey)
 	nid := identity.NodeID(pubkey)
 
@@ -177,11 +178,12 @@
 		resolver: s.Resolver,
 	})
 	s.bootstrapData.Set(&bootstrapData{
-		nodePrivateKey:     privkey,
-		initialOwnerKey:    iok,
-		clusterUnlockKey:   cuk,
-		nodeUnlockKey:      nuk,
-		nodePrivateJoinKey: jkey,
+		nodePrivateKey:              privkey,
+		initialOwnerKey:             iok,
+		clusterUnlockKey:            cuk,
+		nodeUnlockKey:               nuk,
+		nodePrivateJoinKey:          jkey,
+		initialClusterConfiguration: icc,
 	})
 }
 
diff --git a/metropolis/node/core/roleserve/value_bootstrapdata.go b/metropolis/node/core/roleserve/value_bootstrapdata.go
index 29a6ae2..90af955 100644
--- a/metropolis/node/core/roleserve/value_bootstrapdata.go
+++ b/metropolis/node/core/roleserve/value_bootstrapdata.go
@@ -2,6 +2,8 @@
 
 import (
 	"crypto/ed25519"
+
+	"source.monogon.dev/metropolis/node/core/curator"
 )
 
 // bootstrapData is an internal EventValue structure which is populated by the
@@ -9,9 +11,10 @@
 // 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
+	nodePrivateKey              ed25519.PrivateKey
+	clusterUnlockKey            []byte
+	nodeUnlockKey               []byte
+	initialOwnerKey             []byte
+	nodePrivateJoinKey          ed25519.PrivateKey
+	initialClusterConfiguration *curator.Cluster
 }
diff --git a/metropolis/node/core/roleserve/worker_controlplane.go b/metropolis/node/core/roleserve/worker_controlplane.go
index 885b39e..1a8e420 100644
--- a/metropolis/node/core/roleserve/worker_controlplane.go
+++ b/metropolis/node/core/roleserve/worker_controlplane.go
@@ -295,7 +295,7 @@
 				n.EnableKubernetesController()
 
 				var nodeCert []byte
-				caCert, nodeCert, err = curator.BootstrapNodeFinish(ctx, ckv, &n, b.initialOwnerKey)
+				caCert, nodeCert, err = curator.BootstrapNodeFinish(ctx, ckv, &n, b.initialOwnerKey, b.initialClusterConfiguration)
 				if err != nil {
 					return fmt.Errorf("while bootstrapping node: %w", err)
 				}