raft: Introduce CommittedEntries pagination#9982
Merged
xiang90 merged 2 commits intoetcd-io:masterfrom Aug 11, 2018
Merged
Conversation
Ensure that this limit is respected when generating MsgApp messages.
Codecov Report
@@ Coverage Diff @@
## master #9982 +/- ##
==========================================
+ Coverage 69.34% 69.35% +<.01%
==========================================
Files 386 386
Lines 35914 35919 +5
==========================================
+ Hits 24905 24912 +7
+ Misses 9212 9209 -3
- Partials 1797 1798 +1
Continue to review full report at Codecov.
|
siddontang
reviewed
Aug 7, 2018
raft/log.go
Outdated
| // newLog returns log using the given storage. It recovers the log to the state | ||
| // that it just commits and applies the latest snapshot. | ||
| func newLog(storage Storage, logger Logger) *raftLog { | ||
| func newLog(storage Storage, logger Logger, maxMsgSize uint64) *raftLog { |
Contributor
There was a problem hiding this comment.
maybe provide another function newLogWithSize, so we don't need to change origin newLog in many places?
The MaxSizePerMsg setting is now used to limit the size of Ready.CommittedEntries. This prevents out-of-memory errors if the raft log has become very large and commits all at once.
Contributor
|
lgtm. defer to @siddontang |
Contributor
Author
|
Any more comments @siddontang ? |
Contributor
|
LGTM |
siddontang
approved these changes
Aug 10, 2018
bdarnell
added a commit
to cockroachdb/vendored
that referenced
this pull request
Aug 13, 2018
Picks up etcd-io/etcd#9982 and etcd-io/etcd#9985 (and no other changes to packages we use).
bdarnell
added a commit
to bdarnell/cockroach
that referenced
this pull request
Aug 13, 2018
Picks up etcd-io/etcd#9982 and etcd-io/etcd#9985 (and no other changes to packages we use). Fixes cockroachdb#27983 Fixes cockroachdb#27804 Release note (bug fix): Additional fixes for out-of-memory errors caused by very large raft logs. Release note (performance improvement): Greatly improved performance when catching up followers that are behind when raft logs are large.
craig bot
pushed a commit
to cockroachdb/cockroach
that referenced
this pull request
Aug 13, 2018
28511: vendor: Update etcd r=tschottdorf a=bdarnell Picks up etcd-io/etcd#9982 and etcd-io/etcd#9985 (and no other changes to packages we use). Fixes #27983 Fixes #27804 Release note (bug fix): Additional fixes for out-of-memory errors caused by very large raft logs. Release note (performance improvement): Greatly improved performance when catching up followers that are behind when raft logs are large. Co-authored-by: Ben Darnell <ben@bendarnell.com>
tbg
added a commit
to tbg/etcd
that referenced
this pull request
Sep 4, 2018
In etcd-io#9982, a mechanism to limit the size of `CommittedEntries` was introduced. The way this mechanism works was that it would load applicable entries (passing the max size hint) and would emit a `HardState` whose commit index was truncated to match the limitation applied to the entries. Unfortunately, this was subtly incorrect when the user-provided `Entries` implementation didn't exactly match what Raft uses internally. Depending on whether a `Node` or a `RawNode` was used, this would either lead to regressing the HardState's commit index or outright forgetting to apply entries, respectively. Asking implementers to precisely match the Raft size limitation semantics was considered but looks like a bad idea as it puts correctness squarely in the hands of downstream users. Instead, this PR removes the truncation of `HardState` when limiting is active and tracks the applied index separately. This removes the old paradigm (that the previous code tried to work around) that the client will always apply all the way to the commit index, which isn't true when commit entries are paginated. See [1] for more on the discovery of this bug (CockroachDB's implementation of `Entries` returns one more entry than Raft's when the size limit hits). [1]: cockroachdb/cockroach#28918 (comment)
tbg
added a commit
to tbg/etcd
that referenced
this pull request
Sep 4, 2018
In etcd-io#9982, a mechanism to limit the size of `CommittedEntries` was introduced. The way this mechanism worked was that it would load applicable entries (passing the max size hint) and would emit a `HardState` whose commit index was truncated to match the limitation applied to the entries. Unfortunately, this was subtly incorrect when the user-provided `Entries` implementation didn't exactly match what Raft uses internally. Depending on whether a `Node` or a `RawNode` was used, this would either lead to regressing the HardState's commit index or outright forgetting to apply entries, respectively. Asking implementers to precisely match the Raft size limitation semantics was considered but looks like a bad idea as it puts correctness squarely in the hands of downstream users. Instead, this PR removes the truncation of `HardState` when limiting is active and tracks the applied index separately. This removes the old paradigm (that the previous code tried to work around) that the client will always apply all the way to the commit index, which isn't true when commit entries are paginated. See [1] for more on the discovery of this bug (CockroachDB's implementation of `Entries` returns one more entry than Raft's when the size limit hits). [1]: cockroachdb/cockroach#28918 (comment)
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
The MaxSizePerMsg setting is now used to limit the size of
Ready.CommittedEntries. This prevents out-of-memory errors if the raft
log has become very large and commits all at once.
I just used the same size limit as we have for MsgApp, although I'd be willing to make it a separate setting if you think it would be useful to configure them separately.