-
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: excessively large raft logs #27772
Comments
I wrote up a test case that proves this theory to be valid. See #27774. |
Because we use a On a side note, paging this back in/learning new stuff about re-proposals has made me much more excited about #21849. I think we should begin handling |
The following is enough to get the new test (and all others in the
|
No, PreVote and CheckQuorum are compatible. We treated them as mutually exclusive because they are partially redundant (our original interest in CheckQuorum was for its ability to minimize disruption when recovering from partitions, which is also addressed in a better way by PreVote). The reason we disabled CheckQuorum when we enabled PreVote is that CheckQuorum is not quite compatible with quiesced replicas, and required hacks like the (now removed) tickQuiesced method.
What specifically would you do with this error to improve latency? We can't do anything until we know the leader, and when we know we'll generate a reasonNewLeader refresh. As for the fix, I wasn't thinking we'd need to track whether a particular proposal had been proposed by a leader. All we need is to check whether this replica is currently the leader, and if so ignore reasonTicks. The theory is that if we were not the leader at the time of the previous proposal, we'd get a reasonNewLeader when we became leader, and are therefore not relying on reasonTicks. Alternately, we could use NodeLiveness, and ignore reasonTicks (regardless of our current state) whenever NodeLiveness indicates that there is not a quorum of live nodes. This is a little more expensive and a little more code than checking the state, but it feels like a narrower fix and might be more suitable to backport (I'm vaguely concerned about missing an edge case if we base everything on raft leadership). |
Ah, so |
I didn't fully understand the interactions here when posting that. I think I do a little better now. Is it safe to think about
I think this would also adress the 5x replication factor issue discussion mentioned in #27774 (comment). Do you see any downsides to tying this to node liveness though? For instance, I'm worried we might run into issues on the node liveness range itself when a node that is currently down is trying to indicate that it is now live. Maybe that's fine because node liveness updates are in their own retry loop anyway. |
There is no queuing at the raft level. The raft Propose method does one of three things:
ErrProposalDropped tells us that the leader is unknown, so we could infer that no proposals will succeed until we learn of the new leader.
Yeah, as I said on the other issue, there might be trouble with the liveness range itself. The fact that liveness updates have short timeouts and get retried themselves may mitigate it, though. |
Right, I didn't mean actual queuing, I was just making an analogy for the sake of my own understanding. It is safe to ignore an |
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log.
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log.
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log.
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent unbounded growth of the raft log caused by a loss of quorum.
27774: storage: prevent unbounded raft log growth without quorum r=nvanbenschoten a=nvanbenschoten Fixes #27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in #27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log. Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent unbounded growth of the raft log caused by a loss of quorum.
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent unbounded growth of the raft log caused by a loss of quorum.
27868: backport-2.0: storage: prevent unbounded raft log growth without quorum r=nvanbenschoten a=nvanbenschoten Backport 2/2 commits from #27774. /cc @cockroachdb/release --- Fixes #27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in #27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log. 28225: release-2.0: importccl: Preserve '\r\n' during CSV import r=dt a=dt Backport 1/1 commits from #28181. /cc @cockroachdb/release --- See #25344. Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com> Co-authored-by: neeral <neeral@users.noreply.github.com> Co-authored-by: David Taylor <tinystatemachine@gmail.com>
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 `MaxUncommittedEntries` configuration. This config limits the max number 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
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 `MaxUncommittedEntries` configuration. This config limits the max number 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
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
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
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
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
We've seen a handful of occurrences of excessively large Raft logs in clusters in the wild without being able to identify the root cause. A large Raft log should normally not occur as the proposal quota mechanism should limit the size of the Raft log if a range is healthy. If one node in the range is down the proposal quota mechanism will drop that replica from consideration, but at the same time allow Raft log truncation to occur, so the size of the Raft log should be limited to whatever Raft log truncation dictates (default 4MiB).
While each follower in a range maintains a Raft log, control of what gets written to the Raft log is in the hands of the leader. No leader and there are no Raft log writes. What happens if there is a leader but the Raft log entries it proposes are never getting applied? This can happen when the range is below quorum. Prior to 2.0, the Raft
CheckQuorum
mechanism would kick in and the leader would quickly step down. In 2.0 we turned onPreVote
and disabledCheckQuorum
(@bdarnell are they incompatible?). WithCheckQuorum
disabled andPreVote
enabled a leader can remain the leader forever when the range is below quorum. ThePreVote
mechanism prevents another range from calling an election and withoutCheckQuorum
enabled the leader won't step down.Note that the proposal quota mechanism still applies in this scenario and incoming operations will eventually block waiting for quota. There is another mechanism at work. The periodic Raft proposal refresh (which is necessary to deal with dropped proposals) interacts badly with this scenario. In particular, we refresh pending proposals every leader election timeout period (3s?). Refreshing a proposal only results in reproposing if the lease index is still compatible, but if a range is below quorum no other commands will be applied so it seems like we would see reproposals.
The above is a theory based on a reading of the code. It matches the conditions in the clusters that have experienced problems with large Raft logs (i.e. ranges that have gone through long periods of unavailability). @nvanbenschoten is going to work on writing a test to reproduce which should be straightforward to do if the above is correct.
Fixing this problem shouldn't be too difficult. @bdarnell says:
and
The text was updated successfully, but these errors were encountered: