m/node: introduce node storage setting and cluster policy
This adds NodeStorageSecurity and a corresponding
ClusterConfiguration.StorageSecurityPolicy, and pipes it into the
Metropolis node bootstrap and registration flow.
All the various settings have so far only been tested manually. For now
the default behaviour (which is exercised by tests) is the same as
previously: require encryption and authentication.
In the future, we will have to expand our end-to-end testing to properly
exercise all the various settings and verify their enforcement and
effect. But that has to come in a follow-up CR as this one is already
large enough as is.
Change-Id: I76f3e37639ef02f4fc708af47ae5014408dc7c21
Reviewed-on: https://review.monogon.dev/c/monogon/+/1747
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
Tested-by: Jenkins CI
diff --git a/metropolis/node/core/cluster/cluster_bootstrap.go b/metropolis/node/core/cluster/cluster_bootstrap.go
index d86f995..1306cff 100644
--- a/metropolis/node/core/cluster/cluster_bootstrap.go
+++ b/metropolis/node/core/cluster/cluster_bootstrap.go
@@ -49,11 +49,16 @@
tpmUsage, err := cc.NodeTPMUsage(m.haveTPM)
if err != nil {
- return fmt.Errorf("cannot join cluster: %w", err)
+ return fmt.Errorf("cannot bootstrap cluster: %w", err)
}
- supervisor.Logger(ctx).Infof("TPM: cluster TPM mode: %s", cc.TPMMode)
- supervisor.Logger(ctx).Infof("TPM: node TPM usage: %s", tpmUsage)
+ storageSecurity, err := cc.NodeStorageSecurity()
+ if err != nil {
+ return fmt.Errorf("cannot bootstrap cluster: %w", err)
+ }
+
+ supervisor.Logger(ctx).Infof("TPM: cluster policy: %s, node: %s", cc.TPMMode, tpmUsage)
+ supervisor.Logger(ctx).Infof("Storage Security: cluster policy: %s, node: %s", cc.StorageSecurityPolicy, storageSecurity)
ownerKey := bootstrap.OwnerPublicKey
configuration := ppb.SealedConfiguration{}
@@ -71,7 +76,7 @@
supervisor.Logger(ctx).Infof("Bootstrapping: still waiting for storage....")
}
}()
- cuk, err := m.storageRoot.Data.MountNew(&configuration)
+ cuk, err := m.storageRoot.Data.MountNew(&configuration, storageSecurity)
close(storageDone)
if err != nil {
return fmt.Errorf("could not make and mount data partition: %w", err)
diff --git a/metropolis/node/core/cluster/cluster_join.go b/metropolis/node/core/cluster/cluster_join.go
index 4fd6473..fbec05d 100644
--- a/metropolis/node/core/cluster/cluster_join.go
+++ b/metropolis/node/core/cluster/cluster_join.go
@@ -70,23 +70,38 @@
}
cur := ipb.NewCuratorClient(eph)
- // Join the cluster and use the newly obtained CUK to mount the data
- // partition.
- var jr *ipb.JoinNodeResponse
- bo := backoff.NewExponentialBackOff()
- bo.MaxElapsedTime = 0
- backoff.Retry(func() error {
- jr, err = cur.JoinNode(ctx, &ipb.JoinNodeRequest{
- UsingSealedConfiguration: sealed,
- })
- if err != nil {
- supervisor.Logger(ctx).Warningf("Join failed: %v", err)
- // This is never used.
- return fmt.Errorf("join call failed")
+ // Retrieve CUK from cluster and reconstruct encryption key if we're not in
+ // insecure mode.
+ var cuk []byte
+ if sc.StorageSecurity != cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE {
+ if want, got := 32, len(sc.NodeUnlockKey); want != got {
+ return fmt.Errorf("sealed configuration has invalid node unlock key (wanted %d bytes, got %d)", want, got)
}
- return nil
- }, bo)
- if err := m.storageRoot.Data.MountExisting(sc, jr.ClusterUnlockKey); err != nil {
+
+ // Join the cluster and use the newly obtained CUK to mount the data
+ // partition.
+ var jr *ipb.JoinNodeResponse
+ bo := backoff.NewExponentialBackOff()
+ bo.MaxElapsedTime = 0
+ backoff.Retry(func() error {
+ jr, err = cur.JoinNode(ctx, &ipb.JoinNodeRequest{
+ UsingSealedConfiguration: sealed,
+ })
+ if err != nil {
+ supervisor.Logger(ctx).Warningf("Join failed: %v", err)
+ // This is never used.
+ return fmt.Errorf("join call failed")
+ }
+ return nil
+ }, bo)
+ cuk = jr.ClusterUnlockKey
+
+ if want, got := 32, len(cuk); want != got {
+ return fmt.Errorf("cluster returned invalid cluster unlock key (wanted %d bytes, got %d)", want, got)
+ }
+ }
+
+ if err := m.storageRoot.Data.MountExisting(sc, cuk); err != nil {
return fmt.Errorf("while mounting Data: %w", err)
}
diff --git a/metropolis/node/core/cluster/cluster_register.go b/metropolis/node/core/cluster/cluster_register.go
index 0096fa7..167f2f7 100644
--- a/metropolis/node/core/cluster/cluster_register.go
+++ b/metropolis/node/core/cluster/cluster_register.go
@@ -70,28 +70,10 @@
}
// Validation passed, let's start working on registering us into the cluster.
-
- // Tell the user what we're doing.
- supervisor.Logger(ctx).Infof("Registering into existing cluster.")
- supervisor.Logger(ctx).Infof(" Cluster CA public key: %s", hex.EncodeToString(ca.PublicKey.(ed25519.PublicKey)))
- supervisor.Logger(ctx).Infof(" Register Ticket: %s", hex.EncodeToString(register.RegisterTicket))
- supervisor.Logger(ctx).Infof(" Directory:")
- logClusterDirectory(ctx, register.ClusterDirectory)
-
- // Mount new storage with generated CUK, MountNew will save NUK into sc, to be
- // saved into the ESP after successful registration.
- var sc ppb.SealedConfiguration
- supervisor.Logger(ctx).Infof("Registering: mounting new storage...")
- cuk, err := m.storageRoot.Data.MountNew(&sc)
- if err != nil {
- return fmt.Errorf("could not make and mount data partition: %w", err)
- }
-
pub, priv, err := ed25519.GenerateKey(rand.Reader)
if err != nil {
return fmt.Errorf("could not generate node keypair: %w", err)
}
- supervisor.Logger(ctx).Infof("Registering: node public key: %s", hex.EncodeToString([]byte(pub)))
// Build resolver used by the register process, authenticating with ephemeral
// credentials. Once the join is complete, the rolesever will start its own
@@ -124,13 +106,22 @@
}
cur := ipb.NewCuratorClient(eph)
+ // TODO(q3k): allow node to pick storage security per given policy
+
+ // Tell the user what we're doing.
+ supervisor.Logger(ctx).Infof("Registering into existing cluster.")
+ supervisor.Logger(ctx).Infof(" Cluster CA public key: %s", hex.EncodeToString(ca.PublicKey.(ed25519.PublicKey)))
+ supervisor.Logger(ctx).Infof(" Node public key: %s", hex.EncodeToString(pub))
+ supervisor.Logger(ctx).Infof(" Register Ticket: %s", hex.EncodeToString(register.RegisterTicket))
+ supervisor.Logger(ctx).Infof(" Directory:")
+ logClusterDirectory(ctx, register.ClusterDirectory)
+
// Generate Join Credentials. The private key will be stored in
// SealedConfiguration only if RegisterNode succeeds.
jpub, jpriv, err := ed25519.GenerateKey(rand.Reader)
if err != nil {
return fmt.Errorf("could not generate join keypair: %w", err)
}
- sc.JoinKey = jpriv
supervisor.Logger(ctx).Infof("Registering: join public key: %s", hex.EncodeToString([]byte(jpub)))
// Register this node.
@@ -148,7 +139,20 @@
if err != nil {
return fmt.Errorf("register call failed: %w", err)
}
+ storageSecurity := res.RecommendedNodeStorageSecurity
+ // Mount new storage with generated CUK, MountNew will save NUK into sc, to be
+ // saved into the ESP after successful registration.
+ var sc ppb.SealedConfiguration
+ supervisor.Logger(ctx).Infof("Registering: mounting new storage...")
+ cuk, err := m.storageRoot.Data.MountNew(&sc, storageSecurity)
+ if err != nil {
+ return fmt.Errorf("could not make and mount data partition: %w", err)
+ }
+ sc.JoinKey = jpriv
+
+ supervisor.Logger(ctx).Infof("Storage Security: cluster policy: %s", res.ClusterConfiguration.StorageSecurityPolicy)
+ supervisor.Logger(ctx).Infof("Storage Security: node: %s", storageSecurity)
supervisor.Logger(ctx).Infof("TPM: cluster TPM mode: %s", res.ClusterConfiguration.TpmMode)
supervisor.Logger(ctx).Infof("TPM: node TPM usage: %v", res.TpmUsage)
@@ -158,6 +162,7 @@
for {
resC, err := cur.CommitNode(ctx, &ipb.CommitNodeRequest{
ClusterUnlockKey: cuk,
+ StorageSecurity: storageSecurity,
})
if err == nil {
supervisor.Logger(ctx).Infof("Registering: Commit successful, received certificate")
diff --git a/metropolis/node/core/curator/impl_leader_curator.go b/metropolis/node/core/curator/impl_leader_curator.go
index f698661..e3cc2cf 100644
--- a/metropolis/node/core/curator/impl_leader_curator.go
+++ b/metropolis/node/core/curator/impl_leader_curator.go
@@ -326,6 +326,11 @@
if err != nil {
return nil, err
}
+ nodeStorageSecurity, err := cl.NodeStorageSecurity()
+ if err != nil {
+ rpc.Trace(ctx).Printf("NodeStorageSecurity: %v", err)
+ return nil, status.Error(codes.InvalidArgument, "cannot generate recommended node storage security")
+ }
// Figure out if node should be using TPM.
tpmUsage, err := cl.NodeTPMUsage(req.HaveLocalTpm)
@@ -369,8 +374,9 @@
// Eat error, as we just deserialized this from a proto.
clusterConfig, _ := cl.proto()
return &ipb.RegisterNodeResponse{
- ClusterConfiguration: clusterConfig,
- TpmUsage: tpmUsage,
+ ClusterConfiguration: clusterConfig,
+ TpmUsage: tpmUsage,
+ RecommendedNodeStorageSecurity: nodeStorageSecurity,
}, nil
}
@@ -384,6 +390,16 @@
}
pubkey := pi.Unauthenticated.SelfSignedPublicKey
+ // First pass check of node storage security, before loading the cluster data and
+ // taking a lock on it.
+ switch req.StorageSecurity {
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE:
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_ENCRYPTED:
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED:
+ default:
+ return nil, status.Error(codes.InvalidArgument, "invalid storage_security (is it set?)")
+ }
+
// Doing a read-then-write operation below, take lock.
//
// MVP: This can lock up the cluster if too many RegisterNode calls get issued,
@@ -391,6 +407,14 @@
l.muNodes.Lock()
defer l.muNodes.Unlock()
+ cl, err := clusterLoad(ctx, l.leadership)
+ if err != nil {
+ return nil, err
+ }
+ if err := cl.ValidateNodeStorage(req.StorageSecurity); err != nil {
+ return nil, err
+ }
+
// Retrieve the node and act on its current state, either returning early or
// mutating it and continuing with the rest of the Commit logic.
id := identity.NodeID(pubkey)
@@ -420,8 +444,10 @@
// Check the given CUK is valid.
// TODO(q3k): unify length with localstorage/crypt keySize.
- if want, got := 32, len(req.ClusterUnlockKey); want != got {
- return nil, status.Errorf(codes.InvalidArgument, "invalid ClusterUnlockKey length, wanted %d bytes, got %d", want, got)
+ if req.StorageSecurity != cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE {
+ if want, got := 32, len(req.ClusterUnlockKey); want != got {
+ return nil, status.Errorf(codes.InvalidArgument, "invalid ClusterUnlockKey length, wanted %d bytes, got %d", want, got)
+ }
}
// Generate certificate for node, save new node state, return.
diff --git a/metropolis/node/core/curator/impl_leader_test.go b/metropolis/node/core/curator/impl_leader_test.go
index c28ba7a..b55efe6 100644
--- a/metropolis/node/core/curator/impl_leader_test.go
+++ b/metropolis/node/core/curator/impl_leader_test.go
@@ -697,6 +697,7 @@
// Make 'other node' commit itself into the cluster.
_, err = cur.CommitNode(ctx, &ipb.CommitNodeRequest{
ClusterUnlockKey: []byte("fakefakefakefakefakefakefakefake"),
+ StorageSecurity: cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED,
})
if err != nil {
t.Fatalf("CommitNode failed: %v", err)
@@ -1528,7 +1529,8 @@
t.Run(fmt.Sprintf("case %d", i), func(t *testing.T) {
cl := fakeLeader(t, &fakeLeaderOption{
icc: &cpb.ClusterConfiguration{
- TpmMode: te.mode,
+ TpmMode: te.mode,
+ StorageSecurityPolicy: cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION,
},
})
// Register node and make sure it's either successful or not, depending on the
@@ -1569,6 +1571,7 @@
_, err = cur.CommitNode(ctx, &ipb.CommitNodeRequest{
ClusterUnlockKey: []byte("fakefakefakefakefakefakefakefake"),
+ StorageSecurity: cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED,
})
if err != nil {
t.Fatalf("CommitNode failed: %v", err)
diff --git a/metropolis/node/core/curator/proto/api/api.proto b/metropolis/node/core/curator/proto/api/api.proto
index ea264e5..3c9cc6b 100644
--- a/metropolis/node/core/curator/proto/api/api.proto
+++ b/metropolis/node/core/curator/proto/api/api.proto
@@ -262,11 +262,18 @@
}
message RegisterNodeResponse {
- // cluster_configuration is currently returned to the node just for
- // informative reasons.
+ // cluster_configuration contains, amongst others, the cluster policies.
+ // These can be used by the node to eg. validate whether it is joining a
+ // secure cluster or to pick local settings based on said policies.
metropolis.proto.common.ClusterConfiguration cluster_configuration = 1;
// tpm_usage tells the node whether it should use its TPM or not.
metropolis.proto.common.NodeTPMUsage tpm_usage = 2;
+ // recommended_node_storage_security tells the node what storage security
+ // it should use according to the cluster. However, the node is free to
+ // choose any other storage security setting which still fits the cluster
+ // node storage policy, eg. due to administrative overrides in startup
+ // parameters, available CPU features, etc.
+ metropolis.proto.common.NodeStorageSecurity recommended_node_storage_security = 3;
}
message CommitNodeRequest {
@@ -274,6 +281,10 @@
// disk encryption key. The node submits it for safekeeping by the cluster,
// and keeps the local part (node unlock key, NUK) local, sealed by TPM.
bytes cluster_unlock_key = 1;
+ // storage_security is the node storage security setting which the node has
+ // implemented as part of its registration flow. The cluster will validate
+ // it against its configured policy.
+ metropolis.proto.common.NodeStorageSecurity storage_security = 2;
}
message CommitNodeResponse {
diff --git a/metropolis/node/core/curator/state_cluster.go b/metropolis/node/core/curator/state_cluster.go
index 226bf53..2ef7349 100644
--- a/metropolis/node/core/curator/state_cluster.go
+++ b/metropolis/node/core/curator/state_cluster.go
@@ -20,7 +20,8 @@
// Cluster is the cluster's configuration, as (un)marshaled to/from
// common.ClusterConfiguration.
type Cluster struct {
- TPMMode cpb.ClusterConfiguration_TPMMode
+ TPMMode cpb.ClusterConfiguration_TPMMode
+ StorageSecurityPolicy cpb.ClusterConfiguration_StorageSecurityPolicy
}
// DefaultClusterConfiguration is the default cluster configuration for a newly
@@ -28,7 +29,8 @@
// user.
func DefaultClusterConfiguration() *Cluster {
return &Cluster{
- TPMMode: cpb.ClusterConfiguration_TPM_MODE_REQUIRED,
+ TPMMode: cpb.ClusterConfiguration_TPM_MODE_REQUIRED,
+ StorageSecurityPolicy: cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION,
}
}
@@ -84,6 +86,52 @@
return
}
+// NodeStorageSecurity returns the recommended NodeStorageSecurity for nodes
+// joining the cluster.
+func (c *Cluster) NodeStorageSecurity() (security cpb.NodeStorageSecurity, err error) {
+ switch c.StorageSecurityPolicy {
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_PERMISSIVE:
+ // TODO(q3k): allow per-node configuration. Be conservative for now.
+ return cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED, nil
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION:
+ return cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED, nil
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION:
+ // TODO(q3k): allow per-node configuration. Be conservative for now.
+ return cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_ENCRYPTED, nil
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_INSECURE:
+ return cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE, nil
+ default:
+ return cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INVALID, fmt.Errorf("invalid cluster storage policy %d", c.StorageSecurityPolicy)
+ }
+}
+
+// ValidateNodeStorage checks the given NodeStorageSecurity and returns a gRPC
+// status if the security setting is not compliant with the cluster node storage
+// policy.
+func (c *Cluster) ValidateNodeStorage(ns cpb.NodeStorageSecurity) error {
+ switch c.StorageSecurityPolicy {
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_PERMISSIVE:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_INSECURE:
+ if ns != cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE {
+ return status.Error(codes.FailedPrecondition, "cluster policy requires insecure node storage")
+ }
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION:
+ switch ns {
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED:
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_ENCRYPTED:
+ default:
+ return status.Error(codes.FailedPrecondition, "cluster policy requires encrypted node storage")
+ }
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION:
+ if ns != cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED {
+ return status.Error(codes.FailedPrecondition, "cluster policy requires encrypted and authenticated node storage")
+ }
+ default:
+ return status.Error(codes.Internal, "cannot interpret cluster node storage policy")
+ }
+ return nil
+}
+
func clusterUnmarshal(data []byte) (*Cluster, error) {
msg := cpb.ClusterConfiguration{}
if err := proto.Unmarshal(data, &msg); err != nil {
@@ -101,8 +149,18 @@
return nil, fmt.Errorf("invalid TpmMode: %v", cc.TpmMode)
}
+ switch cc.StorageSecurityPolicy {
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_PERMISSIVE:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_INSECURE:
+ default:
+ return nil, fmt.Errorf("invalid StorageSecurityPolicy: %v", cc.StorageSecurityPolicy)
+ }
+
c := &Cluster{
- TPMMode: cc.TpmMode,
+ TPMMode: cc.TpmMode,
+ StorageSecurityPolicy: cc.StorageSecurityPolicy,
}
return c, nil
@@ -116,8 +174,19 @@
default:
return nil, fmt.Errorf("invalid TPMMode %d", c.TPMMode)
}
+
+ switch c.StorageSecurityPolicy {
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_PERMISSIVE:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION:
+ case cpb.ClusterConfiguration_STORAGE_SECURITY_POLICY_NEEDS_INSECURE:
+ default:
+ return nil, fmt.Errorf("invalid StorageSecurityPolicy %d", c.StorageSecurityPolicy)
+ }
+
return &cpb.ClusterConfiguration{
- TpmMode: c.TPMMode,
+ TpmMode: c.TPMMode,
+ StorageSecurityPolicy: c.StorageSecurityPolicy,
}, nil
}
diff --git a/metropolis/node/core/localstorage/directory_data.go b/metropolis/node/core/localstorage/directory_data.go
index f824ac2..c255089 100644
--- a/metropolis/node/core/localstorage/directory_data.go
+++ b/metropolis/node/core/localstorage/directory_data.go
@@ -26,14 +26,36 @@
"source.monogon.dev/metropolis/node/core/localstorage/crypt"
"source.monogon.dev/metropolis/node/core/localstorage/declarative"
"source.monogon.dev/metropolis/pkg/tpm"
+ cpb "source.monogon.dev/metropolis/proto/common"
ppb "source.monogon.dev/metropolis/proto/private"
)
var keySize uint16 = 256 / 8
-// MountData mounts the node data partition with the given cluster unlock key.
+// MountExisting mounts the node data partition with the given cluster unlock key.
// It automatically unseals the node unlock key from the TPM.
func (d *DataDirectory) MountExisting(config *ppb.SealedConfiguration, clusterUnlockKey []byte) error {
+ var mode crypt.Mode
+ switch config.StorageSecurity {
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE:
+ mode = crypt.ModeInsecure
+ if len(clusterUnlockKey) != 0 {
+ return fmt.Errorf("storage security set to insecure, but cluster unlock key received")
+ }
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_ENCRYPTED:
+ mode = crypt.ModeEncrypted
+ if len(clusterUnlockKey) != 32 {
+ return fmt.Errorf("storage security set to encrypted, but invalid cluster unlock key received")
+ }
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED:
+ mode = crypt.ModeEncryptedAuthenticated
+ if len(clusterUnlockKey) != 32 {
+ return fmt.Errorf("storage security set to encrypted and authenticated, but invalid cluster unlock key received")
+ }
+ default:
+ return fmt.Errorf("invalid storage security in sealed configuration: %d", config.StorageSecurity)
+ }
+
d.flagLock.Lock()
defer d.flagLock.Unlock()
@@ -50,7 +72,7 @@
key[i] = config.NodeUnlockKey[i] ^ clusterUnlockKey[i]
}
- target, err := crypt.Map("data", crypt.NodeDataRawPath, key, crypt.ModeEncryptedAuthenticated)
+ target, err := crypt.Map("data", crypt.NodeDataRawPath, key, mode)
if err != nil {
return err
}
@@ -60,12 +82,13 @@
return nil
}
-// InitializeData initializes the node data partition and returns the node and
-// cluster unlock keys. It seals the local portion into the TPM. This is a
-// potentially slow operation since it touches the whole partition.
-func (d *DataDirectory) MountNew(config *ppb.SealedConfiguration) ([]byte, error) {
+// MountNew initializes the node data partition and returns the cluster unlock
+// key. It seals the local portion into the TPM. This is a potentially slow
+// operation since it touches the whole partition.
+func (d *DataDirectory) MountNew(config *ppb.SealedConfiguration, security cpb.NodeStorageSecurity) ([]byte, error) {
d.flagLock.Lock()
defer d.flagLock.Unlock()
+
if !d.canMount {
return nil, fmt.Errorf("cannot mount yet (root not ready?)")
}
@@ -74,36 +97,53 @@
}
d.mounted = true
- var nodeUnlockKey, globalUnlockKey []byte
- var err error
- if tpm.IsInitialized() {
- nodeUnlockKey, err = tpm.GenerateSafeKey(keySize)
- } else {
- nodeUnlockKey = make([]byte, keySize)
- _, err = rand.Read(nodeUnlockKey)
+ var mode crypt.Mode
+ switch security {
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED:
+ mode = crypt.ModeEncryptedAuthenticated
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_ENCRYPTED:
+ mode = crypt.ModeEncrypted
+ case cpb.NodeStorageSecurity_NODE_STORAGE_SECURITY_INSECURE:
+ mode = crypt.ModeInsecure
+ default:
+ return nil, fmt.Errorf("invalid node storage security: %d", security)
}
- if err != nil {
- return nil, fmt.Errorf("generating node unlock key: %w", err)
- }
- if tpm.IsInitialized() {
- globalUnlockKey, err = tpm.GenerateSafeKey(keySize)
- } else {
- globalUnlockKey = make([]byte, keySize)
- _, err = rand.Read(globalUnlockKey)
- }
- if err != nil {
- return nil, fmt.Errorf("generating cluster unlock key: %w", err)
+ config.StorageSecurity = security
+
+ var nodeUnlockKey, clusterUnlockKey, key []byte
+
+ // Generate keys unless we're in insecure mode.
+ if mode != crypt.ModeInsecure {
+ var err error
+ if tpm.IsInitialized() {
+ nodeUnlockKey, err = tpm.GenerateSafeKey(keySize)
+ } else {
+ nodeUnlockKey = make([]byte, keySize)
+ _, err = rand.Read(nodeUnlockKey)
+ }
+ if err != nil {
+ return nil, fmt.Errorf("generating node unlock key: %w", err)
+ }
+ if tpm.IsInitialized() {
+ clusterUnlockKey, err = tpm.GenerateSafeKey(keySize)
+ } else {
+ clusterUnlockKey = make([]byte, keySize)
+ _, err = rand.Read(clusterUnlockKey)
+ }
+ if err != nil {
+ return nil, fmt.Errorf("generating cluster unlock key: %w", err)
+ }
+
+ // The actual key is generated by XORing together the nodeUnlockKey and the
+ // globalUnlockKey This provides us with a mathematical guarantee that the
+ // resulting key cannot be recovered without knowledge of both parts.
+ key = make([]byte, keySize)
+ for i := uint16(0); i < keySize; i++ {
+ key[i] = nodeUnlockKey[i] ^ clusterUnlockKey[i]
+ }
}
- // The actual key is generated by XORing together the nodeUnlockKey and the
- // globalUnlockKey This provides us with a mathematical guarantee that the
- // resulting key cannot be recovered whithout knowledge of both parts.
- key := make([]byte, keySize)
- for i := uint16(0); i < keySize; i++ {
- key[i] = nodeUnlockKey[i] ^ globalUnlockKey[i]
- }
-
- target, err := crypt.Init("data", crypt.NodeDataRawPath, key, crypt.ModeEncryptedAuthenticated)
+ target, err := crypt.Init("data", crypt.NodeDataRawPath, key, mode)
if err != nil {
return nil, fmt.Errorf("initializing encrypted block device: %w", err)
}
@@ -135,7 +175,7 @@
config.NodeUnlockKey = nodeUnlockKey
- return globalUnlockKey, nil
+ return clusterUnlockKey, nil
}
func (d *DataDirectory) mount(path string) error {
diff --git a/metropolis/proto/common/common.proto b/metropolis/proto/common/common.proto
index 8e0fa58..28becd3 100644
--- a/metropolis/proto/common/common.proto
+++ b/metropolis/proto/common/common.proto
@@ -301,6 +301,25 @@
TPM_MODE_DISABLED = 3;
}
TPMMode tpm_mode = 1;
+
+ // storage_security_policy defines which node storage security settings are
+ // accepted by the cluster. Nodes are informed of the cluster policy when
+ // registering into the cluster, alongside a cluster-recommended storage
+ // security setting. The node then reports its selected node storage setting
+ // during its Commit call which the cluster verifies against its policy.
+ enum StorageSecurityPolicy {
+ STORAGE_SECURITY_POLICY_INVALID = 0;
+ // The cluster accepts any storage security.
+ STORAGE_SECURITY_POLICY_PERMISSIVE = 1;
+ // The cluster accepts any storage security that offers encryption.
+ STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION = 2;
+ // The cluster accepts any storage security that offers encryption and
+ // authentication.
+ STORAGE_SECURITY_POLICY_NEEDS_ENCRYPTION_AND_AUTHENTICATION = 3;
+ // The cluster only accepts unencrypted and unauthenticated node storage.
+ STORAGE_SECURITY_POLICY_NEEDS_INSECURE = 4;
+ }
+ StorageSecurityPolicy storage_security_policy = 2;
}
// NodeTPMUsage describes whether a node has a TPM2.0 and if it is/should be
@@ -314,4 +333,22 @@
NODE_TPM_PRESENT_BUT_UNUSED = 2;
// This node has a TPM 2.0 and it is being actively used.
NODE_TPM_PRESENT_AND_USED = 3;
+}
+
+// NodeStorageSecurity describes how a node encrypts and/or authenticates its
+// local storage. In other words, it's a configuration setting for disk
+// encryption (ie. via dm-crypt) and disk integrity (ie. via dm-integrity) of
+// the Metropolis data partition.
+enum NodeStorageSecurity {
+ NODE_STORAGE_SECURITY_INVALID = 0;
+ // The node has unencrypted and unauthenticated disk storage. Its data
+ // partition is a plain XFS partition, and the node's credentials are stored
+ // on it directly.
+ NODE_STORAGE_SECURITY_INSECURE = 1;
+ // The node has encrypted but unauthenticated disk storage. Its data
+ // partition is an XFS partition mounted through dm-crypt.
+ NODE_STORAGE_SECURITY_ENCRYPTED = 2;
+ // The node has encrypted and authenticated storage. Its data
+ // partition is an XFS partition mounted through dm-integrity and dm-crypt.
+ NODE_STORAGE_SECURITY_AUTHENTICATED_ENCRYPTED = 3;
}
\ No newline at end of file
diff --git a/metropolis/proto/private/BUILD.bazel b/metropolis/proto/private/BUILD.bazel
index 57b7ba4..1729ce9 100644
--- a/metropolis/proto/private/BUILD.bazel
+++ b/metropolis/proto/private/BUILD.bazel
@@ -6,6 +6,7 @@
name = "private_proto",
srcs = ["private.proto"],
visibility = ["//metropolis:__subpackages__"],
+ deps = ["//metropolis/proto/common:common_proto"],
)
go_proto_library(
@@ -13,6 +14,7 @@
importpath = "source.monogon.dev/metropolis/proto/private",
proto = ":private_proto",
visibility = ["//metropolis:__subpackages__"],
+ deps = ["//metropolis/proto/common"],
)
go_library(
diff --git a/metropolis/proto/private/private.proto b/metropolis/proto/private/private.proto
index c682311..4cdbef2 100644
--- a/metropolis/proto/private/private.proto
+++ b/metropolis/proto/private/private.proto
@@ -18,6 +18,8 @@
option go_package = "source.monogon.dev/metropolis/proto/private";
package metropolis.proto.private;
+import "metropolis/proto/common/common.proto";
+
// Node describes a single node's state in etcd
// DEPRECATED: this will be moved to //metropolis/node/curator.
message Node {
@@ -73,4 +75,9 @@
// cluster_ca is the X509 CA certificate of the cluster set during
// registration and used by nodes joining the cluster.
bytes cluster_ca = 3;
+ // storage_security is the node storage security that this node has been
+ // created with, and is used to determine the way the local storage (ie.
+ // Metropolis data partition) will be attempted to be mounted on subsequent
+ // node startups.
+ metropolis.proto.common.NodeStorageSecurity storage_security = 4;
}