Skip to content
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

Merged

Conversation

nvanbenschoten
Copy link
Contributor

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

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
Copy link
Contributor

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)

Copy link
Contributor Author

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.

Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Contributor

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. PTAL.

r.becomeLeader()

// Set the two followers to the replicate state. Commit to tail of log.
const followers = 2
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/followers/numFollowers/

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

@bdarnell
Copy link
Contributor

bdarnell commented Oct 9, 2018

It might be a good idea to make a separate error for this (in retrospect ErrProposalDropped should have been a type with a reason field instead of a singleton). The ideal retry behavior depends on whether the proposal was dropped because of no leader or because the log was full.

@codecov-io
Copy link

codecov-io commented Oct 9, 2018

Codecov Report

Merging #10167 into master will increase coverage by 0.15%.
The diff coverage is 100%.

Impacted file tree graph

@@            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
Impacted Files Coverage Δ
raft/rawnode.go 68.59% <100%> (+0.26%) ⬆️
raft/node.go 92.74% <100%> (+1.56%) ⬆️
raft/raft.go 91.63% <100%> (+0.28%) ⬆️
pkg/logutil/zap_grpc.go 47.61% <0%> (-4.77%) ⬇️
etcdserver/api/v3rpc/watch.go 80.06% <0%> (-3.93%) ⬇️
pkg/testutil/recorder.go 77.77% <0%> (-3.71%) ⬇️
proxy/grpcproxy/watch.go 88.55% <0%> (-3.02%) ⬇️
clientv3/concurrency/election.go 79.52% <0%> (-2.37%) ⬇️
clientv3/watch.go 90.25% <0%> (-1.91%) ⬇️
etcdserver/raft.go 78.67% <0%> (-1.43%) ⬇️
... and 19 more

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 3c6c05b...73c20cc. Read the comment docs.

@nvanbenschoten
Copy link
Contributor Author

It might be a good idea to make a separate error for this (in retrospect ErrProposalDropped should have been a type with a reason field instead of a singleton). The ideal retry behavior depends on whether the proposal was dropped because of no leader or because the log was full.

I don't feel particularly strongly about this, but for what it's worth there are already 5 cases that use ErrProposalDropped and not all of them have to do with an unknown leader. The error description is certainly broad enough to apply here.

Copy link
Contributor

@bdarnell bdarnell left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

I'm not sure what's going on in that travis-ci failure. @xiang90 or @gyuho, any thoughts?

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
Copy link
Contributor

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)

Copy link
Contributor Author

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.

// 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
Copy link
Contributor

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

@xiang90
Copy link
Contributor

xiang90 commented Oct 12, 2018

@nvanbenschoten LGTM. After fixing the nit.

@xiang90
Copy link
Contributor

xiang90 commented Oct 12, 2018

@nvanbenschoten we also need to update https://github.com/etcd-io/etcd/tree/master/raft#features to include this feature.

@nvanbenschoten
Copy link
Contributor Author

we also need to update https://github.com/etcd-io/etcd/tree/master/raft#features to include this feature.

Done. Thanks for the reviews.

@xiang90
Copy link
Contributor

xiang90 commented Oct 12, 2018

LGTM. Will merge when CI passes.

@nvanbenschoten
Copy link
Contributor Author

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 {
Copy link
Contributor

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)

Copy link
Contributor Author

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
@xiang90 xiang90 merged commit dac8c6f into etcd-io:master Oct 14, 2018
nvanbenschoten added a commit to cockroachdb/vendored that referenced this pull request Oct 14, 2018
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Oct 15, 2018
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
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Oct 16, 2018
…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.
nvanbenschoten added a commit to cockroachdb/vendored that referenced this pull request Oct 17, 2018
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Oct 17, 2018
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
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Oct 17, 2018
…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.
craig bot pushed a commit to cockroachdb/cockroach that referenced this pull request Oct 17, 2018
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]>
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Nov 26, 2018
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
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this pull request Nov 26, 2018
…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.
nvanbenschoten added a commit to cockroachdb/vendored that referenced this pull request Nov 26, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Development

Successfully merging this pull request may close these issues.

5 participants