-
Notifications
You must be signed in to change notification settings - Fork 9.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
raft: provide protection against unbounded Raft log growth #10167
raft: provide protection against unbounded Raft log growth #10167
Conversation
raft/raft.go
Outdated
// MaxUncommittedEntries 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. Note: 0 for no limit. | ||
MaxUncommittedEntries int |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think I'd rather make this about the total size of the entries instead of just the count. (Maybe with a provision that at least one uncommitted entry will always be allowed regardless of its size)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Basing this off the aggregate entry size is tricky. It means we'll need to keep track of the uncommitted entry size as entries are proposed and committed and probably also compute this on leadership changes, which would effectively undo #9073.
Also, for what it's worth, other flow control mechanisms like MaxInflightMsgs
operate off message count instead of message size, so there is precendence for this approach.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Well, if the size of the log were bounded, #9073 wouldn't have been necessary (or at least not as critical). Also, an approximation here is reasonable: only track the uncommitted log size within a single term; assume that the uncommitted size is zero on becoming leader.
I don't feel too strongly about this given the precedent of MaxInflightMsgs, but if I could go back in time I think I'd change that one to count bytes too.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I made the change, but I don't know how I feel about it. I think it's actually harder to pick a reasonable limit based on size than it is is based on entry count. We want the limit to catch runaway proposals without ever affecting normal operation. What size do we set this to to ensure that? A multiple of the max proposal size? An arbitrary large value like 128MB? If we do that then the log can still grow to millions of small entries. A smaller value is no better because then we start dropping large proposals pretty quickly.
Maybe the argument is that a million small entries is fine as long as they remain under a certain byte size.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, my argument is that a million small entries are fine.
You haven't pushed your branch with this change.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done. PTAL.
raft/raft_test.go
Outdated
r.becomeLeader() | ||
|
||
// Set the two followers to the replicate state. Commit to tail of log. | ||
const followers = 2 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
s/followers/numFollowers/
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
It might be a good idea to make a separate error for this (in retrospect ErrProposalDropped should have been a type with a |
11f4f21
to
5e7d14c
Compare
Codecov Report
@@ Coverage Diff @@
## master #10167 +/- ##
==========================================
+ Coverage 71.54% 71.7% +0.15%
==========================================
Files 390 390
Lines 36319 36346 +27
==========================================
+ Hits 25986 26063 +77
+ Misses 8518 8472 -46
+ Partials 1815 1811 -4
Continue to review full report at Codecov.
|
5e7d14c
to
4470b3d
Compare
I don't feel particularly strongly about this, but for what it's worth there are already 5 cases that use |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
raft/raft_test.go
Outdated
nw.drop(one, other, 2.0) // always drop | ||
nw.drop(other, one, 2.0) // always drop | ||
nw.drop(one, other, 1.0) // always drop | ||
nw.drop(other, one, 1.0) // always drop |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we change this?
raft/raft_test.go:4201:42: (*network).drop - perc always receives 1.0 (1)
Seems like now we only use 1.0
for nw.drop
(CI was complaining :0)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It was a fix for what I thought was a typo. Dropping 200% of messages doesn't really make sense.
I didn't think it would trigger a lint warning. Reverted.
4470b3d
to
8276a9f
Compare
// entries that may be appended to a leader's log. Once this limit is | ||
// exceeded, proposals will begin to return ErrProposalDropped errors. | ||
// Note: 0 for no limit. | ||
MaxUncommittedEntriesSize uint64 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what is the unit of the size?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
@nvanbenschoten LGTM. After fixing the nit. |
@nvanbenschoten we also need to update https://github.com/etcd-io/etcd/tree/master/raft#features to include this feature. |
8276a9f
to
697dddf
Compare
Done. Thanks for the reviews. |
LGTM. Will merge when CI passes. |
CI looks flaky. Anything you need me to do? |
raft/raft.go
Outdated
@@ -215,6 +220,12 @@ func (c *Config) validate() error { | |||
return errors.New("storage cannot be nil") | |||
} | |||
|
|||
if c.MaxUncommittedEntriesSize < 0 { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@nvanbenschoten CI failed due to this checking.
raft/raft.go:223:5: unsigned values are never < 0 (SA4003)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed. CI is passing now.
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
697dddf
to
f894f32
Compare
f894f32
to
73c20cc
Compare
This picks up etcd-io/etcd#10167.
Picks up etcd-io/etcd#10167. Future commits will use the new setting to replace broken logic that prevented unbounded Raft log growth. This also picks up etcd-io/etcd#10063. Release note: None
…rotection This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log.
This picks up etcd-io/etcd#10167.
Picks up etcd-io/etcd#10167. Future commits will use the new setting to replace broken logic that prevented unbounded Raft log growth. This also picks up etcd-io/etcd#10063. Release note: None
…rotection This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log.
31408: storage: replace remote proposal tracking with uncommitted log size protection r=nvanbenschoten a=nvanbenschoten Closes #30064. This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log. This will be backported, but not to 2.1.0. The plan is to get it into 2.1.1. Co-authored-by: Nathan VanBenschoten <[email protected]>
Picks up etcd-io/etcd#10167. Future commits will use the new setting to replace broken logic that prevented unbounded Raft log growth. This also picks up etcd-io/etcd#10063. Release note: None
…rotection This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log.
This picks up etcd-io/etcd#10167.
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