-
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: Make flow control more aggressive #9985
Conversation
Codecov Report
@@ Coverage Diff @@
## master #9985 +/- ##
==========================================
- Coverage 69.34% 69.11% -0.23%
==========================================
Files 386 386
Lines 35914 35918 +4
==========================================
- Hits 24905 24826 -79
- Misses 9212 9297 +85
+ Partials 1797 1795 -2
Continue to review full report at Codecov.
|
raft/raft.go
Outdated
// argument controls whether messages with no entries will be sent | ||
// ("empty" messages are useful to convey updated Commit indexes, but | ||
// are undesirable when we're sending multiple messages in a batch). | ||
func (r *raft) sendAppend(to uint64, sendIfEmpty bool) bool { |
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.
can we keep the old sendAppend? and add a new func maybeSendAppend (or sendAppendIfNotEmpty)?
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.
@bdarnell Have you tried this in CockroachDB setup? |
LGTM after fixing the nit. |
We allow multiple in-flight append messages, but prior to this change the only way we'd ever send them is if there is a steady stream of new proposals. Catching up a follower that is far behind would be unnecessarily slow (this is exacerbated by a quirk of CockroachDB's use of raft which limits our ability to catch up via snapshot in some cases). See cockroachdb/cockroach#27983
Yes, we've tested it. We had a node that had accumulated a multi-gigabyte raft log. Without this change it was going to take weeks to catch up, and with this it caught up in hours. |
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]>
Hi @benbjohnson Why don't you use snapshot here? seem your Range size is 64 MB, the total log size has exceeded this threshold too much. |
Signed-off-by: Jay Lee <[email protected]>
The patch is to speed up log replication when a node is way behind than leader and logs are not compacted yet. Signed-off-by: Jay Lee <[email protected]>
The patch is to speed up log replication when a node is way behind than leader and logs are not compacted yet. Signed-off-by: Jay Lee <[email protected]>
The patch is to speed up log replication when a node is way behind than leader and logs are not compacted yet. Signed-off-by: Jay Lee <[email protected]>
The patch is to speed up log replication when a node is way behind than leader and logs are not compacted yet. Signed-off-by: Jay Lee <[email protected]>
We allow multiple in-flight append messages, but prior to this change
the only way we'd ever send them is if there is a steady stream of new
proposals. Catching up a follower that is far behind would be
unnecessarily slow (this is exacerbated by a quirk of CockroachDB's
use of raft which limits our ability to catch up via snapshot in some
cases).
See cockroachdb/cockroach#27983
Please read https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md#contribution-flow.