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;
+}