m/n/core/curator: expose consensus service status

This replaces https://review.monogon.dev/2071 .

Consensus service status will be used in e2e tests to more accurately
determine the health of a cluster.

Change-Id: Ia304d2a679f5ffdccd5737d5770d40a4a08f7f4a
Reviewed-on: https://review.monogon.dev/c/monogon/+/2879
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
Tested-by: Jenkins CI
diff --git a/metropolis/node/core/curator/curator.go b/metropolis/node/core/curator/curator.go
index e88493a..0026d28 100644
--- a/metropolis/node/core/curator/curator.go
+++ b/metropolis/node/core/curator/curator.go
@@ -345,10 +345,11 @@
 	// the Curator API to consumers, dispatching to either a locally running leader,
 	// or forwarding to a remotely running leader.
 	lis := listener{
-		node:      s.config.NodeCredentials,
-		etcd:      etcd,
-		consensus: s.config.Consensus,
-		status:    &s.status,
+		node:        s.config.NodeCredentials,
+		etcd:        etcd,
+		etcdCluster: st.ClusterClient(),
+		consensus:   s.config.Consensus,
+		status:      &s.status,
 	}
 	if err := supervisor.Run(ctx, "listener", lis.run); err != nil {
 		return fmt.Errorf("when starting listener: %w", err)
diff --git a/metropolis/node/core/curator/impl_leader.go b/metropolis/node/core/curator/impl_leader.go
index 177d436..d7c6222 100644
--- a/metropolis/node/core/curator/impl_leader.go
+++ b/metropolis/node/core/curator/impl_leader.go
@@ -50,7 +50,8 @@
 	leaderID string
 	// etcd is the etcd client in which curator data and leader election state is
 	// stored.
-	etcd client.Namespaced
+	etcd        client.Namespaced
+	etcdCluster clientv3.Cluster
 
 	// muNodes guards any changes to nodes, and prevents race conditions where the
 	// curator performs a read-modify-write operation to node data. The curator's
diff --git a/metropolis/node/core/curator/impl_leader_curator.go b/metropolis/node/core/curator/impl_leader_curator.go
index d4e9e29..9cbbbe6 100644
--- a/metropolis/node/core/curator/impl_leader_curator.go
+++ b/metropolis/node/core/curator/impl_leader_curator.go
@@ -576,3 +576,24 @@
 	rpc.Trace(ctx).Printf("Interrupting due to context cancellation")
 	return nil
 }
+
+func (l *leaderCurator) GetConsensusStatus(ctx context.Context, _ *ipb.GetConsensusStatusRequest) (*ipb.GetConsensusStatusResponse, error) {
+	var res ipb.GetConsensusStatusResponse
+	members, err := l.etcdCluster.MemberList(ctx)
+	if err != nil {
+		rpc.Trace(ctx).Printf("Could not get etcd members: %v", err)
+		return nil, status.Errorf(codes.Internal, "could not get etcd members")
+	}
+	for _, member := range members.Members {
+		st := ipb.GetConsensusStatusResponse_EtcdMember{
+			Id:     member.Name,
+			Status: ipb.GetConsensusStatusResponse_EtcdMember_STATUS_FULL,
+		}
+		if member.IsLearner {
+			st.Status = ipb.GetConsensusStatusResponse_EtcdMember_STATUS_LEARNER
+		}
+		res.EtcdMember = append(res.EtcdMember, &st)
+	}
+
+	return &res, nil
+}
diff --git a/metropolis/node/core/curator/listener.go b/metropolis/node/core/curator/listener.go
index 620bc05..77fe0e0 100644
--- a/metropolis/node/core/curator/listener.go
+++ b/metropolis/node/core/curator/listener.go
@@ -6,6 +6,7 @@
 	"net"
 	"time"
 
+	clientv3 "go.etcd.io/etcd/client/v3"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/keepalive"
 
@@ -41,7 +42,8 @@
 	node *identity.NodeCredentials
 	// 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.
-	etcd client.Namespaced
+	etcd        client.Namespaced
+	etcdCluster clientv3.Cluster
 
 	consensus consensus.ServiceHandle
 	status    *memory.Value[*electionStatus]
@@ -99,11 +101,12 @@
 
 		// Create a leader instance and serve it over gRPC.
 		leader := newCuratorLeader(&leadership{
-			lockKey:   st.leader.lockKey,
-			lockRev:   st.leader.lockRev,
-			leaderID:  l.node.ID(),
-			etcd:      l.etcd,
-			consensus: l.consensus,
+			lockKey:     st.leader.lockKey,
+			lockRev:     st.leader.lockRev,
+			leaderID:    l.node.ID(),
+			etcd:        l.etcd,
+			etcdCluster: l.etcdCluster,
+			consensus:   l.consensus,
 		}, &l.node.Node)
 
 		cpb.RegisterCuratorServer(srv, leader)
diff --git a/metropolis/node/core/curator/proto/api/api.proto b/metropolis/node/core/curator/proto/api/api.proto
index 31c221d..ff30ff1 100644
--- a/metropolis/node/core/curator/proto/api/api.proto
+++ b/metropolis/node/core/curator/proto/api/api.proto
@@ -153,6 +153,15 @@
             need: PERMISSION_UPDATE_NODE_SELF
         };
     }
+
+    // GetConsensusStatus returns the status of the consensus service (etcd)
+    // running on curators. This can be used to detect the health of the cluster
+    // before operational changes.
+    rpc GetConsensusStatus(GetConsensusStatusRequest) returns (GetConsensusStatusResponse) {
+        option (metropolis.proto.ext.authorization) = {
+            need: PERMISSION_READ_CLUSTER_STATUS
+        };
+    }
 }
 
 // Node is the state and configuration of a node in the cluster.
@@ -437,4 +446,29 @@
 }
 
 message UpdateNodeClusterNetworkingResponse {
-}
\ No newline at end of file
+}
+
+message GetConsensusStatusRequest {
+}
+
+message GetConsensusStatusResponse {
+  // A control plane member as seen by the etcd cluster.
+  message EtcdMember {
+    // The ID of the etcd member node. This should be the same as the ID of the
+    // Metropolis node running this etcd member.
+    string id = 1;
+    // Status of this node from the point of view of etcd.
+    enum Status {
+      STATUS_UNKNOWN = 0;
+      // The node is an etcd learner - it cannot run the Curator control plane
+      // services yet because it's still catching up to the etcd cluster state.
+      STATUS_LEARNER = 1;
+      // This node is a full etcd member, and can become a Curator control plane
+      // leader.
+      STATUS_FULL = 2;
+    };
+    Status status = 2;
+  }
+  // All members of the etcd cluster.
+  repeated EtcdMember etcd_member = 1;
+}