metropolis/proto: EnrolmentConfig -> NodeParameters
This starts off the move to a node configuration API conforming to
the lifecycle management design document.
Instead of an Enrolment Config used only to join an existing cluster, we
move to a NodeParameters proto that must always be given to a node if
it's supposed to either bootstrap a new cluster or join an existing one.
This links the existing cluster management code (and its state machine)
to work with this file. However, that state machine will be removed very
soon, anyway.
We also remove everything related to golden tickets.
This breaks multi-node tests.
X-Origin-Diff: phab/D710
GitOrigin-RevId: f22615fbccab975f2d5e6928bdc7387ab3aa5714
diff --git a/metropolis/node/build/mkimage/main.go b/metropolis/node/build/mkimage/main.go
index c2f7d5a..5546055 100644
--- a/metropolis/node/build/mkimage/main.go
+++ b/metropolis/node/build/mkimage/main.go
@@ -18,7 +18,7 @@
// mkimage is a tool to generate a Metropolis node disk image containing the
// given EFI payload, and optionally, a given external initramfs image and
-// enrolment credentials.
+// node parameters
import (
"flag"
@@ -38,13 +38,13 @@
var NodeSystemPartition gpt.Type = gpt.Type("ee96055b-f6d0-4267-8bbb-724b2afea74c")
var (
- flagEFI string
- flagOut string
- flagSystemPath string
- flagEnrolmentCredentials string
- flagDataPartitionSize uint64
- flagESPPartitionSize uint64
- flagSystemPartitionSize uint64
+ flagEFI string
+ flagOut string
+ flagSystemPath string
+ flagNodeParameters string
+ flagDataPartitionSize uint64
+ flagESPPartitionSize uint64
+ flagSystemPartitionSize uint64
)
func mibToSectors(size uint64) uint64 {
@@ -67,7 +67,7 @@
flag.StringVar(&flagEFI, "efi", "", "UEFI payload")
flag.StringVar(&flagOut, "out", "", "Output disk image")
flag.StringVar(&flagSystemPath, "system", "", "System partition [optional]")
- flag.StringVar(&flagEnrolmentCredentials, "enrolment_credentials", "", "Enrolment credentials [optional]")
+ flag.StringVar(&flagNodeParameters, "node_parameters", "", "Node parameters [optional]")
flag.Uint64Var(&flagDataPartitionSize, "data_partition_size", 2048, "Override the data partition size (default 2048 MiB)")
flag.Uint64Var(&flagESPPartitionSize, "esp_partition_size", 128, "Override the ESP partition size (default: 128MiB)")
flag.Uint64Var(&flagSystemPartitionSize, "system_partition_size", 1024, "Override the System partition size (default: 1024MiB)")
@@ -144,8 +144,8 @@
put(fs, flagEFI, "/EFI/BOOT/BOOTX64.EFI")
- if flagEnrolmentCredentials != "" {
- put(fs, flagEnrolmentCredentials, "/EFI/metropolis/enrolment.pb")
+ if flagNodeParameters != "" {
+ put(fs, flagNodeParameters, "/EFI/metropolis/parameters.pb")
}
if err := diskImg.File.Close(); err != nil {
diff --git a/metropolis/node/core/BUILD.bazel b/metropolis/node/core/BUILD.bazel
index 28f4d2a..e7e6179 100644
--- a/metropolis/node/core/BUILD.bazel
+++ b/metropolis/node/core/BUILD.bazel
@@ -16,7 +16,6 @@
deps = [
"//metropolis/node:go_default_library",
"//metropolis/node/core/cluster:go_default_library",
- "//metropolis/node/core/consensus/ca:go_default_library",
"//metropolis/node/core/localstorage:go_default_library",
"//metropolis/node/core/localstorage/declarative:go_default_library",
"//metropolis/node/core/network:go_default_library",
diff --git a/metropolis/node/core/cluster/BUILD.bazel b/metropolis/node/core/cluster/BUILD.bazel
index ad2620c..4490e1f 100644
--- a/metropolis/node/core/cluster/BUILD.bazel
+++ b/metropolis/node/core/cluster/BUILD.bazel
@@ -9,10 +9,8 @@
importpath = "source.monogon.dev/metropolis/node/core/cluster",
visibility = ["//metropolis/node/core:__subpackages__"],
deps = [
- "//metropolis/node:go_default_library",
"//metropolis/node/core/consensus:go_default_library",
"//metropolis/node/core/localstorage:go_default_library",
- "//metropolis/node/core/localstorage/declarative:go_default_library",
"//metropolis/node/core/network:go_default_library",
"//metropolis/pkg/supervisor:go_default_library",
"//metropolis/proto/api:go_default_library",
@@ -20,6 +18,7 @@
"@com_github_cenkalti_backoff_v4//:go_default_library",
"@com_github_golang_protobuf//proto:go_default_library",
"@io_etcd_go_etcd//clientv3:go_default_library",
+ "@org_golang_google_protobuf//proto:go_default_library",
"@org_golang_x_sys//unix:go_default_library",
],
)
diff --git a/metropolis/node/core/cluster/manager.go b/metropolis/node/core/cluster/manager.go
index 1df9bac..6d6d592 100644
--- a/metropolis/node/core/cluster/manager.go
+++ b/metropolis/node/core/cluster/manager.go
@@ -18,23 +18,18 @@
import (
"context"
- "crypto/x509"
- "encoding/pem"
"fmt"
"io/ioutil"
"os"
- "strings"
"sync"
"time"
"github.com/cenkalti/backoff/v4"
- "github.com/golang/protobuf/proto"
"go.etcd.io/etcd/clientv3"
+ "google.golang.org/protobuf/proto"
- common "source.monogon.dev/metropolis/node"
"source.monogon.dev/metropolis/node/core/consensus"
"source.monogon.dev/metropolis/node/core/localstorage"
- "source.monogon.dev/metropolis/node/core/localstorage/declarative"
"source.monogon.dev/metropolis/node/core/network"
"source.monogon.dev/metropolis/pkg/supervisor"
apb "source.monogon.dev/metropolis/proto/api"
@@ -73,9 +68,6 @@
// reaches a final state (Running or Failed respectively).
stateWaiters []chan bool
- // goldenTicket is the Golden Ticket present in the enrolment config, if any.
- goldenTicket *apb.GoldenTicket
-
// consensus is the spawned etcd/consensus service, if the Manager brought up a Node that should run one.
consensus *consensus.Service
}
@@ -98,9 +90,6 @@
// StateCreatingCluster is when the Manager attempts to create a new cluster - this happens when a node is started
// with no EnrolmentConfig.
StateCreatingCluster
- // StateCharlie is when the Manager uses the Golden Ticket debug/stopgap system to join an already
- // existing cluster. This mechanism will be removed before the first Metropolis release.
- StateCharlie
// StateRunning is when the Manager successfully got the node to be part of a cluster. stateRunningNode is valid.
StateRunning
// StateFailed is when the Manager failed to ge the node to be part of a cluster.
@@ -113,8 +102,6 @@
return "New"
case StateCreatingCluster:
return "CreatingCluster"
- case StateCharlie:
- return "Charlie"
case StateRunning:
return "Running"
case StateFailed:
@@ -126,9 +113,8 @@
// allowedTransition describes all allowed state transitions (map[From][]To).
var allowedTransitions = map[State][]State{
- StateNew: {StateCreatingCluster, StateCharlie},
+ StateNew: {StateCreatingCluster},
StateCreatingCluster: {StateRunning, StateFailed},
- StateCharlie: {StateRunning, StateFailed},
}
// allowed returns whether a transition from a state to another state is allowed (ie. is defined in allowedTransitions).
@@ -221,8 +207,6 @@
err = m.stateNew(ctx)
case StateCreatingCluster:
err = m.stateCreatingCluster(ctx)
- case StateCharlie:
- err = m.stateCharlie(ctx)
default:
done = true
break
@@ -261,40 +245,37 @@
func (m *Manager) stateNew(ctx context.Context) error {
supervisor.Logger(ctx).Info("Starting enrolment process...")
+ // STOPGAP when migrating to enrolment config and cluster lifecycle: always
+ // expect NodeParameters with ClusterBootstrap.
+
// Check for presence of EnrolmentConfig on ESP or in qemu firmware variables.
var configRaw []byte
- configRaw, err := m.storageRoot.ESP.Enrolment.Read()
+ configRaw, err := m.storageRoot.ESP.NodeParameters.Read()
if err != nil && !os.IsNotExist(err) {
return fmt.Errorf("could not read local enrolment file: %w", err)
} else if err != nil {
- configRaw, err = ioutil.ReadFile("/sys/firmware/qemu_fw_cfg/by_name/dev.monogon.metropolis/enrolment.pb/raw")
+ configRaw, err = ioutil.ReadFile("/sys/firmware/qemu_fw_cfg/by_name/dev.monogon.metropolis/parameters.pb/raw")
if err != nil && !os.IsNotExist(err) {
return fmt.Errorf("could not read firmware enrolment file: %w", err)
}
}
- // If no enrolment file exists, we create a new cluster.
if configRaw == nil {
- m.next(ctx, StateCreatingCluster)
- return nil
+ return fmt.Errorf("no enrolment config present")
}
- // Enrolment file exists, parse it.
-
- enrolmentConfig := apb.EnrolmentConfig{}
- if err := proto.Unmarshal(configRaw, &enrolmentConfig); err != nil {
- return fmt.Errorf("could not unmarshal local enrolment file: %w", err)
+ parameters := &apb.NodeParameters{}
+ if err := proto.Unmarshal(configRaw, parameters); err != nil {
+ return fmt.Errorf("enrolment config could not get unmarshaled: %w", err)
}
- // If no join ticket exists, we can't do anything yet.
- if enrolmentConfig.GoldenTicket == nil {
- return fmt.Errorf("joining a cluster without a golden ticket not yet implemented")
+ switch parameters.Cluster.(type) {
+ case *apb.NodeParameters_ClusterBootstrap_:
+ default:
+ return fmt.Errorf("enrolment config has no ClusterBootstrap: %w", err)
}
- m.goldenTicket = enrolmentConfig.GoldenTicket
-
- // Otherwise, we begin enrolling with the Golden Ticket.
- m.next(ctx, StateCharlie)
+ m.next(ctx, StateCreatingCluster)
return nil
}
@@ -380,121 +361,6 @@
return nil
}
-// stateCharlie is used to join an existing cluster via the GoldenTicket mechanism. This mechanism is temporarily
-// implemented in Metropolis in order to allow for testing multi-node clusters without a TPM attestation flow implemented.
-// The Golden Ticket contains a pregenerated node certificate, etcd certificate, and other data that any node can
-// use to join the cluster.
-// Since this flow is temporary, it has a slight impedance mismatch with methods exposed by localstorage, node, etc.,
-// and the resulting sequencing is a bit odd:
-// - the {node,etcd} certificates/keys are loaded (this already dictates the new node name, as the node name is based
-// off of the node public key)
-// - local storage is initialized, a local/cluster unlock keypair is generated
-// - etcd keys are manually saved to localstorage (vs. being generated locally by CA)
-// - an etcd/consensus member is started, knowing that the remote member was already generated when the golden ticket
-// was generated (vs. being created now by an RPC call, via an promote-node-to-etcd-member flow)
-// - the node is then promoted to a consensus member and kubernetes worker, its clusterunlock key is set, and then it
-// is saved to etcd.
-// As such, in this flow, we first create an etcd member (on goldenticket generation), and then only create a new
-// Metropolis node (when the goldenticket is used).
-func (m *Manager) stateCharlie(ctx context.Context) error {
- t := m.goldenTicket
- nodeCert, err := x509.ParseCertificate(t.NodeCert)
- if err != nil {
- return fmt.Errorf("parsing node certificate from ticket: %w", err)
- }
-
- supervisor.Logger(ctx).Info("Joining cluster: waiting for IP address...")
- ip, err := m.networkService.GetIP(ctx, true)
- if err != nil {
- return fmt.Errorf("when getting IP address: %w", err)
- }
- supervisor.Logger(ctx).Info("Joining cluster: got IP address %s", ip.String())
-
- supervisor.Logger(ctx).Info("Joining cluster: initializing storage...")
- cuk, err := m.storageRoot.Data.MountNew(&m.storageRoot.ESP.LocalUnlock)
- if err != nil {
- return fmt.Errorf("when making new data partition: %w", err)
- }
- supervisor.Logger(ctx).Info("Joining cluster: storage initialized")
- node := NewNode(cuk, *ip, *nodeCert)
-
- // Save etcd PKI to disk.
- for _, f := range []struct {
- target declarative.FilePlacement
- data []byte
- blockType string
- }{
- {m.storageRoot.Data.Etcd.PeerPKI.Key, t.EtcdClientKey, "PRIVATE KEY"},
- {m.storageRoot.Data.Etcd.PeerPKI.Certificate, t.EtcdClientCert, "CERTIFICATE"},
- {m.storageRoot.Data.Etcd.PeerPKI.CACertificate, t.EtcdCaCert, "CERTIFICATE"},
- } {
- if err := f.target.Write(pem.EncodeToMemory(&pem.Block{Type: f.blockType, Bytes: f.data}), 0600); err != nil {
- return fmt.Errorf("when writing etcd PKI data: %w", err)
- }
- }
- if err := m.storageRoot.Data.Etcd.PeerCRL.Write(t.EtcdCrl, 0600); err != nil {
- return fmt.Errorf("when writing etcd CRL: %w", err)
- }
-
- https := func(p *apb.GoldenTicket_EtcdPeer) string {
- return fmt.Sprintf("%s=https://%s:%d", p.Name, p.Address, common.ConsensusPort)
- }
- var initialCluster []string
- for _, p := range t.Peers {
- initialCluster = append(initialCluster, https(p))
- }
- initialCluster = append(initialCluster, https(t.This))
-
- supervisor.Logger(ctx).Infof("Joining cluster: starting etcd join, name: %s, initial_cluster: %s", node.ID(), strings.Join(initialCluster, ","))
- m.consensus = consensus.New(consensus.Config{
- Data: &m.storageRoot.Data.Etcd,
- Ephemeral: &m.storageRoot.Ephemeral.Consensus,
- Name: node.ID(),
- InitialCluster: strings.Join(initialCluster, ","),
- ExternalHost: ip.String(),
- ListenHost: ip.String(),
- })
-
- if err := supervisor.Run(ctx, "consensus", m.consensus.Run); err != nil {
- return fmt.Errorf("when starting consensus: %w", err)
- }
-
- // TODO(q3k): make timeout configurable?
- ctxT, ctxC := context.WithTimeout(ctx, 5*time.Second)
- defer ctxC()
-
- supervisor.Logger(ctx).Info("Joining cluster: waiting for consensus...")
- if err := m.consensus.WaitReady(ctxT); err != nil {
- return fmt.Errorf("consensus service failed to become ready: %w", err)
- }
-
- // Configure node to be a consensus member and kubernetes worker. In the future, different nodes will have
- // different roles, but for now they're all symmetrical.
- _, consensusName, err := m.consensus.MemberInfo(ctx)
- if err != nil {
- return fmt.Errorf("could not get consensus MemberInfo: %w", err)
- }
- if err := node.MakeConsensusMember(consensusName); err != nil {
- return fmt.Errorf("could not make new node into consensus member: %w", err)
- }
- if err := node.MakeKubernetesWorker(node.ID()); err != nil {
- return fmt.Errorf("could not make new node into kubernetes worker: %w", err)
- }
-
- // Save node into etcd.
- supervisor.Logger(ctx).Info("Creating new cluster: storing first node...")
- if err := node.Store(ctx, m.consensus.KV("cluster", "enrolment")); err != nil {
- return fmt.Errorf("could not save new node: %w", err)
- }
-
- m.stateLock.Lock()
- m.stateRunningNode = node
- m.stateLock.Unlock()
-
- m.next(ctx, StateRunning)
- return nil
-}
-
// Node returns the Node that the Manager brought into a cluster, or nil if the Manager is not Running.
// This is safe to call from any goroutine.
func (m *Manager) Node() *Node {
diff --git a/metropolis/node/core/debug_service.go b/metropolis/node/core/debug_service.go
index 62a9a8a..6dee3d9 100644
--- a/metropolis/node/core/debug_service.go
+++ b/metropolis/node/core/debug_service.go
@@ -18,16 +18,11 @@
import (
"context"
- "crypto/x509"
- "fmt"
- "net"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
- common "source.monogon.dev/metropolis/node"
"source.monogon.dev/metropolis/node/core/cluster"
- "source.monogon.dev/metropolis/node/core/consensus/ca"
"source.monogon.dev/metropolis/node/kubernetes"
"source.monogon.dev/metropolis/pkg/logtree"
apb "source.monogon.dev/metropolis/proto/api"
@@ -44,61 +39,6 @@
logtree *logtree.LogTree
}
-func (s *debugService) GetGoldenTicket(ctx context.Context, req *apb.GetGoldenTicketRequest) (*apb.GetGoldenTicketResponse, error) {
- ip := net.ParseIP(req.ExternalIp)
- if ip == nil {
- return nil, status.Errorf(codes.InvalidArgument, "could not parse IP %q", req.ExternalIp)
- }
- this := s.cluster.Node()
-
- certRaw, key, err := s.nodeCertificate()
- if err != nil {
- return nil, status.Errorf(codes.Unavailable, "failed to generate node certificate: %v", err)
- }
- cert, err := x509.ParseCertificate(certRaw)
- if err != nil {
- panic(err)
- }
- kv := s.cluster.ConsensusKVRoot()
- ca, err := ca.Load(ctx, kv)
- if err != nil {
- return nil, status.Errorf(codes.Unavailable, "could not load CA: %v", err)
- }
- etcdCert, etcdKey, err := ca.Issue(ctx, kv, cert.Subject.CommonName, ip)
- if err != nil {
- return nil, status.Errorf(codes.Unavailable, "could not generate etcd peer certificate: %v", err)
- }
- etcdCRL, err := ca.GetCurrentCRL(ctx, kv)
- if err != nil {
- return nil, status.Errorf(codes.Unavailable, "could not get etcd CRL: %v", err)
- }
-
- // Add new etcd member to etcd cluster.
- etcd := s.cluster.ConsensusCluster()
- etcdAddr := fmt.Sprintf("https://%s:%d", ip.String(), common.ConsensusPort)
- _, err = etcd.MemberAddAsLearner(ctx, []string{etcdAddr})
- if err != nil {
- return nil, status.Errorf(codes.Unavailable, "could not add as new etcd consensus member: %v", err)
- }
-
- return &apb.GetGoldenTicketResponse{
- Ticket: &apb.GoldenTicket{
- EtcdCaCert: ca.CACertRaw,
- EtcdClientCert: etcdCert,
- EtcdClientKey: etcdKey,
- EtcdCrl: etcdCRL,
- Peers: []*apb.GoldenTicket_EtcdPeer{
- {Name: this.ID(), Address: this.Address().String()},
- },
- This: &apb.GoldenTicket_EtcdPeer{Name: cert.Subject.CommonName, Address: ip.String()},
-
- NodeId: cert.Subject.CommonName,
- NodeCert: certRaw,
- NodeKey: key,
- },
- }, nil
-}
-
func (s *debugService) GetDebugKubeconfig(ctx context.Context, req *apb.GetDebugKubeconfigRequest) (*apb.GetDebugKubeconfigResponse, error) {
return s.kubernetes.GetDebugKubeconfig(ctx, req)
}
diff --git a/metropolis/node/core/localstorage/storage.go b/metropolis/node/core/localstorage/storage.go
index 73d33e1..a1f567b 100644
--- a/metropolis/node/core/localstorage/storage.go
+++ b/metropolis/node/core/localstorage/storage.go
@@ -63,10 +63,8 @@
// ESPDirectory is the EFI System Partition.
type ESPDirectory struct {
declarative.Directory
- LocalUnlock ESPLocalUnlockFile `file:"local_unlock.bin"`
- // Enrolment is the configuration/provisioning file for this node, containing information required to begin
- // joining the cluster.
- Enrolment declarative.File `file:"enrolment.pb"`
+ LocalUnlock ESPLocalUnlockFile `file:"local_unlock.bin"`
+ NodeParameters ESPNodeParameters `file:"parameters.pb"`
}
// ESPLocalUnlockFile is the localUnlock file, encrypted by the TPM of this node. After decrypting by the TPM it is used
@@ -75,6 +73,13 @@
declarative.File
}
+// ESPNodeParameters is the configuration for this node when first
+// bootstrapping a cluster or registering into an existing one. It's a
+// api.NodeParameters protobuf message.
+type ESPNodeParameters struct {
+ declarative.File
+}
+
// DataDirectory is an xfs partition mounted via cryptsetup/LUKS, with a key derived from {global,local}Unlock keys.
type DataDirectory struct {
declarative.Directory