mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raft: provide protection against unbounded Raft log growth
The suggested pattern for Raft proposals is that they be retried periodically until they succeed. This turns out to be an issue when a leader cannot commit entries because the leader will continue to append re-proposed entries to its log without committing anything. This can result in the uncommitted tail of a leader's log growing without bound until it is able to commit entries. This change add a safeguard to protect against this case where a leader's log can grow without bound during loss of quorum scenarios. It does so by introducing a new, optional ``MaxUncommittedEntriesSize configuration. This config limits the max aggregate size of uncommitted entries that may be appended to a leader's log. Once this limit is exceeded, proposals will begin to return ErrProposalDropped errors. See cockroachdb/cockroach#27772
This commit is contained in:
@@ -401,6 +401,7 @@ func (n *node) run(r *raft) {
|
||||
|
||||
r.msgs = nil
|
||||
r.readStates = nil
|
||||
r.reduceUncommittedSize(rd.CommittedEntries)
|
||||
advancec = n.advancec
|
||||
case <-advancec:
|
||||
if applyingToI != 0 {
|
||||
|
||||
Reference in New Issue
Block a user