m/n/core: factor out gRPC/TLS into rpc and identity libraries

This is an annoying large change, which started its life as me pulling
the 'let's add tests for authentication' thread, and ended up in
unifying a whole bunch of dispersed logic under two new libraries.

Notable changes:

 - m/n/core/identity now contains the NodeCertificate (now called Node)
   and NodeCredentials types. These used to exist in the cluster code,
   but were factored out to prevent loops between the curator, the
   cluster enrolment logic, and other code. They can now be shared by
   nearly all of the node code, removing the need for some conversions
   between subsystems/packages.
 - Alongside Node{,Credentials} types, the identity package contains
   code that creates x509 certificate templates and verifies x509
   certificates, and has functions specific to nodes and users - not
   clients and servers. This allows moving most of the rest of
   certificate checking code into a single set of functions, and allows
   us to test this logic thoroughly.
 - pki.{Client,Server,CA} are not used by the node core code anymore,
   and can now be moved to kubernetes-specific code (as that was their
   original purpose and that's their only current use).
 - m/n/core/rpc has been refactored to deduplicate code between the
   local/external gRPC servers and unary/stream interceptors for these
   servers, also allowing for more thorough testing and unified
   behaviour between all.
 - A PeerInfo structure is now injected into all gRPC handlers, and is
   unified to contain information both about nodes, users, and possibly
   unauthenticated callers.
 - The AAA.Escrow implementation now makes use of PeerInfo in order to
   retrieve the client's certificate, instead of rolling its own logic.
 - The EphemeralClusterCredentials test helper has been moved to the rpc
   library, and now returns identity objects, allowing for simplified
   test code (less juggling of bare public keys and
   {x509,tls}.Certificate objects).

Change-Id: I9284966b4f18c0d7628167ca3168b4b4037808c1
Reviewed-on: https://review.monogon.dev/c/monogon/+/325
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/core/rpc/BUILD.bazel b/metropolis/node/core/rpc/BUILD.bazel
index df03356..d281a92 100644
--- a/metropolis/node/core/rpc/BUILD.bazel
+++ b/metropolis/node/core/rpc/BUILD.bazel
@@ -1,15 +1,20 @@
-load("@io_bazel_rules_go//go:def.bzl", "go_library")
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
 
 go_library(
     name = "go_default_library",
     srcs = [
         "client.go",
+        "methodinfo.go",
+        "peerinfo.go",
         "server.go",
+        "server_authentication.go",
+        "testhelpers.go",
     ],
     importpath = "source.monogon.dev/metropolis/node/core/rpc",
     visibility = ["//visibility:public"],
     deps = [
         "//metropolis/node/core/curator/proto/api:go_default_library",
+        "//metropolis/node/core/identity:go_default_library",
         "//metropolis/pkg/pki:go_default_library",
         "//metropolis/proto/api:go_default_library",
         "//metropolis/proto/ext:go_default_library",
@@ -18,8 +23,23 @@
         "@org_golang_google_grpc//credentials:go_default_library",
         "@org_golang_google_grpc//peer:go_default_library",
         "@org_golang_google_grpc//status:go_default_library",
+        "@org_golang_google_grpc//test/bufconn:go_default_library",
         "@org_golang_google_protobuf//proto:go_default_library",
         "@org_golang_google_protobuf//reflect/protoreflect:go_default_library",
         "@org_golang_google_protobuf//reflect/protoregistry:go_default_library",
     ],
 )
+
+go_test(
+    name = "go_default_test",
+    srcs = ["server_authentication_test.go"],
+    embed = [":go_default_library"],
+    deps = [
+        "//metropolis/node/core/curator/proto/api:go_default_library",
+        "//metropolis/proto/api:go_default_library",
+        "//metropolis/proto/ext:go_default_library",
+        "@org_golang_google_grpc//codes:go_default_library",
+        "@org_golang_google_grpc//status:go_default_library",
+        "@org_golang_google_grpc//test/bufconn:go_default_library",
+    ],
+)
diff --git a/metropolis/node/core/rpc/client.go b/metropolis/node/core/rpc/client.go
index cc48f95..10d2545 100644
--- a/metropolis/node/core/rpc/client.go
+++ b/metropolis/node/core/rpc/client.go
@@ -8,15 +8,37 @@
 	"crypto/x509"
 	"fmt"
 	"math/big"
+	"net"
 	"time"
 
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/credentials"
+	"google.golang.org/grpc/test/bufconn"
 
+	"source.monogon.dev/metropolis/node/core/identity"
 	"source.monogon.dev/metropolis/pkg/pki"
 	apb "source.monogon.dev/metropolis/proto/api"
 )
 
+type verifyPeerCertificate func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error
+
+func verifyClusterCertificate(ca *x509.Certificate) verifyPeerCertificate {
+	return func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error {
+		if len(rawCerts) != 1 {
+			return fmt.Errorf("server presented %d certificates, wanted exactly one", len(rawCerts))
+		}
+		serverCert, err := x509.ParseCertificate(rawCerts[0])
+		if err != nil {
+			return fmt.Errorf("server presented unparseable certificate: %w", err)
+		}
+		if _, err := identity.VerifyNodeInCluster(serverCert, ca); err != nil {
+			return fmt.Errorf("node certificate verification failed: %w", err)
+		}
+
+		return nil
+	}
+}
+
 // NewEphemeralClient dials a cluster's services using just a self-signed
 // certificate and can be used to then escrow real cluster credentials for the
 // owner.
@@ -26,18 +48,16 @@
 // 'real' client certificate (yet). Current users include users of AAA.Escrow
 // and new nodes Registering into the Cluster.
 //
-// If ca is given, the other side of the connection is verified to be served by
-// a node presenting a certificate signed by that CA. Otherwise, no
-// verification of the other side is performed (however, any attacker
-// impersonating the cluster cannot use the escrowed credentials as the private
-// key is never passed to the server).
-func NewEphemeralClient(remote string, private ed25519.PrivateKey, ca *x509.Certificate) (*grpc.ClientConn, error) {
+// If 'ca' is given, the remote side will be cryptographically verified to be a
+// node that's part of the cluster represented by the ca. Otherwise, no
+// verification is performed and this function is unsafe.
+func NewEphemeralClient(remote string, private ed25519.PrivateKey, ca *x509.Certificate, opts ...grpc.DialOption) (*grpc.ClientConn, error) {
 	template := x509.Certificate{
 		SerialNumber: big.NewInt(1),
 		NotBefore:    time.Now(),
 		NotAfter:     pki.UnknownNotAfter,
 
-		KeyUsage:              x509.KeyUsageKeyEncipherment | x509.KeyUsageDigitalSignature,
+		KeyUsage:              x509.KeyUsageKeyEncipherment | x509.KeyUsageDigitalSignature | x509.KeyUsageCertSign,
 		ExtKeyUsage:           []x509.ExtKeyUsage{x509.ExtKeyUsageClientAuth},
 		BasicConstraintsValid: true,
 	}
@@ -49,53 +69,13 @@
 		Certificate: [][]byte{certificateBytes},
 		PrivateKey:  private,
 	}
-	creds := credentials.NewTLS(&tls.Config{
-		Certificates: []tls.Certificate{
-			certificate,
-		},
-		InsecureSkipVerify: true,
-		VerifyPeerCertificate: func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error {
-			if len(rawCerts) < 1 {
-				return fmt.Errorf("server presented no certificate")
-			}
-			certs := make([]*x509.Certificate, len(rawCerts))
-			for i, rawCert := range rawCerts {
-				cert, err := x509.ParseCertificate(rawCert)
-				if err != nil {
-					return fmt.Errorf("could not parse server certificate %d: %v", i, err)
-				}
-				certs[i] = cert
-			}
+	return NewAuthenticatedClient(remote, certificate, ca, opts...)
+}
 
-			if ca != nil {
-				// CA given, perform full chain verification.
-				roots := x509.NewCertPool()
-				roots.AddCert(ca)
-				opts := x509.VerifyOptions{
-					Roots:         roots,
-					Intermediates: x509.NewCertPool(),
-				}
-				for _, cert := range certs[1:] {
-					opts.Intermediates.AddCert(cert)
-				}
-				_, err := certs[0].Verify(opts)
-				if err != nil {
-					return err
-				}
-			}
-
-			// Regardless of CA given, ensure that the leaf certificate has the
-			// right ExtKeyUsage.
-			for _, ku := range certs[0].ExtKeyUsage {
-				if ku == x509.ExtKeyUsageServerAuth {
-					return nil
-				}
-			}
-			return fmt.Errorf("server presented a certificate without server auth ext key usage")
-		},
-	})
-
-	return grpc.Dial(remote, grpc.WithTransportCredentials(creds))
+func NewEphemeralClientTest(listener *bufconn.Listener, private ed25519.PrivateKey, ca *x509.Certificate) (*grpc.ClientConn, error) {
+	return NewEphemeralClient("local", private, ca, grpc.WithContextDialer(func(_ context.Context, _ string) (net.Conn, error) {
+		return listener.Dial()
+	}))
 }
 
 // RetrieveOwnerCertificates uses AAA.Escrow to retrieve a cluster manager
@@ -129,3 +109,38 @@
 		PrivateKey:  private,
 	}, nil
 }
+
+// NewAuthenticatedClient dials a cluster's services using the given TLS
+// credentials (either user or node credentials).
+//
+// If 'ca' is given, the remote side will be cryptographically verified to be a
+// node that's part of the cluster represented by the ca. Otherwise, no
+// verification is performed and this function is unsafe.
+func NewAuthenticatedClient(remote string, cert tls.Certificate, ca *x509.Certificate, opts ...grpc.DialOption) (*grpc.ClientConn, error) {
+	config := &tls.Config{
+		Certificates:       []tls.Certificate{cert},
+		InsecureSkipVerify: true,
+	}
+	if ca != nil {
+		config.VerifyPeerCertificate = verifyClusterCertificate(ca)
+	}
+	opts = append(opts, grpc.WithTransportCredentials(credentials.NewTLS(config)))
+	return grpc.Dial(remote, opts...)
+}
+
+func NewNodeClient(remote string, opts ...grpc.DialOption) (*grpc.ClientConn, error) {
+	opts = append(opts, grpc.WithInsecure())
+	return grpc.Dial(remote, opts...)
+}
+
+func NewAuthenticatedClientTest(listener *bufconn.Listener, cert tls.Certificate, ca *x509.Certificate) (*grpc.ClientConn, error) {
+	return NewAuthenticatedClient("local", cert, ca, grpc.WithContextDialer(func(_ context.Context, _ string) (net.Conn, error) {
+		return listener.Dial()
+	}))
+}
+
+func NewNodeClientTest(listener *bufconn.Listener) (*grpc.ClientConn, error) {
+	return NewNodeClient("local", grpc.WithContextDialer(func(_ context.Context, _ string) (net.Conn, error) {
+		return listener.Dial()
+	}))
+}
diff --git a/metropolis/node/core/rpc/methodinfo.go b/metropolis/node/core/rpc/methodinfo.go
new file mode 100644
index 0000000..0a597fa
--- /dev/null
+++ b/metropolis/node/core/rpc/methodinfo.go
@@ -0,0 +1,86 @@
+package rpc
+
+import (
+	"fmt"
+	"regexp"
+
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+
+	epb "source.monogon.dev/metropolis/proto/ext"
+)
+
+// methodInfo is the parsed information for a given RPC method, as configured by
+// the metropolis.common.ext.authorization extension.
+type methodInfo struct {
+	// unauthenticated is true if the method is defined as 'unauthenticated', ie.
+	// that all requests should be passed to the gRPC handler without any
+	// authentication or authorization performed.
+	unauthenticated bool
+	// need is a map of permissions that the caller needs to have in order to be
+	// allowed to call this method. If not empty, unauthenticated cannot be set to
+	// true.
+	need map[epb.Permission]bool
+}
+
+var (
+	// reMethodName matches a /some.service/Method string from
+	// {Stream,Unary}ServerInfo.FullMethod.
+	reMethodName = regexp.MustCompile(`^/([^/]+)/([^/.]+)$`)
+)
+
+// getMethodInfo returns the methodInfo for a given method name, as retrieved
+// from grpc.{Stream,Unary}ServerInfo.FullMethod, or nil if the method could not
+// be found.
+//
+// SECURITY: If the given method does not have any
+// metropolis.common.ext.authorization annotations, a methodInfo which requires
+// authorization but no permissions is returned, defaulting to a mildly secure
+// default of a method that can be called by any authenticated user.
+func getMethodInfo(methodName string) (*methodInfo, error) {
+	m := reMethodName.FindStringSubmatch(methodName)
+	if len(m) != 3 {
+		return nil, status.Errorf(codes.InvalidArgument, "invalid method name %q", methodName)
+	}
+	// Convert /foo.bar/Method to foo.bar.Method, which is used by the protoregistry.
+	methodName = fmt.Sprintf("%s.%s", m[1], m[2])
+	desc, err := protoregistry.GlobalFiles.FindDescriptorByName(protoreflect.FullName(methodName))
+	if err != nil {
+		return nil, status.Errorf(codes.InvalidArgument, "could not retrieve descriptor for method: %v", err)
+	}
+	method, ok := desc.(protoreflect.MethodDescriptor)
+	if !ok {
+		return nil, status.Error(codes.InvalidArgument, "querying method name did not yield a MethodDescriptor")
+	}
+
+	// Get authorization extension, defaults to no options set.
+	if !proto.HasExtension(method.Options(), epb.E_Authorization) {
+		return nil, status.Errorf(codes.Internal, "method does not provide Authorization extension, failing safe")
+	}
+	authz, ok := proto.GetExtension(method.Options(), epb.E_Authorization).(*epb.Authorization)
+	if !ok {
+		return nil, status.Errorf(codes.Internal, "method contains Authorization extension with wrong type, failing safe")
+	}
+	if authz == nil {
+		return nil, status.Errorf(codes.Internal, "method contains nil Authorization extension, failing safe")
+	}
+
+	// If unauthenticated connections are allowed, return immediately.
+	if authz.AllowUnauthenticated && len(authz.Need) == 0 {
+		return &methodInfo{
+			unauthenticated: true,
+		}, nil
+	}
+
+	// Otherwise, return needed permissions.
+	res := &methodInfo{
+		need: make(map[epb.Permission]bool),
+	}
+	for _, n := range authz.Need {
+		res.need[n] = true
+	}
+	return res, nil
+}
diff --git a/metropolis/node/core/rpc/peerinfo.go b/metropolis/node/core/rpc/peerinfo.go
new file mode 100644
index 0000000..a4d685c
--- /dev/null
+++ b/metropolis/node/core/rpc/peerinfo.go
@@ -0,0 +1,129 @@
+package rpc
+
+import (
+	"context"
+	"fmt"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+
+	epb "source.monogon.dev/metropolis/proto/ext"
+)
+
+type Permissions map[epb.Permission]bool
+
+// PeerInfo represents the Metropolis-level information about the remote side
+// of a gRPC RPC, ie. about the calling client in server handlers and about the
+// handling server in client code.
+//
+// Exactly one of {Node, User, Unauthenticated} will be non-nil.
+type PeerInfo struct {
+	// Node is the information about a peer Node, and identifies that the other side
+	// of the connection is either a Node servicng gRPC requests for a cluster, or a
+	// Node connecting to a gRPC service.
+	Node *PeerInfoNode
+	// User is the information about a peer User, and identifies that the other side
+	// of the connection is a Metropolis user or manager (eg. owner). This will only
+	// be set in service handlers, as users cannot serve gRPC connections.
+	User *PeerInfoUser
+	// Unauthenticated is set for incoming gRPC connections which that have the
+	// Unauthenticated authorization extension set to true, and mark that the other
+	// side of the connection has not been verified at all.
+	Unauthenticated *PeerInfoUnauthenticated
+}
+
+// PeerInfoNode contains information about a Node on the other side of a gRPC
+// connection.
+type PeerInfoNode struct {
+	// PublicKey is the ED25519 public key bytes of the node.
+	PublicKey []byte
+
+	// Permissions are the set of permissions this node has.
+	Permissions Permissions
+}
+
+// PeerInfoUser contains information about a user on the other side of a gRPC
+// connection.
+type PeerInfoUser struct {
+	// Identity is an opaque identifier for the user. MVP: Currently this is always
+	// "manager".
+	Identity string
+}
+
+type PeerInfoUnauthenticated struct {
+	// SelfSignedPublicKey is the ED25519 public key bytes of the other side of the
+	// connection, if that side presented a self-signed certificate to prove control
+	// of a private key corresponding to this public key. If it did not present a
+	// self-signed certificate that can be parsed for such a key, this will be nil.
+	//
+	// This can be used by code with expects Unauthenticated RPCs but wants to
+	// authenticate the connection based on ownership of some keypair, for example
+	// in the AAA.Escrow method.
+	SelfSignedPublicKey []byte
+}
+
+// GetPeerInfo returns the PeerInfo of the peer of a gRPC connection, or nil if
+// this connection does not carry any PeerInfo.
+func GetPeerInfo(ctx context.Context) *PeerInfo {
+	if pi, ok := ctx.Value(peerInfoKey).(*PeerInfo); ok {
+		return pi
+	}
+	return nil
+}
+
+func (p *PeerInfo) CheckPermissions(need Permissions) error {
+	if p.Unauthenticated != nil {
+		// This generally shouldn't happen, as unauthenticated users shouldn't be
+		// allowed to reach this part of the code - methods with Need != nil will not be
+		// processed as unauthenticated for security, and will instead act as
+		// authenticated methods and reject unauthenticated connections.
+		for _, v := range need {
+			if v {
+				return status.Error(codes.Unauthenticated, "unauthenticated connection")
+			}
+		}
+		return nil
+	} else if p.User != nil {
+		// MVP: all permissions are granted to all users.
+		// TODO(q3k): check authz.Need once we have a user/identity system implemented.
+		return nil
+	} else if p.Node != nil {
+		for n, v := range need {
+			if v && !p.Node.Permissions[n] {
+				return status.Errorf(codes.PermissionDenied, "node missing %s permission", n.String())
+			}
+		}
+		return nil
+	}
+
+	return fmt.Errorf("invalid PeerInfo: neither Unauthenticated, User nor Node is set")
+}
+
+type peerInfoKeyType string
+
+// peerInfoKey is the context key for storing PeerInfo.
+const peerInfoKey = peerInfoKeyType("peerInfo")
+
+// apply returns the given context with itself stored under a unique key, that
+// can be later retrieved via GetPeerInfo.
+func (p *PeerInfo) apply(ctx context.Context) context.Context {
+	return context.WithValue(ctx, peerInfoKey, p)
+}
+
+// peerInfoServerStream is a grpc.ServerStream wrapper which contains some
+// PeerInfo, and returns it as part of the Context() of the ServerStream.
+type peerInfoServerStream struct {
+	grpc.ServerStream
+	pi *PeerInfo
+}
+
+func (p *peerInfoServerStream) Context() context.Context {
+	return p.pi.apply(p.ServerStream.Context())
+}
+
+// serverStream wraps a grpc.ServerStream with a structure that attaches this
+// PeerInfo in all contexts returned by Context().
+func (p *PeerInfo) serverStream(ss grpc.ServerStream) grpc.ServerStream {
+	return &peerInfoServerStream{ss, p}
+}
diff --git a/metropolis/node/core/rpc/server.go b/metropolis/node/core/rpc/server.go
index 3d6917d..70a0a74 100644
--- a/metropolis/node/core/rpc/server.go
+++ b/metropolis/node/core/rpc/server.go
@@ -1,165 +1,37 @@
 package rpc
 
 import (
-	"context"
-	"crypto/tls"
-	"crypto/x509"
-	"strings"
-
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/credentials"
-	"google.golang.org/grpc/peer"
-	"google.golang.org/grpc/status"
-	"google.golang.org/protobuf/proto"
-	"google.golang.org/protobuf/reflect/protoreflect"
-	"google.golang.org/protobuf/reflect/protoregistry"
-
 	cpb "source.monogon.dev/metropolis/node/core/curator/proto/api"
 	apb "source.monogon.dev/metropolis/proto/api"
 	epb "source.monogon.dev/metropolis/proto/ext"
 )
 
-// ClusterServices is the interface containing all gRPC services that a
-// Metropolis Cluster implements on its public interface. With the current
-// implementaiton of Metropolis, this is all implemented by the Curator.
-type ClusterServices interface {
+var (
+	// nodePermissions are the set of metropolis.common.ext.authorization
+	// permissions automatically given to nodes when connecting to curator gRPC
+	// services, either locally or remotely.
+	nodePermissions = Permissions{
+		epb.Permission_PERMISSION_READ_CLUSTER_STATUS: true,
+	}
+)
+
+// ClusterExternalServices is the interface containing all gRPC services that a
+// Metropolis Cluster implements on its external interface. With the current
+// implementation of Metropolis, this is all implemented by the Curator.
+type ClusterExternalServices interface {
 	cpb.CuratorServer
 	apb.AAAServer
 	apb.ManagementServer
 }
 
-// ServerSecurity are the security options of a RPC server that will run
-// ClusterServices on a Metropolis node. It contains all the data for the
-// server implementation to authenticate itself to the clients and authenticate
-// and authorize clients connecting to it.
-type ServerSecurity struct {
-	// NodeCredentials is the TLS certificate/key of the node that the server
-	// implementation is running on. It should be signed by
-	// ClusterCACertificate.
-	NodeCredentials tls.Certificate
-	// ClusterCACertificate is the cluster's CA certificate. It will be used to
-	// authenticate the client certificates of incoming gRPC connections.
-	ClusterCACertificate *x509.Certificate
+// ClusterInternalServices is the interface containing all gRPC services that a
+// Metropolis Cluster implements on its internal interface. Currently this is
+// just the Curator service.
+type ClusterInternalServices interface {
+	cpb.CuratorServer
 }
 
-// SetupPublicGRPC returns a grpc.Server ready to listen and serve all public
-// gRPC APIs that the cluster server implementation should run, with all calls
-// being authenticated and authorized based on the data in ServerSecurity. The
-// argument 'impls' is the object implementing the gRPC APIs.
-//
-// This effectively configures gRPC interceptors that verify
-// metropolis.proto.ext.authorizaton options and authenticate/authorize
-// incoming connections. It also runs the gRPC server with the correct TLS
-// settings for authenticating itself to callers.
-func (l *ServerSecurity) SetupPublicGRPC(impls ClusterServices) *grpc.Server {
-	publicCreds := credentials.NewTLS(&tls.Config{
-		Certificates: []tls.Certificate{l.NodeCredentials},
-		ClientAuth:   tls.RequestClientCert,
-	})
-
-	s := grpc.NewServer(
-		grpc.Creds(publicCreds),
-		grpc.UnaryInterceptor(l.unaryInterceptor),
-		grpc.StreamInterceptor(l.streamInterceptor),
-	)
-	cpb.RegisterCuratorServer(s, impls)
-	apb.RegisterAAAServer(s, impls)
-	apb.RegisterManagementServer(s, impls)
-	return s
-}
-
-// authorize performs an authorization check for the given gRPC context
-// (containing peer information) and given RPC method name (as obtained from
-// FullMethodName in {Unary,Stream}ServerInfo). The actual authorization
-// requirements per method are retrieved from the Authorization protobuf
-// option applied to the RPC method.
-//
-// If the peer (as retrieved from the context) is authorized to run this method,
-// no error is returned. Otherwise, a gRPC status is returned outlining the
-// reason the authorization being rejected.
-func (l *ServerSecurity) authorize(ctx context.Context, methodName string) error {
-	if !strings.HasPrefix(methodName, "/") {
-		return status.Errorf(codes.InvalidArgument, "invalid method name %q", methodName)
-	}
-	methodName = strings.ReplaceAll(methodName[1:], "/", ".")
-	desc, err := protoregistry.GlobalFiles.FindDescriptorByName(protoreflect.FullName(methodName))
-	if err != nil {
-		return status.Errorf(codes.InvalidArgument, "could not retrieve descriptor for method: %v", err)
-	}
-	method, ok := desc.(protoreflect.MethodDescriptor)
-	if !ok {
-		return status.Error(codes.InvalidArgument, "querying method name did not yield a MethodDescriptor")
-	}
-
-	// Get authorization extension, defaults to no options set.
-	authz, ok := proto.GetExtension(method.Options(), epb.E_Authorization).(*epb.Authorization)
-	if !ok || authz == nil {
-		authz = &epb.Authorization{}
-	}
-
-	// If unauthenticated connections are allowed, let them through immediately.
-	if authz.AllowUnauthenticated && len(authz.Need) == 0 {
-		return nil
-	}
-
-	// Otherwise, we check that the other side of the connection is authenticated
-	// using a valid cluster CA client certificate.
-	p, ok := peer.FromContext(ctx)
-	if !ok {
-		return status.Error(codes.Unavailable, "could not retrive peer info")
-	}
-	tlsInfo, ok := p.AuthInfo.(credentials.TLSInfo)
-	if !ok {
-		return status.Error(codes.Unauthenticated, "connection not secure")
-	}
-	count := len(tlsInfo.State.PeerCertificates)
-	if count == 0 {
-		return status.Errorf(codes.Unauthenticated, "no client certificate presented")
-	}
-	if count > 1 {
-		return status.Errorf(codes.Unauthenticated, "exactly one client certificate must be sent (got %d)", count)
-	}
-	pCert := tlsInfo.State.PeerCertificates[0]
-
-	// Ensure that the certificate is signed by the cluster CA.
-	if err := pCert.CheckSignatureFrom(l.ClusterCACertificate); err != nil {
-		return status.Errorf(codes.Unauthenticated, "invalid client certificate: %v", err)
-	}
-	// Ensure that the certificate is a client certificate.
-	// TODO(q3k): synchronize this with //metropolis/pkg/pki Client()/Server()/...
-	isClient := false
-	for _, ku := range pCert.ExtKeyUsage {
-		if ku == x509.ExtKeyUsageClientAuth {
-			isClient = true
-			break
-		}
-	}
-	if !isClient {
-		return status.Error(codes.PermissionDenied, "presented certificate is not a client certificate")
-	}
-
-	// MVP: all permissions are granted to all users.
-	// TODO(q3k): check authz.Need once we have a user/identity system implemented.
-	return nil
-}
-
-// streamInterceptor is a gRPC server stream interceptor that performs
-// authentication and authorization of incoming RPCs based on the Authorization
-// option set on each method.
-func (l *ServerSecurity) streamInterceptor(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
-	if err := l.authorize(ss.Context(), info.FullMethod); err != nil {
-		return err
-	}
-	return handler(srv, ss)
-}
-
-// unaryInterceptor is a gRPC server unary interceptor that performs
-// authentication and authorization of incoming RPCs based on the Authorization
-// option set on each method.
-func (l *ServerSecurity) unaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
-	if err := l.authorize(ctx, info.FullMethod); err != nil {
-		return nil, err
-	}
-	return handler(ctx, req)
+type ClusterServices interface {
+	ClusterExternalServices
+	ClusterInternalServices
 }
diff --git a/metropolis/node/core/rpc/server_authentication.go b/metropolis/node/core/rpc/server_authentication.go
new file mode 100644
index 0000000..6ae2618
--- /dev/null
+++ b/metropolis/node/core/rpc/server_authentication.go
@@ -0,0 +1,268 @@
+package rpc
+
+import (
+	"context"
+	"crypto/ed25519"
+	"crypto/tls"
+	"crypto/x509"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/credentials"
+	"google.golang.org/grpc/peer"
+	"google.golang.org/grpc/status"
+
+	cpb "source.monogon.dev/metropolis/node/core/curator/proto/api"
+	"source.monogon.dev/metropolis/node/core/identity"
+	apb "source.monogon.dev/metropolis/proto/api"
+)
+
+// authenticationStrategy is implemented by {Local,External}ServerSecurity to
+// share logic between the two implementations.
+type authenticationStrategy interface {
+	// getPeerInfo will be called by the stream and unary gRPC server interceptors
+	// to authenticate incoming gRPC calls. It's given the gRPC context of the call
+	// (therefore allowing access to information about the underlying gRPC
+	// transport), and should return a PeerInfo structure describing the
+	// authenticated other end of the connection, or a gRPC status if the other
+	// side could not be successfully authenticated.
+	//
+	// The returned PeerInfo will then be used to perform authorization checks based
+	// on the configured authentication of a given gRPC method, as described by the
+	// metropolis.proto.ext.authorization extension. The same PeerInfo will then be
+	// available to the gRPC handler for this method by retrieving it from the
+	// context (via GetPeerInfo).
+	getPeerInfo(ctx context.Context) (*PeerInfo, error)
+
+	// getPeerInfoUnauthenticated is an equivalent to getPeerInfo, but called by the
+	// interceptors when a method is marked as 'unauthenticated'. The implementation
+	// should return a PeerInfo containing Unauthenticated, potentially populating
+	// it with UnauthenticatedPublicKey if such a public key could be retrieved.
+	getPeerInfoUnauthenticated(ctx context.Context) (*PeerInfo, error)
+}
+
+// stream implements the gRPC StreamInterceptor interface for use with
+// grpc.NewServer, based on an authenticationStrategy.
+func streamInterceptor(a authenticationStrategy) grpc.StreamServerInterceptor {
+	return func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		pi, err := check(ss.Context(), a, info.FullMethod)
+		if err != nil {
+			return err
+		}
+		return handler(srv, pi.serverStream(ss))
+	}
+}
+
+// unaryInterceptor implements the gRPC UnaryInterceptor interface for use with
+// grpc.NewServer, based on an authenticationStrategy.
+func unaryInterceptor(a authenticationStrategy) grpc.UnaryServerInterceptor {
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
+		pi, err := check(ctx, a, info.FullMethod)
+		if err != nil {
+			return nil, err
+		}
+		return handler(pi.apply(ctx), req)
+	}
+}
+
+// check is called by the unary and server interceptors to perform
+// authentication and authorization checks for a given RPC, calling the
+// serverInterceptors' authenticate function if needed.
+func check(ctx context.Context, a authenticationStrategy, methodName string) (*PeerInfo, error) {
+	mi, err := getMethodInfo(methodName)
+	if err != nil {
+		return nil, err
+	}
+
+	if mi.unauthenticated {
+		return a.getPeerInfoUnauthenticated(ctx)
+	}
+
+	pi, err := a.getPeerInfo(ctx)
+	if err != nil {
+		return nil, err
+	}
+	if err := pi.CheckPermissions(mi.need); err != nil {
+		return nil, err
+	}
+	return pi, nil
+}
+
+// ServerSecurity are the security options of a RPC server that will run
+// ClusterServices on a Metropolis node. It contains all the data for the
+// server implementation to authenticate itself to the clients and authenticate
+// and authorize clients connecting to it.
+//
+// It implements authenticationStrategy.
+type ExternalServerSecurity struct {
+	// NodeCredentials which will be used to run the gRPC server, and whose CA
+	// certificate will be used to authenticate incoming requests.
+	NodeCredentials *identity.NodeCredentials
+
+	// nodePermissions is used by tests to inject the permissions available to a
+	// node. When not set, it defaults to the global nodePermissions map.
+	nodePermissions Permissions
+}
+
+// SetupExternalGRPC returns a grpc.Server ready to listen and serve all external
+// gRPC APIs that the cluster server implementation should run, with all calls
+// being authenticated and authorized based on the data in ServerSecurity. The
+// argument 'impls' is the object implementing the gRPC APIs.
+//
+// This effectively configures gRPC interceptors that verify
+// metropolis.proto.ext.authorization options and authenticate/authorize
+// incoming connections. It also runs the gRPC server with the correct TLS
+// settings for authenticating itself to callers.
+func (l *ExternalServerSecurity) SetupExternalGRPC(impls ClusterExternalServices) *grpc.Server {
+	externalCreds := credentials.NewTLS(&tls.Config{
+		Certificates: []tls.Certificate{l.NodeCredentials.TLSCredentials()},
+		ClientAuth:   tls.RequestClientCert,
+	})
+
+	s := grpc.NewServer(
+		grpc.Creds(externalCreds),
+		grpc.UnaryInterceptor(unaryInterceptor(l)),
+		grpc.StreamInterceptor(streamInterceptor(l)),
+	)
+	cpb.RegisterCuratorServer(s, impls)
+	apb.RegisterAAAServer(s, impls)
+	apb.RegisterManagementServer(s, impls)
+	return s
+}
+
+func (l *ExternalServerSecurity) getPeerInfo(ctx context.Context) (*PeerInfo, error) {
+	cert, err := getPeerCertificate(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	// Ensure that the certificate is signed by the cluster CA.
+	if err := cert.CheckSignatureFrom(l.NodeCredentials.ClusterCA()); err != nil {
+		return nil, status.Errorf(codes.Unauthenticated, "certificate not signed by cluster CA: %v", err)
+	}
+
+	nodepk, errNode := identity.VerifyNodeInCluster(cert, l.NodeCredentials.ClusterCA())
+	if errNode == nil {
+		// This is a Metropolis node.
+		np := l.nodePermissions
+		if np == nil {
+			np = nodePermissions
+		}
+		return &PeerInfo{
+			Node: &PeerInfoNode{
+				PublicKey:   nodepk,
+				Permissions: np,
+			},
+		}, nil
+	}
+
+	userid, errUser := identity.VerifyUserInCluster(cert, l.NodeCredentials.ClusterCA())
+	if errUser == nil {
+		// This is a Metropolis user/manager.
+		return &PeerInfo{
+			User: &PeerInfoUser{
+				Identity: userid,
+			},
+		}, nil
+	}
+
+	// Could not parse as either node or user certificate.
+	return nil, status.Errorf(codes.Unauthenticated, "presented certificate is neither user certificate (%v) nor node certificate (%v)", errUser, errNode)
+}
+
+func (l *ExternalServerSecurity) getPeerInfoUnauthenticated(ctx context.Context) (*PeerInfo, error) {
+	res := PeerInfo{
+		Unauthenticated: &PeerInfoUnauthenticated{},
+	}
+
+	// If peer presented a valid self-signed certificate, attach that to the
+	// Unauthenticated struct.
+	cert, err := getPeerCertificate(ctx)
+	if err == nil {
+		if err := cert.CheckSignature(cert.SignatureAlgorithm, cert.RawTBSCertificate, cert.Signature); err != nil {
+			return nil, status.Errorf(codes.Unauthenticated, "presented certificate must be self-signed (check error: %v)", err)
+		}
+		res.Unauthenticated.SelfSignedPublicKey = cert.PublicKey.(ed25519.PublicKey)
+	}
+
+	return &res, nil
+}
+
+// getPeerCertificate returns the x509 certificate associated with the given
+// gRPC connection's context and ensures that it is a certificate for an Ed25519
+// keypair. The certificate is _not_ checked against the cluster CA.
+//
+// A gRPC status is returned if the certificate is invalid / unauthenticated for
+// any reason.
+func getPeerCertificate(ctx context.Context) (*x509.Certificate, error) {
+	p, ok := peer.FromContext(ctx)
+	if !ok {
+		return nil, status.Error(codes.Unavailable, "could not retrive peer info")
+	}
+	tlsInfo, ok := p.AuthInfo.(credentials.TLSInfo)
+	if !ok {
+		return nil, status.Error(codes.Unauthenticated, "connection not secure")
+	}
+	count := len(tlsInfo.State.PeerCertificates)
+	if count == 0 {
+		return nil, status.Errorf(codes.Unauthenticated, "no client certificate presented")
+	}
+	if count > 1 {
+		return nil, status.Errorf(codes.Unauthenticated, "exactly one client certificate must be sent (got %d)", count)
+	}
+	cert := tlsInfo.State.PeerCertificates[0]
+	if _, ok := cert.PublicKey.(ed25519.PublicKey); !ok {
+		return nil, status.Errorf(codes.Unauthenticated, "certificate must be issued for an ED25519 keypair")
+	}
+
+	return cert, nil
+}
+
+// LocalServerSecurity are the security options of an RPC server that will run
+// the Curator service over a local domain socket. When set up using
+// LocalServerSecurity, all incoming RPCs will be authenticated as coming from
+// the node that this service is running on.
+//
+// It implements authenticationStrategy.
+type LocalServerSecurity struct {
+	// Node for which the gRPC server will authenticate all incoming requests as
+	// originating from.
+	Node *identity.Node
+
+	// nodePermissions is used by tests to inject the permissions available to a
+	// node. When not set, it defaults to the global nodePermissions map.
+	nodePermissions Permissions
+}
+
+// SetupLocalGRPC returns a grpc.Server ready to listen on a local domain socket
+// and serve the Curator service. All incoming RPCs will be authenticated as
+// originating from the node for which LocalServerSecurity has been configured.
+func (l *LocalServerSecurity) SetupLocalGRPC(impls ClusterInternalServices) *grpc.Server {
+	s := grpc.NewServer(
+		grpc.UnaryInterceptor(unaryInterceptor(l)),
+		grpc.StreamInterceptor(streamInterceptor(l)),
+	)
+	cpb.RegisterCuratorServer(s, impls)
+	return s
+}
+
+func (l *LocalServerSecurity) getPeerInfo(_ context.Context) (*PeerInfo, error) {
+	// Local connections are always node connections.
+	np := l.nodePermissions
+	if np == nil {
+		np = nodePermissions
+	}
+	return &PeerInfo{
+		Node: &PeerInfoNode{
+			PublicKey:   l.Node.PublicKey(),
+			Permissions: np,
+		},
+	}, nil
+}
+
+func (l *LocalServerSecurity) getPeerInfoUnauthenticated(_ context.Context) (*PeerInfo, error) {
+	// This shouldn't happen - why would we call unauthenticated methods locally?
+	// This can be implemented, but doesn't really make sense. For now, assume this
+	// is a programming error. This can be changed if needed.
+	return nil, status.Errorf(codes.Unauthenticated, "unauthenticated methods not supported over local connections")
+}
diff --git a/metropolis/node/core/rpc/server_authentication_test.go b/metropolis/node/core/rpc/server_authentication_test.go
new file mode 100644
index 0000000..d383150
--- /dev/null
+++ b/metropolis/node/core/rpc/server_authentication_test.go
@@ -0,0 +1,171 @@
+package rpc
+
+import (
+	"context"
+	"crypto/ed25519"
+	"crypto/rand"
+	"testing"
+
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"google.golang.org/grpc/test/bufconn"
+
+	cpb "source.monogon.dev/metropolis/node/core/curator/proto/api"
+	apb "source.monogon.dev/metropolis/proto/api"
+	epb "source.monogon.dev/metropolis/proto/ext"
+)
+
+// testImplementations implements ClusterServices by returning 'unimplementd'
+// for every RPC call.
+type testImplementation struct {
+	cpb.UnimplementedCuratorServer
+	apb.UnimplementedAAAServer
+	apb.UnimplementedManagementServer
+}
+
+// TestExternalServerSecurity ensures that the unary interceptor of the
+// ServerSecurity structure works, and authenticates/authorizes incoming RPCs as
+// expected.
+func TestExternalServerSecurity(t *testing.T) {
+	ctx, ctxC := context.WithCancel(context.Background())
+	defer ctxC()
+
+	eph := NewEphemeralClusterCredentials(t, 1)
+	permissions := make(Permissions)
+	for k, v := range nodePermissions {
+		permissions[k] = v
+	}
+	ss := ExternalServerSecurity{
+		NodeCredentials: eph.Nodes[0],
+		nodePermissions: permissions,
+	}
+
+	impl := &testImplementation{}
+	srv := ss.SetupExternalGRPC(impl)
+	lis := bufconn.Listen(1024 * 1024)
+	go func() {
+		if err := srv.Serve(lis); err != nil {
+			t.Fatalf("GRPC serve failed: %v", err)
+		}
+	}()
+	defer lis.Close()
+	defer srv.Stop()
+
+	// Authenticate as manager externally, ensure that GetRegisterTicket runs.
+	cl, err := NewAuthenticatedClientTest(lis, eph.Manager, eph.CA)
+	if err != nil {
+		t.Fatalf("NewAuthenticatedClient: %v", err)
+	}
+	defer cl.Close()
+	mgmt := apb.NewManagementClient(cl)
+	_, err = mgmt.GetRegisterTicket(ctx, &apb.GetRegisterTicketRequest{})
+	if s, ok := status.FromError(err); !ok || s.Code() != codes.Unimplemented {
+		t.Errorf("GetRegisterTicket returned %v, wanted codes.Unimplemented", err)
+	}
+
+	// Authenticate as node externally, ensure that GetRegisterTicket is refused
+	// (this is because nodes miss the GET_REGISTER_TICKET permissions).
+	cl, err = NewAuthenticatedClientTest(lis, eph.Nodes[0].TLSCredentials(), eph.CA)
+	if err != nil {
+		t.Fatalf("NewAuthenticatedClient: %v", err)
+	}
+	defer cl.Close()
+	mgmt = apb.NewManagementClient(cl)
+	_, err = mgmt.GetRegisterTicket(ctx, &apb.GetRegisterTicketRequest{})
+	if s, ok := status.FromError(err); !ok || s.Code() != codes.PermissionDenied {
+		t.Errorf("GetRegisterTicket (by external node) returned %v, wanted codes.PermissionDenied", err)
+	}
+
+	// Give the node GET_REGISTER_TICKET permissions and try again. This should pass.
+	permissions[epb.Permission_PERMISSION_GET_REGISTER_TICKET] = true
+	_, err = mgmt.GetRegisterTicket(ctx, &apb.GetRegisterTicketRequest{})
+	if s, ok := status.FromError(err); !ok || s.Code() != codes.Unimplemented {
+		t.Errorf("GetRegisterTicket returned %v, wanted codes.Unimplemented", err)
+	}
+	permissions[epb.Permission_PERMISSION_GET_REGISTER_TICKET] = false
+
+	// Authenticate with an ephemeral/self-signed certificate, ensure that
+	// GetRegisterTicket is refused (this is because GetRegisterTicket requires an
+	// authenticated connection).
+	_, sk, err := ed25519.GenerateKey(rand.Reader)
+	if err != nil {
+		t.Fatalf("GenerateKey: %v", err)
+	}
+	cl, err = NewEphemeralClientTest(lis, sk, eph.CA)
+	if err != nil {
+		t.Fatalf("NewEphemeralClient: %v", err)
+	}
+	defer cl.Close()
+	mgmt = apb.NewManagementClient(cl)
+	_, err = mgmt.GetRegisterTicket(ctx, &apb.GetRegisterTicketRequest{})
+	if s, ok := status.FromError(err); !ok || s.Code() != codes.Unauthenticated {
+		t.Errorf("GetRegisterTicket (by ephemeral cert) returned %v, wanted codes.Unauthenticated", err)
+	}
+}
+
+// TestLocalServerSecurity ensures that the unary interceptor of the
+// LocalServerSecurity structure works, and authenticates/authorizes incoming
+// RPCs as expected.
+func TestLocalServerSecurity(t *testing.T) {
+	ctx, ctxC := context.WithCancel(context.Background())
+	defer ctxC()
+
+	eph := NewEphemeralClusterCredentials(t, 1)
+
+	permissions := make(Permissions)
+	for k, v := range nodePermissions {
+		permissions[k] = v
+	}
+
+	ls := LocalServerSecurity{
+		Node:            &eph.Nodes[0].Node,
+		nodePermissions: permissions,
+	}
+
+	impl := &testImplementation{}
+	srv := ls.SetupLocalGRPC(impl)
+	lis := bufconn.Listen(1024 * 1024)
+	go func() {
+		if err := srv.Serve(lis); err != nil {
+			t.Fatalf("GRPC serve failed: %v", err)
+		}
+	}()
+	defer lis.Close()
+	defer srv.Stop()
+
+	// Nodes should have access to Curator.Watch.
+	cl, err := NewNodeClientTest(lis)
+	if err != nil {
+		t.Fatalf("NewAuthenticatedClient: %v", err)
+	}
+	defer cl.Close()
+
+	curator := cpb.NewCuratorClient(cl)
+	req := &cpb.WatchRequest{
+		Kind: &cpb.WatchRequest_NodeInCluster_{
+			NodeInCluster: &cpb.WatchRequest_NodeInCluster{
+				NodeId: eph.Nodes[0].ID(),
+			},
+		},
+	}
+	w, err := curator.Watch(ctx, req)
+	if err != nil {
+		t.Fatalf("Watch: %v", err)
+	}
+	_, err = w.Recv()
+	if s, ok := status.FromError(err); !ok || s.Code() != codes.Unimplemented {
+		t.Errorf("Watch (by local node) returned %v, wanted codes.Unimplemented", err)
+	}
+
+	// Take away the node's PERMISSION_READ_CLUSTER_STATUS permissions and try
+	// again. This should fail.
+	permissions[epb.Permission_PERMISSION_READ_CLUSTER_STATUS] = false
+	w, err = curator.Watch(ctx, req)
+	if err != nil {
+		t.Fatalf("Watch: %v", err)
+	}
+	_, err = w.Recv()
+	if s, ok := status.FromError(err); !ok || s.Code() != codes.PermissionDenied {
+		t.Errorf("Watch (by local node after removing permission) returned %v, wanted codes.PermissionDenied", err)
+	}
+}
diff --git a/metropolis/node/core/rpc/testhelpers.go b/metropolis/node/core/rpc/testhelpers.go
new file mode 100644
index 0000000..93e4b46
--- /dev/null
+++ b/metropolis/node/core/rpc/testhelpers.go
@@ -0,0 +1,100 @@
+package rpc
+
+import (
+	"context"
+	"crypto/ed25519"
+	"crypto/rand"
+	"crypto/tls"
+	"crypto/x509"
+	"testing"
+
+	"source.monogon.dev/metropolis/node/core/identity"
+	"source.monogon.dev/metropolis/pkg/pki"
+)
+
+// NewEphemeralClusterCredentials creates a set of TLS certificates for use in a
+// test Metropolis cluster. These are a CA certificate, a Manager certificate
+// and an arbitrary amount of Node certificates (per the nodes argument).
+//
+// All of these are ephemeral, ie. not stored anywhere - including the CA
+// certificate. This function is for use by tests which want to bring up a
+// minimum set of PKI credentials for a fake Metropolis cluster.
+func NewEphemeralClusterCredentials(t *testing.T, nodes int) *EphemeralClusterCredentials {
+	ctx := context.Background()
+	t.Helper()
+
+	ns := pki.Namespaced("unused")
+	caCert := pki.Certificate{
+		Namespace: &ns,
+		Issuer:    pki.SelfSigned,
+		Template:  identity.CACertificate("test cluster ca"),
+		Mode:      pki.CertificateEphemeral,
+	}
+	caBytes, err := caCert.Ensure(ctx, nil)
+	if err != nil {
+		t.Fatalf("Could not ensure CA certificate: %v", err)
+	}
+	ca, err := x509.ParseCertificate(caBytes)
+	if err != nil {
+		t.Fatalf("Could not parse new CA certificate: %v", err)
+	}
+
+	managerCert := pki.Certificate{
+		Namespace: &ns,
+		Issuer:    &caCert,
+		Template:  identity.UserCertificate("owner"),
+		Mode:      pki.CertificateEphemeral,
+	}
+	managerBytes, err := managerCert.Ensure(ctx, nil)
+	if err != nil {
+		t.Fatalf("Could not ensure manager certificate: %v", err)
+	}
+	res := &EphemeralClusterCredentials{
+		Nodes: make([]*identity.NodeCredentials, nodes),
+		Manager: tls.Certificate{
+			Certificate: [][]byte{managerBytes},
+			PrivateKey:  managerCert.PrivateKey,
+		},
+		CA: ca,
+	}
+
+	for i := 0; i < nodes; i++ {
+		npk, npr, err := ed25519.GenerateKey(rand.Reader)
+		if err != nil {
+			t.Fatalf("Could not generate node keypair: %v", err)
+		}
+		nodeCert := pki.Certificate{
+			Namespace: &ns,
+			Issuer:    &caCert,
+			Template:  identity.NodeCertificate(npk),
+			Mode:      pki.CertificateEphemeral,
+			PublicKey: npk,
+			Name:      "",
+		}
+		nodeBytes, err := nodeCert.Ensure(ctx, nil)
+		if err != nil {
+			t.Fatalf("Could not ensure node certificate: %v", err)
+		}
+		node, err := identity.NewNodeCredentials(npr, nodeBytes, caBytes)
+		if err != nil {
+			t.Fatalf("Could not build node credentials: %v", err)
+		}
+		res.Nodes[i] = node
+	}
+
+	return res
+}
+
+// EphemeralClusterCredentials are TLS/PKI credentials for use in a Metropolis
+// test cluster.
+type EphemeralClusterCredentials struct {
+	// Nodes are the node credentials for the cluster. Each contains a private
+	// key and x509 certificate authenticating the bearer as a Metropolis node.
+	Nodes []*identity.NodeCredentials
+	// Manager TLS certificate for the cluster. Contains a private key and x509
+	// certificate authenticating the bearer as a Metropolis manager.
+	Manager tls.Certificate
+	// CA is the x509 certificate of the CA certificate for the cluster. Manager and
+	// Node certificates are signed by this CA.
+	CA *x509.Certificate
+}