m/n/core/rpc: create library for common gRPC functions

This is the beginning of consolidating all gRPC-related code into a
single package.

We also run the Curator service publicly and place it behind a new
authorization permission bit. This is in preparation for Curator
followers needing access to this Service.

Some of the service split and authorization options are likely to be
changed in the future (I'm considering renaming Curator to something
else, or at least clearly stating that it's a node-to-node service).

Change-Id: I0a4a57da15b35688aefe7bf669ba6342d46aa3f5
Reviewed-on: https://review.monogon.dev/c/monogon/+/316
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/core/curator/BUILD.bazel b/metropolis/node/core/curator/BUILD.bazel
index db02a3c..25be134 100644
--- a/metropolis/node/core/curator/BUILD.bazel
+++ b/metropolis/node/core/curator/BUILD.bazel
@@ -3,7 +3,6 @@
 go_library(
     name = "go_default_library",
     srcs = [
-        "authorization.go",
         "bootstrap.go",
         "curator.go",
         "impl_follower.go",
@@ -23,6 +22,7 @@
         "//metropolis/node/core/curator/proto/api:go_default_library",
         "//metropolis/node/core/curator/proto/private:go_default_library",
         "//metropolis/node/core/localstorage:go_default_library",
+        "//metropolis/node/core/rpc:go_default_library",
         "//metropolis/pkg/combinectx:go_default_library",
         "//metropolis/pkg/event:go_default_library",
         "//metropolis/pkg/event/etcd:go_default_library",
@@ -31,7 +31,6 @@
         "//metropolis/pkg/supervisor:go_default_library",
         "//metropolis/proto/api:go_default_library",
         "//metropolis/proto/common:go_default_library",
-        "//metropolis/proto/ext:go_default_library",
         "@io_etcd_go_etcd//clientv3:go_default_library",
         "@io_etcd_go_etcd//clientv3/concurrency:go_default_library",
         "@org_golang_google_grpc//:go_default_library",
@@ -40,8 +39,6 @@
         "@org_golang_google_grpc//peer:go_default_library",
         "@org_golang_google_grpc//status: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",
         "@org_golang_x_sys//unix:go_default_library",
     ],
 )
@@ -58,6 +55,7 @@
         "//metropolis/node/core/consensus/client:go_default_library",
         "//metropolis/node/core/localstorage:go_default_library",
         "//metropolis/node/core/localstorage/declarative:go_default_library",
+        "//metropolis/node/core/rpc:go_default_library",
         "//metropolis/pkg/event/memory:go_default_library",
         "//metropolis/pkg/pki:go_default_library",
         "//metropolis/pkg/supervisor:go_default_library",
diff --git a/metropolis/node/core/curator/curator.go b/metropolis/node/core/curator/curator.go
index a8504f9..35b066e 100644
--- a/metropolis/node/core/curator/curator.go
+++ b/metropolis/node/core/curator/curator.go
@@ -26,6 +26,7 @@
 	"source.monogon.dev/metropolis/node/core/consensus/client"
 	ppb "source.monogon.dev/metropolis/node/core/curator/proto/private"
 	"source.monogon.dev/metropolis/node/core/localstorage"
+	"source.monogon.dev/metropolis/node/core/rpc"
 	"source.monogon.dev/metropolis/pkg/event"
 	"source.monogon.dev/metropolis/pkg/event/memory"
 	"source.monogon.dev/metropolis/pkg/supervisor"
@@ -270,9 +271,9 @@
 	// running leader, or forwarding to a remotely running leader.
 	lis := listener{
 		directory: s.config.Directory,
-		listenerSecurity: listenerSecurity{
-			nodeCredentials:      s.config.ServerCredentials,
-			clusterCACertificate: s.config.ClusterCACertificate,
+		ServerSecurity: rpc.ServerSecurity{
+			NodeCredentials:      s.config.ServerCredentials,
+			ClusterCACertificate: s.config.ClusterCACertificate,
 		},
 		electionWatch: s.electionWatch,
 		etcd:          s.config.Etcd,
diff --git a/metropolis/node/core/curator/impl_leader_test.go b/metropolis/node/core/curator/impl_leader_test.go
index 2aa8e53..eb1db38 100644
--- a/metropolis/node/core/curator/impl_leader_test.go
+++ b/metropolis/node/core/curator/impl_leader_test.go
@@ -14,6 +14,7 @@
 	"google.golang.org/grpc/test/bufconn"
 
 	"source.monogon.dev/metropolis/node/core/consensus/client"
+	"source.monogon.dev/metropolis/node/core/rpc"
 	"source.monogon.dev/metropolis/pkg/pki"
 	apb "source.monogon.dev/metropolis/proto/api"
 )
@@ -69,14 +70,14 @@
 	// Build a test cluster PKI and node/manager certificates, and create the
 	// listener security parameters which will authenticate incoming requests.
 	node, manager, ca := pki.EphemeralClusterCredentials(t)
-	sec := &listenerSecurity{
-		nodeCredentials:      node,
-		clusterCACertificate: ca,
+	sec := &rpc.ServerSecurity{
+		NodeCredentials:      node,
+		ClusterCACertificate: ca,
 	}
 
 	// Create a curator gRPC server which performs authentication as per the created
 	// listenerSecurity and is backed by the created leader.
-	srv := sec.setupPublicGRPC(leader)
+	srv := sec.SetupPublicGRPC(leader)
 	// The gRPC server will listen on an internal 'loopback' buffer.
 	lis := bufconn.Listen(1024 * 1024)
 	go func() {
diff --git a/metropolis/node/core/curator/listener.go b/metropolis/node/core/curator/listener.go
index 5a8cb5e..c49eab3 100644
--- a/metropolis/node/core/curator/listener.go
+++ b/metropolis/node/core/curator/listener.go
@@ -14,6 +14,7 @@
 	"source.monogon.dev/metropolis/node/core/consensus/client"
 	cpb "source.monogon.dev/metropolis/node/core/curator/proto/api"
 	"source.monogon.dev/metropolis/node/core/localstorage"
+	"source.monogon.dev/metropolis/node/core/rpc"
 	"source.monogon.dev/metropolis/pkg/combinectx"
 	"source.monogon.dev/metropolis/pkg/supervisor"
 	apb "source.monogon.dev/metropolis/proto/api"
@@ -37,7 +38,7 @@
 // some calls might not be idempotent and the caller is better equipped to know
 // when to retry.
 type listener struct {
-	listenerSecurity
+	rpc.ServerSecurity
 
 	// etcd is a client to the locally running consensus (etcd) server which is used
 	// both for storing lock/leader election status and actual Curator data.
@@ -113,14 +114,6 @@
 	}
 }
 
-// services is the interface containing all gRPC services that a curator
-// must implement.
-type services interface {
-	cpb.CuratorServer
-	apb.AAAServer
-	apb.ManagementServer
-}
-
 // activeTarget is the active implementation used by the listener dispatcher, or
 // nil if none is active yet.
 type activeTarget struct {
@@ -130,7 +123,7 @@
 	// context cancel function for ctx, or nil if ctx is nil.
 	ctxC *context.CancelFunc
 	// active Curator implementation, or nil if not yet set up.
-	impl services
+	impl rpc.ClusterServices
 }
 
 // switchTo switches the activeTarget over to a Curator implementation as per
@@ -170,7 +163,7 @@
 	ctx context.Context
 	// impl is the CuratorServer implementation to which RPCs should be directed
 	// according to the dispatcher.
-	impl services
+	impl rpc.ClusterServices
 }
 
 // dispatch contacts the dispatcher to retrieve an up-to-date listenerTarget.
@@ -207,7 +200,7 @@
 	srvLocal := grpc.NewServer()
 	cpb.RegisterCuratorServer(srvLocal, l)
 
-	srvPublic := l.setupPublicGRPC(l)
+	srvPublic := l.SetupPublicGRPC(l)
 
 	err := supervisor.Run(ctx, "local", func(ctx context.Context) error {
 		lisLocal, err := net.ListenUnix("unix", &net.UnixAddr{Name: l.directory.ClientSocket.FullPath(), Net: "unix"})
@@ -256,7 +249,7 @@
 // returned are either returned directly or converted to an UNAVAILABLE status
 // if the error is as a result of the context being canceled due to the
 // implementation switching.
-type implOperation func(ctx context.Context, impl services) error
+type implOperation func(ctx context.Context, impl rpc.ClusterServices) error
 
 // callImpl gets the newest listenerTarget from the dispatcher, combines the
 // given context with the context of the listenerTarget implementation and calls
@@ -316,7 +309,7 @@
 }
 
 func (l *listener) Watch(req *cpb.WatchRequest, srv cpb.Curator_WatchServer) error {
-	proxy := func(ctx context.Context, impl services) error {
+	proxy := func(ctx context.Context, impl rpc.ClusterServices) error {
 		return impl.Watch(req, &curatorWatchServer{
 			ServerStream: srv,
 			ctx:          ctx,
@@ -347,7 +340,7 @@
 }
 
 func (l *listener) Escrow(srv apb.AAA_EscrowServer) error {
-	return l.callImpl(srv.Context(), func(ctx context.Context, impl services) error {
+	return l.callImpl(srv.Context(), func(ctx context.Context, impl rpc.ClusterServices) error {
 		return impl.Escrow(&aaaEscrowServer{
 			ServerStream: srv,
 			ctx:          ctx,
@@ -356,7 +349,7 @@
 }
 
 func (l *listener) GetRegisterTicket(ctx context.Context, req *apb.GetRegisterTicketRequest) (res *apb.GetRegisterTicketResponse, err error) {
-	err = l.callImpl(ctx, func(ctx context.Context, impl services) error {
+	err = l.callImpl(ctx, func(ctx context.Context, impl rpc.ClusterServices) error {
 		var err2 error
 		res, err2 = impl.GetRegisterTicket(ctx, req)
 		return err2
diff --git a/metropolis/node/core/curator/listener_test.go b/metropolis/node/core/curator/listener_test.go
index 95afe85..4644f6c 100644
--- a/metropolis/node/core/curator/listener_test.go
+++ b/metropolis/node/core/curator/listener_test.go
@@ -11,6 +11,7 @@
 
 	"source.monogon.dev/metropolis/node/core/localstorage"
 	"source.monogon.dev/metropolis/node/core/localstorage/declarative"
+	"source.monogon.dev/metropolis/node/core/rpc"
 	"source.monogon.dev/metropolis/pkg/event/memory"
 	"source.monogon.dev/metropolis/pkg/supervisor"
 )
@@ -70,7 +71,7 @@
 	// Check that canceling the request unblocks a pending dispatched call.
 	errC := make(chan error)
 	go func() {
-		errC <- l.callImpl(ctxR, func(ctx context.Context, impl services) error {
+		errC <- l.callImpl(ctxR, func(ctx context.Context, impl rpc.ClusterServices) error {
 			<-ctx.Done()
 			return ctx.Err()
 		})
@@ -84,7 +85,7 @@
 	// Check that switching implementations unblocks a pending dispatched call.
 	scheduledC := make(chan struct{})
 	go func() {
-		errC <- l.callImpl(ctx, func(ctx context.Context, impl services) error {
+		errC <- l.callImpl(ctx, func(ctx context.Context, impl rpc.ClusterServices) error {
 			close(scheduledC)
 			<-ctx.Done()
 			return ctx.Err()
diff --git a/metropolis/node/core/curator/proto/api/BUILD.bazel b/metropolis/node/core/curator/proto/api/BUILD.bazel
index e777b02..2cec346 100644
--- a/metropolis/node/core/curator/proto/api/BUILD.bazel
+++ b/metropolis/node/core/curator/proto/api/BUILD.bazel
@@ -6,7 +6,10 @@
     name = "api_proto",
     srcs = ["api.proto"],
     visibility = ["//visibility:public"],
-    deps = ["//metropolis/proto/common:common_proto"],
+    deps = [
+        "//metropolis/proto/common:common_proto",
+        "//metropolis/proto/ext:ext_proto",
+    ],
 )
 
 go_proto_library(
@@ -15,7 +18,10 @@
     importpath = "source.monogon.dev/metropolis/node/core/curator/proto/api",
     proto = ":api_proto",
     visibility = ["//visibility:public"],
-    deps = ["//metropolis/proto/common:go_default_library"],
+    deps = [
+        "//metropolis/proto/common:go_default_library",
+        "//metropolis/proto/ext:go_default_library",
+    ],
 )
 
 go_library(
diff --git a/metropolis/node/core/curator/proto/api/api.proto b/metropolis/node/core/curator/proto/api/api.proto
index 372e361..da3e3c3 100644
--- a/metropolis/node/core/curator/proto/api/api.proto
+++ b/metropolis/node/core/curator/proto/api/api.proto
@@ -3,6 +3,7 @@
 package metropolis.node.core.curator.proto.api;
 
 import "metropolis/proto/common/common.proto";
+import "metropolis/proto/ext/authorization.proto";
 
 // The Curator is the main cluster management service of Metropolis.
 //
@@ -40,7 +41,11 @@
     // object state, as streamed WatchEvents are not synchronous to internal
     // state changes within the Curator. Effectively, the view of Watch clients
     // is eventually consistent with the state of the objects in the Curator.
-    rpc Watch(WatchRequest) returns (stream WatchEvent);
+    rpc Watch(WatchRequest) returns (stream WatchEvent) {
+        option (metropolis.proto.ext.authorization) = {
+            need: PERMISSION_READ_CLUSTER_STATUS;
+        };
+    }
 }
 
 // Node is the state and configuration of a node in the cluster.
diff --git a/metropolis/node/core/rpc/BUILD.bazel b/metropolis/node/core/rpc/BUILD.bazel
new file mode 100644
index 0000000..df03356
--- /dev/null
+++ b/metropolis/node/core/rpc/BUILD.bazel
@@ -0,0 +1,25 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+    name = "go_default_library",
+    srcs = [
+        "client.go",
+        "server.go",
+    ],
+    importpath = "source.monogon.dev/metropolis/node/core/rpc",
+    visibility = ["//visibility:public"],
+    deps = [
+        "//metropolis/node/core/curator/proto/api:go_default_library",
+        "//metropolis/pkg/pki:go_default_library",
+        "//metropolis/proto/api:go_default_library",
+        "//metropolis/proto/ext:go_default_library",
+        "@org_golang_google_grpc//:go_default_library",
+        "@org_golang_google_grpc//codes:go_default_library",
+        "@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_protobuf//proto:go_default_library",
+        "@org_golang_google_protobuf//reflect/protoreflect:go_default_library",
+        "@org_golang_google_protobuf//reflect/protoregistry:go_default_library",
+    ],
+)
diff --git a/metropolis/node/core/rpc/client.go b/metropolis/node/core/rpc/client.go
new file mode 100644
index 0000000..cc48f95
--- /dev/null
+++ b/metropolis/node/core/rpc/client.go
@@ -0,0 +1,131 @@
+package rpc
+
+import (
+	"context"
+	"crypto/ed25519"
+	"crypto/rand"
+	"crypto/tls"
+	"crypto/x509"
+	"fmt"
+	"math/big"
+	"time"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/credentials"
+
+	"source.monogon.dev/metropolis/pkg/pki"
+	apb "source.monogon.dev/metropolis/proto/api"
+)
+
+// 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.
+//
+// These self-signed certificates are used by clients connecting to the cluster
+// which want to prove ownership of an ED25519 keypair but don't have any
+// '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) {
+	template := x509.Certificate{
+		SerialNumber: big.NewInt(1),
+		NotBefore:    time.Now(),
+		NotAfter:     pki.UnknownNotAfter,
+
+		KeyUsage:              x509.KeyUsageKeyEncipherment | x509.KeyUsageDigitalSignature,
+		ExtKeyUsage:           []x509.ExtKeyUsage{x509.ExtKeyUsageClientAuth},
+		BasicConstraintsValid: true,
+	}
+	certificateBytes, err := x509.CreateCertificate(rand.Reader, &template, &template, private.Public(), private)
+	if err != nil {
+		return nil, fmt.Errorf("when generating self-signed certificate: %w", err)
+	}
+	certificate := tls.Certificate{
+		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
+			}
+
+			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))
+}
+
+// RetrieveOwnerCertificates uses AAA.Escrow to retrieve a cluster manager
+// certificate for the initial owner of the cluster, authenticated by the
+// public/private key set in the clusters NodeParameters.ClusterBoostrap.
+//
+// The retrieved certificate can be used to dial further cluster RPCs.
+func RetrieveOwnerCertificate(ctx context.Context, aaa apb.AAAClient, private ed25519.PrivateKey) (*tls.Certificate, error) {
+	srv, err := aaa.Escrow(ctx)
+	if err != nil {
+		return nil, fmt.Errorf("when opening Escrow RPC: %w", err)
+	}
+	if err := srv.Send(&apb.EscrowFromClient{
+		Parameters: &apb.EscrowFromClient_Parameters{
+			RequestedIdentityName: "owner",
+			PublicKey:             private.Public().(ed25519.PublicKey),
+		},
+	}); err != nil {
+		return nil, fmt.Errorf("when sending client parameters: %w", err)
+	}
+	resp, err := srv.Recv()
+	if err != nil {
+		return nil, fmt.Errorf("when receiving server message: %w", err)
+	}
+	if len(resp.EmittedCertificate) == 0 {
+		return nil, fmt.Errorf("expected certificate, instead got needed proofs: %+v", resp.Needed)
+	}
+
+	return &tls.Certificate{
+		Certificate: [][]byte{resp.EmittedCertificate},
+		PrivateKey:  private,
+	}, nil
+}
diff --git a/metropolis/node/core/curator/authorization.go b/metropolis/node/core/rpc/server.go
similarity index 66%
rename from metropolis/node/core/curator/authorization.go
rename to metropolis/node/core/rpc/server.go
index 066c46b..3d6917d 100644
--- a/metropolis/node/core/curator/authorization.go
+++ b/metropolis/node/core/rpc/server.go
@@ -1,4 +1,4 @@
-package curator
+package rpc
 
 import (
 	"context"
@@ -15,30 +15,46 @@
 	"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"
 )
 
-// listenerSecurity are the security options for the listener, relating to
-// authentication and authorization.
-//
-// They are factored out to a separate struct for ease of testing.
-type listenerSecurity struct {
-	// nodeCredentials is the TLS certificate/key of the node that the listener
-	// 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
+// 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 {
+	cpb.CuratorServer
+	apb.AAAServer
+	apb.ManagementServer
 }
 
-// setupPublicGRPC returns a grpc.Server ready to listen and serve all public
-// gRPC APIs that the listener should run, with all calls being authenticated
-// and authorized based on the data in listenerSecurity. The argument 'impls' is
-// the object implementing the gRPC APIs.
-func (l *listenerSecurity) setupPublicGRPC(impls services) *grpc.Server {
+// 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
+}
+
+// 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},
+		Certificates: []tls.Certificate{l.NodeCredentials},
 		ClientAuth:   tls.RequestClientCert,
 	})
 
@@ -47,6 +63,7 @@
 		grpc.UnaryInterceptor(l.unaryInterceptor),
 		grpc.StreamInterceptor(l.streamInterceptor),
 	)
+	cpb.RegisterCuratorServer(s, impls)
 	apb.RegisterAAAServer(s, impls)
 	apb.RegisterManagementServer(s, impls)
 	return s
@@ -61,7 +78,7 @@
 // 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 *listenerSecurity) authorize(ctx context.Context, methodName string) error {
+func (l *ServerSecurity) authorize(ctx context.Context, methodName string) error {
 	if !strings.HasPrefix(methodName, "/") {
 		return status.Errorf(codes.InvalidArgument, "invalid method name %q", methodName)
 	}
@@ -106,7 +123,7 @@
 	pCert := tlsInfo.State.PeerCertificates[0]
 
 	// Ensure that the certificate is signed by the cluster CA.
-	if err := pCert.CheckSignatureFrom(l.clusterCACertificate); err != nil {
+	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.
@@ -130,7 +147,7 @@
 // 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 *listenerSecurity) streamInterceptor(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+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
 	}
@@ -140,7 +157,7 @@
 // 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 *listenerSecurity) unaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) {
+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
 	}