-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
storage: examine raft log truncation heuristics #10917
Comments
Ah, I messed up my instrumentation. The raft log entries are actually ~640 bytes per entry. Still that translates can translate into 1638 raft entries in a single |
Btw, if you want to reproduce this locally:
|
Adding some instrumentation to
I think the above is pretty good evidence that at least some of our long |
The previous settings allowed up to 256 MB of Raft log entries to be inflight to a follower, resulting in a single Replica.handleRaftReady call processing thousands or 10s of thousands of commands. Log the number of commands processed when Replica.handleRaftReady takes too long. Fixes cockroachdb#10917
The previous settings allowed up to 256 MB of Raft log entries to be inflight to a follower, resulting in a single Replica.handleRaftReady call processing thousands or 10s of thousands of commands. Log the number of commands processed when Replica.handleRaftReady takes too long. Fixes cockroachdb#10917
In experimentation on
blue
, I've noticed that the longer a node is down for the more instances I see of long lock hold warnings. The down times I experimented with were in the 30s-2m range and the warnings are coming out ofReplica.handleRaftReady
. I also noticed that ranges were being caught up via log appends, not snapshots. The raft log truncation heuristics allow the raft log to grow up to the size of Replica before it is truncated. But perhaps that isn't aggressive enough.A related item to investigate is
raft.Config.MaxSizePerMsg
. We currently have this set to 1MB. A bit of instrumentation on local load similar toblue
shows that 1MB of raft log held ~26k entries. That translates to ~40 bytes per entry which doesn't seem right. The problem with settingMaxSizePerMsg
high is that is the unit which Raft will use to catch up a behind node via log appends. Processing ~26k entries in a single Raft ready message would be excessive. One thought which comes to mind is to limit the number of entries we return fromReplica.Entries
. We could either reinterpretMaxSizePerMsg
(themaxBytes
parameter) as a number of entries, or have another limit (though that would require care as raft internally needs to retrieve all of the entries when a replica becomes the leader).@bdarnell thoughts?
The text was updated successfully, but these errors were encountered: