mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raft: proactively probe newly added followers
When the leader applied a new configuration that added voters, it would not immediately probe these voters, delaying when they would be caught up. I noticed this while writing an interaction-driven test, which has now been cleaned up and completed.
This commit is contained in:
12
raft/raft.go
12
raft/raft.go
@@ -1527,10 +1527,18 @@ func (r *raft) switchToConfig(cfg tracker.Config, prs tracker.ProgressMap) pb.Co
|
||||
if r.state != StateLeader || len(cs.Voters) == 0 {
|
||||
return cs
|
||||
}
|
||||
|
||||
if r.maybeCommit() {
|
||||
// The quorum size may have been reduced (but not to zero), so see if
|
||||
// any pending entries can be committed.
|
||||
// If the configuration change means that more entries are committed now,
|
||||
// broadcast/append to everyone in the updated config.
|
||||
r.bcastAppend()
|
||||
} else {
|
||||
// Otherwise, still probe the newly added replicas; there's no reason to
|
||||
// let them wait out a heartbeat interval (or the next incoming
|
||||
// proposal).
|
||||
r.prs.Visit(func(id uint64, pr *tracker.Progress) {
|
||||
r.maybeSendAppend(id, false /* sendIfEmpty */)
|
||||
})
|
||||
}
|
||||
// If the the leadTransferee was removed, abort the leadership transfer.
|
||||
if _, tOK := r.prs.Progress[r.leadTransferee]; !tOK && r.leadTransferee != 0 {
|
||||
|
||||
Reference in New Issue
Block a user