-
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: Introduce CommittedEntries pagination #9982
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.
|
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 { |
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.
maybe provide another function newLogWithSize
, so we don't need to change origin newLog in many places?
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.
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.
lgtm. defer to @siddontang |
Any more comments @siddontang ? |
LGTM |
Picks up etcd-io/etcd#9982 and etcd-io/etcd#9985 (and no other changes to packages we use).
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.
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 <[email protected]>
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)
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)
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.