-
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: fix bug in unbounded log growth prevention mechanism #10199
Conversation
[APPROVALNOTIFIER] This PR is NOT APPROVED This pull-request has been approved by: If they are not already assigned, you can assign the PR to them by writing The full list of commands accepted by this bot can be found here. The pull request process is described here
Needs approval from an approver in each of these files:
Approvers can indicate their approval by writing |
490ed06
to
ea5ff8a
Compare
Please ignore above comments, while we are experimenting Prow integration. /ok-to-test |
Looks good to me as well, although it' kind of strange that the size limit imposed by An alternative you could explore is pushing this uncommitted size limit into |
Codecov Report
@@ Coverage Diff @@
## master #10199 +/- ##
==========================================
+ Coverage 71.85% 71.87% +0.01%
==========================================
Files 390 390
Lines 36346 36348 +2
==========================================
+ Hits 26116 26124 +8
+ Misses 8424 8417 -7
- Partials 1806 1807 +1
Continue to review full report at Codecov.
|
ea5ff8a
to
3c480ce
Compare
I pushed the computation down into |
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
3c480ce
to
ad49c8f
Compare
Various tests failed, but I'm not sure that's really caused by this PR. @gyuho what do you think? |
(I'm going to re-run just to get an idea of whether they're flakes or not) |
@gyuho I re-triggered the build but it's still red. |
https://status.github.com/messages
Seems like github is just back. Will retrigger manually. |
I actually meant that it failed again :-) I triggered directly on travis-ci. Let's see what the next run brings. |
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.
LGTM. etcd is experiencing some flaky tests not related to this change. Should be good to merge. /cc @xiang90
Thanks @gyuho, appreciate the heads up. |
Pick up etcd-io/etcd#10199.
31554: exec: initial commit of execgen tool r=solongordon a=solongordon Execgen will be our tool for generating templated code necessary for columnarized execution. So far it only generates the EncDatumRowsToColVec function, which is used by the columnarizer to convert a RowSource into a columnarized Operator. Release note: None 31610: sql: fix pg_catalog.pg_constraint's confkey column r=BramGruneir a=BramGruneir Prior to this patch, all columns in the index were included instead of only the ones being used in the foreign key reference. Fixes #31545. Release note (bug fix): Fix pg_catalog.pg_constraint's confkey column from including columns that were not involved in the foreign key reference. 31689: storage: pick up fix for Raft uncommitted entry size tracking r=benesch a=tschottdorf Waiting for the upstream PR etcd-io/etcd#10199 to merge, but this is going to be what the result will look like. ---- The tracking of the uncommitted portion of the log had a bug where it wasn't releasing everything as it should've. As a result, over time, all proposals would be dropped. We're hitting this way earlier in our import tests, which propose large proposals. As an intentional implementation detail, a proposal that itself exceeds the max uncommitted log size is allowed only if the uncommitted log is empty. Due to the leak, we weren't ever hitting this case and so AddSSTable commands were often dropped indefinitely. Fixes #31184. Fixes #28693. Fixes #31642. Optimistically: Fixes #31675. Fixes #31654. Fixes #31446. Release note: None Co-authored-by: Solon Gordon <solon@cockroachlabs.com> Co-authored-by: Bram Gruneir <bram@cockroachlabs.com> Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Picks up etcd-io/etcd#10199.
Picks up the bug fix etcd-io/etcd#10199. It also picks up these further small fixes (which is intentional, I authored most of them): 5c209d6 raft: ensure leader is in ProgressStateReplicate 1569f48 raft: print RejectHint of zero on MsgAppResp 9668536 raft: add a test case in TestStorageAppend e4af2be raft: separate MaxCommittedSizePerReady config from MaxSizePerMsg aa4313a *: fix github links 4aa72ca raft: Explain ReportSnapshot and Propose behavior b7ed416 raft: fix godoc in tests 10255cf raft: Fix comment on TestLeaderBcastBeat c561f83 OWNERS: experiment ad49c8f raft: fix bug in unbounded log growth prevention mechanism de47099 raft: fix description in UT 73c20cc raft: Fix comment on sendHeartbeat 7be7ac5 raft: Fix spelling in doc.go Release note: None
The previous code was using the proto-generated
Size()
method totrack the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to
Size()
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.
As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).
Track only the size of the payload actually appended to the Raft log
instead.
For context, see:
cockroachdb/cockroach#31618 (comment)