Skip to content

Commit

Permalink
server: use raftSparseStatus in replica metrics
Browse files Browse the repository at this point in the history
`Replica.Metrics()` calls `raftStatusRLocked()`, periodically across all
replicas, which is moderately expensive due to deep copies of fields we
don't care about. This patch instead uses `raftSparseStatusRLocked()`
which omits these fields.

Epic: none
Release note: None
  • Loading branch information
erikgrinaker committed Jan 4, 2023
1 parent 6225ad6 commit ac3e633
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 11 deletions.
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/replica_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ func (r *Replica) Metrics(
livenessMap: livenessMap,
clusterNodes: clusterNodes,
desc: r.mu.state.Desc,
raftStatus: r.raftStatusRLocked(),
raftStatus: r.raftSparseStatusRLocked(),
leaseStatus: r.leaseStatusAtRLocked(ctx, now),
storeID: r.store.StoreID(),
quiescent: r.mu.quiescent,
Expand All @@ -110,7 +110,7 @@ type calcReplicaMetricsInput struct {
livenessMap livenesspb.IsLiveMap
clusterNodes int
desc *roachpb.RangeDescriptor
raftStatus *raft.Status
raftStatus *raftSparseStatus
leaseStatus kvserverpb.LeaseStatus
storeID roachpb.StoreID
quiescent bool
Expand Down Expand Up @@ -138,7 +138,7 @@ func calcReplicaMetrics(d calcReplicaMetricsInput) ReplicaMetrics {

// The raft leader computes the number of raft entries that replicas are
// behind.
leader := isRaftLeader(d.raftStatus)
leader := d.raftStatus != nil && d.raftStatus.RaftState == raft.StateLeader
var leaderBehindCount, leaderPausedFollowerCount int64
if leader {
leaderBehindCount = calcBehindCount(d.raftStatus, d.desc, d.livenessMap)
Expand Down Expand Up @@ -263,7 +263,7 @@ func calcLiveReplicas(repls []roachpb.ReplicaDescriptor, livenessMap livenesspb.
// calcBehindCount returns a total count of log entries that follower replicas
// are behind. This can only be computed on the raft leader.
func calcBehindCount(
raftStatus *raft.Status, desc *roachpb.RangeDescriptor, livenessMap livenesspb.IsLiveMap,
raftStatus *raftSparseStatus, desc *roachpb.RangeDescriptor, livenessMap livenesspb.IsLiveMap,
) int64 {
var behindCount int64
for _, rd := range desc.Replicas().Descriptors() {
Expand Down
4 changes: 0 additions & 4 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -1707,10 +1707,6 @@ func (r *Replica) hasOutstandingSnapshotInFlightToStore(storeID roachpb.StoreID)
return r.getSnapshotLogTruncationConstraints(storeID) > 0
}

func isRaftLeader(raftStatus *raft.Status) bool {
return raftStatus != nil && raftStatus.SoftState.RaftState == raft.StateLeader
}

// HasRaftLeader returns true if the raft group has a raft leader currently.
func HasRaftLeader(raftStatus *raft.Status) bool {
return raftStatus != nil && raftStatus.SoftState.Lead != 0
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9010,8 +9010,8 @@ func TestReplicaMetrics(t *testing.T) {
}
return m
}
status := func(lead uint64, progress map[uint64]tracker.Progress) *raft.Status {
status := &raft.Status{
status := func(lead uint64, progress map[uint64]tracker.Progress) *raftSparseStatus {
status := &raftSparseStatus{
Progress: progress,
}
// The commit index is set so that a progress.Match value of 1 is behind
Expand Down Expand Up @@ -9055,7 +9055,7 @@ func TestReplicaMetrics(t *testing.T) {
replicas int32
storeID roachpb.StoreID
desc roachpb.RangeDescriptor
raftStatus *raft.Status
raftStatus *raftSparseStatus
liveness livenesspb.IsLiveMap
raftLogSize int64
expected ReplicaMetrics
Expand Down

0 comments on commit ac3e633

Please sign in to comment.