Skip to content

Commit

Permalink
kvserver: log when raft send/recv queue fills up
Browse files Browse the repository at this point in the history
Inspired by cockroachlabs/support#1770.

If either the raft send or receive queue fills up, wide-spread outages
can occur as replication progress stalls. We have metrics that can
indicate this, but straightforward logging is also appropriate to direct
attention to the fact, which this commit achieves.

Touches cockroachdb#79755

Release justification: important logging improvement
Release note: None
  • Loading branch information
tbg committed Sep 20, 2022
1 parent 2672bbd commit 516992f
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 3 deletions.
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/raft_transport.go
Original file line number Diff line number Diff line change
Expand Up @@ -605,6 +605,9 @@ func (t *RaftTransport) SendAsync(
}
return true
default:
if logRaftSendQueueFullEvery.ShouldLog() {
log.Warningf(t.AnnotateCtx(context.Background()), "raft send queue to n%d is full", toNodeID)
}
releaseRaftMessageRequest(req)
return false
}
Expand Down
14 changes: 11 additions & 3 deletions pkg/kv/kvserver/store_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,11 @@ import (
"go.etcd.io/etcd/raft/v3/raftpb"
)

var (
logRaftRecvQueueFullEvery = log.Every(1 * time.Second)
logRaftSendQueueFullEvery = log.Every(1 * time.Second)
)

type raftRequestInfo struct {
req *kvserverpb.RaftMessageRequest
respStream RaftMessageResponseStream
Expand Down Expand Up @@ -177,6 +182,9 @@ func (s *Store) HandleRaftUncoalescedRequest(
// TODO(peter): Return an error indicating the request was dropped. Note
// that dropping the request is safe. Raft will retry.
s.metrics.RaftRcvdMsgDropped.Inc(1)
if logRaftRecvQueueFullEvery.ShouldLog() {
log.Warningf(ctx, "raft receive queue for r%d is full", req.RangeID)
}
return false
}
q.infos = append(q.infos, raftRequestInfo{
Expand Down Expand Up @@ -540,9 +548,9 @@ func (s *Store) processTick(_ context.Context, rangeID roachpb.RangeID) bool {
// See the comment in shouldFollowerQuiesceOnNotify for details on how these two
// functions combine to provide the guarantee that:
//
// If a quorum of replica in a Raft group is alive and at least
// one of these replicas is up-to-date, the Raft group will catch
// up any of the live, lagging replicas.
// If a quorum of replica in a Raft group is alive and at least
// one of these replicas is up-to-date, the Raft group will catch
// up any of the live, lagging replicas.
//
// Note that this mechanism can race with concurrent invocations of processTick,
// which may have a copy of the previous livenessMap where the now-live node is
Expand Down

0 comments on commit 516992f

Please sign in to comment.