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)
}
diff --git a/metropolis/proto/api/configuration.proto b/metropolis/proto/api/configuration.proto
index 3a4a5cd..8c6c454 100644
--- a/metropolis/proto/api/configuration.proto
+++ b/metropolis/proto/api/configuration.proto
@@ -37,6 +37,10 @@
// key can be used to prove ownership of the cluster and retrieve
// management credentials for the cluster via an AAA.Escrow call.
bytes owner_public_key = 1;
+
+ // If not set, defaults to:
+ // - TPM mode: REQUIRED
+ common.ClusterConfiguration initial_cluster_configuration = 3;
}
// ClusterRegister configures the node to attempt to register into an
// existing cluster, ie. contact an existing running cluster and become
diff --git a/metropolis/proto/api/management.proto b/metropolis/proto/api/management.proto
index 5b66311..5560455 100644
--- a/metropolis/proto/api/management.proto
+++ b/metropolis/proto/api/management.proto
@@ -90,6 +90,8 @@
// ca_certificate is the x509 DER encoded CA certificate of the cluster.
bytes ca_certificate = 2;
+
+ metropolis.proto.common.ClusterConfiguration cluster_configuration = 3;
}
message GetNodesRequest {
diff --git a/metropolis/proto/common/common.proto b/metropolis/proto/common/common.proto
index a884dff..27dfa5a 100644
--- a/metropolis/proto/common/common.proto
+++ b/metropolis/proto/common/common.proto
@@ -263,3 +263,42 @@
}
}
+// ClusterConfiguration contains the entirety of the user-configurable behaviour
+// of the cluster that is scoped to the entirety of the cluster (vs. per-node
+// configuration, which is kept alongside Node).
+//
+// It can be set initially when a cluster is being bootstrapped (in
+// NodeParamaters.ClusterBootstrap), and then can be partially managed by
+// management calls to the curator.
+message ClusterConfiguration {
+ // tpm_mode defines the TPM usage policy for cluster nodes. When nodes
+ // register into the cluster (and then join into it) they will report their
+ // TPM availability, and in return the cluster will respond whether they
+ // should use that TPM or not.
+ //
+ // If a node is instructed to use its TPM, it will use it to encrypt its part
+ // of the disk encryption key when saving it to the EFI system partition.
+ // That means that the node will only be able to re-join the cluster if its
+ // secure boot configuration doesn't change.
+ //
+ // If a node is instructed to not use its TPM, it will save its part of the
+ // disk encryption key straight onto the EFI system partition without any
+ // further encryption. It still needs to connect to a working cluster to
+ // retrieve the other part of the key. This means that the configuration is
+ // secure vs. offline disk decryption attempts, but not secure if an
+ // attacker can connect to a cluster and impersonate the node in order to
+ // retrieve the other part of its key.
+ enum TPMMode {
+ TPM_MODE_INVALID = 0;
+ // Nodes need to join with a TPM2.0 device and will be instructed to
+ // use it.
+ TPM_MODE_REQUIRED = 1;
+ // Nodes will be allowed to join regardless of TPM2.0 presence, and will
+ // be instructed to use it if they have one.
+ TPM_MODE_BEST_EFFORT = 2;
+ // Regardless of the node's local TPM presence it will be instructed to
+ // not use it.
+ TPM_MODE_DISABLED = 3;
+ }
+ TPMMode tpm_mode = 1;
+}