Skip to content

Commit 9d42a69

Browse files
committed
tracker: consolidate MsgApp decisions in Progress
This commit consolidates all decision-making about sending append messages into a single maybeSendAppend method. Previously, the behaviour depended on the sendIfEmpty flag which was set/unset depending on the circumstances in which the method is called. This is unnecessary because the Progress struct contains enough information about the leader->follower flow state, so maybeSendAppend can be made stand-alone. Signed-off-by: Pavel Kalinnikov <[email protected]>
1 parent 01d44ed commit 9d42a69

8 files changed

+129
-119
lines changed

doc.go

+4-4
Original file line numberDiff line numberDiff line change
@@ -315,7 +315,7 @@ stale log entries:
315315
rafthttp package.
316316
317317
'MsgApp' contains log entries to replicate. A leader calls bcastAppend,
318-
which calls sendAppend, which sends soon-to-be-replicated logs in 'MsgApp'
318+
which calls maybeSendAppend, which sends soon-to-be-replicated logs in 'MsgApp'
319319
type. When 'MsgApp' is passed to candidate's Step method, candidate reverts
320320
back to follower, because it indicates that there is a valid leader sending
321321
'MsgApp' messages. Candidate and follower respond to this message in
@@ -353,8 +353,8 @@ stale log entries:
353353
354354
'MsgSnap' requests to install a snapshot message. When a node has just
355355
become a leader or the leader receives 'MsgProp' message, it calls
356-
'bcastAppend' method, which then calls 'sendAppend' method to each
357-
follower. In 'sendAppend', if a leader fails to get term or entries,
356+
'bcastAppend' method, which then calls 'maybeSendAppend' method to each
357+
follower. In 'maybeSendAppend', if a leader fails to get term or entries,
358358
the leader requests snapshot by sending 'MsgSnap' type message.
359359
360360
'MsgSnapStatus' tells the result of snapshot install message. When a
@@ -376,7 +376,7 @@ stale log entries:
376376
'MsgHeartbeatResp' is a response to 'MsgHeartbeat'. When 'MsgHeartbeatResp'
377377
is passed to leader's Step method, the leader knows which follower
378378
responded. And only when the leader's last committed index is greater than
379-
follower's Match index, the leader runs 'sendAppend` method.
379+
follower's Match index, the leader runs 'maybeSendAppend` method.
380380
381381
'MsgUnreachable' tells that request(message) wasn't delivered. When
382382
'MsgUnreachable' is passed to leader's Step method, the leader discovers

raft.go

+46-86
Original file line numberDiff line numberDiff line change
@@ -588,24 +588,24 @@ func (r *raft) send(m pb.Message) {
588588
}
589589
}
590590

591-
// sendAppend sends an append RPC with new entries (if any) and the
592-
// current commit index to the given peer.
593-
func (r *raft) sendAppend(to uint64) {
594-
r.maybeSendAppend(to, true)
595-
}
596-
597-
// maybeSendAppend sends an append RPC with new entries to the given peer,
598-
// if necessary. Returns true if a message was sent. The sendIfEmpty
599-
// argument controls whether messages with no entries will be sent
600-
// ("empty" messages are useful to convey updated Commit indexes, but
601-
// are undesirable when we're sending multiple messages in a batch).
591+
// maybeSendAppend sends an append RPC with log entries (if any) that are not
592+
// yet known to be replicated in the given peer's log, as well as the current
593+
// commit index. Usually it sends a MsgApp message, but in some cases (e.g. the
594+
// log has been compacted) it can send a MsgSnap.
595+
//
596+
// In some cases, the MsgApp message can have zero entries, and yet being sent.
597+
// When the follower log is not fully up-to-date, we must send a MsgApp
598+
// periodically so that eventually the flow is either accepted or rejected. Not
599+
// doing so can result in replication stall, in cases when a MsgApp is dropped.
602600
//
603-
// TODO(pav-kv): make invocation of maybeSendAppend stateless. The Progress
604-
// struct contains all the state necessary for deciding whether to send a
605-
// message.
606-
func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
601+
// Returns true if a message was sent, or false otherwise. A message is not sent
602+
// if the follower log and commit index are up-to-date, the flow is paused (for
603+
// reasons like in-flight limits), or the message could not be constructed.
604+
func (r *raft) maybeSendAppend(to uint64) bool {
607605
pr := r.trk.Progress[to]
608-
if pr.IsPaused() {
606+
607+
last, commit := r.raftLog.lastIndex(), r.raftLog.committed
608+
if !pr.ShouldSendMsgApp(last, commit) {
609609
return false
610610
}
611611

@@ -617,35 +617,25 @@ func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
617617
return r.maybeSendSnapshot(to, pr)
618618
}
619619

620-
var ents []pb.Entry
621-
// In a throttled StateReplicate only send empty MsgApp, to ensure progress.
622-
// Otherwise, if we had a full Inflights and all inflight messages were in
623-
// fact dropped, replication to that follower would stall. Instead, an empty
624-
// MsgApp will eventually reach the follower (heartbeats responses prompt the
625-
// leader to send an append), allowing it to be acked or rejected, both of
626-
// which will clear out Inflights.
627-
if pr.State != tracker.StateReplicate || !pr.Inflights.Full() {
628-
ents, err = r.raftLog.entries(pr.Next, r.maxMsgSize)
629-
}
630-
if len(ents) == 0 && !sendIfEmpty {
631-
return false
632-
}
633-
// TODO(pav-kv): move this check up to where err is returned.
634-
if err != nil { // send a snapshot if we failed to get the entries
635-
return r.maybeSendSnapshot(to, pr)
620+
var entries []pb.Entry
621+
if pr.CanSendEntries(last) {
622+
if entries, err = r.raftLog.entries(pr.Next, r.maxMsgSize); err != nil {
623+
// Send a snapshot if we failed to get the entries.
624+
return r.maybeSendSnapshot(to, pr)
625+
}
636626
}
637627

638-
// Send the actual MsgApp otherwise, and update the progress accordingly.
628+
// Send the MsgApp, and update the progress accordingly.
639629
r.send(pb.Message{
640630
To: to,
641631
Type: pb.MsgApp,
642632
Index: prevIndex,
643633
LogTerm: prevTerm,
644-
Entries: ents,
645-
Commit: r.raftLog.committed,
634+
Entries: entries,
635+
Commit: commit,
646636
})
647-
pr.SentEntries(len(ents), uint64(payloadsSize(ents)))
648-
pr.SentCommit(r.raftLog.committed)
637+
pr.SentEntries(len(entries), uint64(payloadsSize(entries)))
638+
pr.SentCommit(commit)
649639
return true
650640
}
651641

@@ -704,7 +694,7 @@ func (r *raft) bcastAppend() {
704694
if id == r.id {
705695
return
706696
}
707-
r.sendAppend(id)
697+
r.maybeSendAppend(id)
708698
})
709699
}
710700

@@ -1482,7 +1472,7 @@ func stepLeader(r *raft, m pb.Message) error {
14821472
if pr.State == tracker.StateReplicate {
14831473
pr.BecomeProbe()
14841474
}
1485-
r.sendAppend(m.From)
1475+
r.maybeSendAppend(m.From)
14861476
}
14871477
} else {
14881478
// We want to update our tracking if the response updates our
@@ -1523,21 +1513,13 @@ func stepLeader(r *raft, m pb.Message) error {
15231513
// to respond to pending read index requests
15241514
releasePendingReadIndexMessages(r)
15251515
r.bcastAppend()
1526-
} else if r.id != m.From && pr.CanBumpCommit(r.raftLog.committed) {
1527-
// This node may be missing the latest commit index, so send it.
1528-
// NB: this is not strictly necessary because the periodic heartbeat
1529-
// messages deliver commit indices too. However, a message sent now
1530-
// may arrive earlier than the next heartbeat fires.
1531-
r.sendAppend(m.From)
15321516
}
1533-
// We've updated flow control information above, which may
1534-
// allow us to send multiple (size-limited) in-flight messages
1535-
// at once (such as when transitioning from probe to
1536-
// replicate, or when freeTo() covers multiple messages). If
1537-
// we have more entries to send, send as many messages as we
1538-
// can (without sending empty messages for the commit index)
1517+
// We've updated flow control information above, which may allow us to
1518+
// send multiple (size-limited) in-flight messages at once (such as when
1519+
// transitioning from probe to replicate, or when freeTo() covers
1520+
// multiple messages). Send as many messages as we can.
15391521
if r.id != m.From {
1540-
for r.maybeSendAppend(m.From, false /* sendIfEmpty */) {
1522+
for r.maybeSendAppend(m.From) {
15411523
}
15421524
}
15431525
// Transfer leadership is in progress.
@@ -1549,24 +1531,8 @@ func stepLeader(r *raft, m pb.Message) error {
15491531
}
15501532
case pb.MsgHeartbeatResp:
15511533
pr.RecentActive = true
1552-
pr.MsgAppFlowPaused = false
1553-
1554-
// NB: if the follower is paused (full Inflights), this will still send an
1555-
// empty append, allowing it to recover from situations in which all the
1556-
// messages that filled up Inflights in the first place were dropped. Note
1557-
// also that the outgoing heartbeat already communicated the commit index.
1558-
//
1559-
// If the follower is fully caught up but also in StateProbe (as can happen
1560-
// if ReportUnreachable was called), we also want to send an append (it will
1561-
// be empty) to allow the follower to transition back to StateReplicate once
1562-
// it responds.
1563-
//
1564-
// Note that StateSnapshot typically satisfies pr.Match < lastIndex, but
1565-
// `pr.Paused()` is always true for StateSnapshot, so sendAppend is a
1566-
// no-op.
1567-
if pr.Match < r.raftLog.lastIndex() || pr.State == tracker.StateProbe {
1568-
r.sendAppend(m.From)
1569-
}
1534+
pr.PauseMsgAppProbes(false)
1535+
r.maybeSendAppend(m.From)
15701536

15711537
if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
15721538
return nil
@@ -1636,7 +1602,8 @@ func stepLeader(r *raft, m pb.Message) error {
16361602
r.sendTimeoutNow(leadTransferee)
16371603
r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
16381604
} else {
1639-
r.sendAppend(leadTransferee)
1605+
pr.PauseMsgAppProbes(false)
1606+
r.maybeSendAppend(leadTransferee)
16401607
}
16411608
}
16421609
return nil
@@ -1984,21 +1951,14 @@ func (r *raft) switchToConfig(cfg tracker.Config, trk tracker.ProgressMap) pb.Co
19841951
return cs
19851952
}
19861953

1987-
if r.maybeCommit() {
1988-
// If the configuration change means that more entries are committed now,
1989-
// broadcast/append to everyone in the updated config.
1990-
r.bcastAppend()
1991-
} else {
1992-
// Otherwise, still probe the newly added replicas; there's no reason to
1993-
// let them wait out a heartbeat interval (or the next incoming
1994-
// proposal).
1995-
r.trk.Visit(func(id uint64, pr *tracker.Progress) {
1996-
if id == r.id {
1997-
return
1998-
}
1999-
r.maybeSendAppend(id, false /* sendIfEmpty */)
2000-
})
2001-
}
1954+
r.maybeCommit()
1955+
// If the configuration change means that more entries are committed now,
1956+
// broadcast/append to everyone in the updated config.
1957+
//
1958+
// Otherwise, still probe the newly added replicas; there's no reason to let
1959+
// them wait out a heartbeat interval (or the next incoming proposal).
1960+
r.bcastAppend()
1961+
20021962
// If the leadTransferee was removed or demoted, abort the leadership transfer.
20031963
if _, tOK := r.trk.Config.Voters.IDs()[r.leadTransferee]; !tOK && r.leadTransferee != 0 {
20041964
r.abortLeaderTransfer()

raft_test.go

+8-8
Original file line numberDiff line numberDiff line change
@@ -141,8 +141,8 @@ func TestProgressResumeByHeartbeatResp(t *testing.T) {
141141
}
142142
r.trk.Progress[2].MsgAppFlowPaused = true
143143
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
144-
if r.trk.Progress[2].MsgAppFlowPaused {
145-
t.Errorf("paused = %v, want false", r.trk.Progress[2].MsgAppFlowPaused)
144+
if !r.trk.Progress[2].MsgAppFlowPaused {
145+
t.Errorf("paused = %v, want true", r.trk.Progress[2].MsgAppFlowPaused)
146146
}
147147
}
148148

@@ -2773,7 +2773,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
27732773
// loop. After that, the follower is paused until a heartbeat response is
27742774
// received.
27752775
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
2776-
r.sendAppend(2)
2776+
r.maybeSendAppend(2)
27772777
msg := r.readMessages()
27782778
if len(msg) != 1 {
27792779
t.Errorf("len(msg) = %d, want %d", len(msg), 1)
@@ -2788,7 +2788,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
27882788
}
27892789
for j := 0; j < 10; j++ {
27902790
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
2791-
r.sendAppend(2)
2791+
r.maybeSendAppend(2)
27922792
if l := len(r.readMessages()); l != 0 {
27932793
t.Errorf("len(msg) = %d, want %d", l, 0)
27942794
}
@@ -2835,7 +2835,7 @@ func TestSendAppendForProgressReplicate(t *testing.T) {
28352835

28362836
for i := 0; i < 10; i++ {
28372837
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
2838-
r.sendAppend(2)
2838+
r.maybeSendAppend(2)
28392839
msgs := r.readMessages()
28402840
if len(msgs) != 1 {
28412841
t.Errorf("len(msg) = %d, want %d", len(msgs), 1)
@@ -2852,7 +2852,7 @@ func TestSendAppendForProgressSnapshot(t *testing.T) {
28522852

28532853
for i := 0; i < 10; i++ {
28542854
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
2855-
r.sendAppend(2)
2855+
r.maybeSendAppend(2)
28562856
msgs := r.readMessages()
28572857
if len(msgs) != 0 {
28582858
t.Errorf("len(msg) = %d, want %d", len(msgs), 0)
@@ -4677,10 +4677,10 @@ func TestLogReplicationWithReorderedMessage(t *testing.T) {
46774677

46784678
// r1 sends 2 MsgApp messages to r2.
46794679
mustAppendEntry(r1, pb.Entry{Data: []byte("somedata")})
4680-
r1.sendAppend(2)
4680+
r1.maybeSendAppend(2)
46814681
req1 := expectOneMessage(t, r1)
46824682
mustAppendEntry(r1, pb.Entry{Data: []byte("somedata")})
4683-
r1.sendAppend(2)
4683+
r1.maybeSendAppend(2)
46844684
req2 := expectOneMessage(t, r1)
46854685

46864686
// r2 receives the second MsgApp first due to reordering.

testdata/replicate_pause.txt

+3
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,9 @@ deliver-msgs drop=3
7676
dropped: 1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1_12"]
7777
dropped: 1->3 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_1_13"]
7878
dropped: 1->3 MsgApp Term:1 Log:1/13 Commit:11 Entries:[1/14 EntryNormal "prop_1_14"]
79+
dropped: 1->3 MsgApp Term:1 Log:1/14 Commit:12
80+
dropped: 1->3 MsgApp Term:1 Log:1/14 Commit:13
81+
dropped: 1->3 MsgApp Term:1 Log:1/14 Commit:14
7982

8083

8184
# Repeat committing 3 entries.

testdata/slow_follower_after_compaction.txt

+2
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,8 @@ deliver-msgs drop=3
8888
----
8989
dropped: 1->3 MsgApp Term:1 Log:1/14 Commit:14 Entries:[1/15 EntryNormal "prop_1_15"]
9090
dropped: 1->3 MsgApp Term:1 Log:1/15 Commit:14 Entries:[1/16 EntryNormal "prop_1_16"]
91+
dropped: 1->3 MsgApp Term:1 Log:1/16 Commit:15
92+
dropped: 1->3 MsgApp Term:1 Log:1/16 Commit:16
9193

9294
# Truncate the leader's log beyond node 3 log size.
9395
compact 1 17

tracker/inflights.go

+2
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,8 @@ type Inflights struct {
3232
count int // number of inflight messages in the buffer
3333
bytes uint64 // number of inflight bytes
3434

35+
// TODO(pav-kv): do not store the limits here, pass them to methods. For flow
36+
// control, we need to support dynamic limits.
3537
size int // the max number of inflight messages
3638
maxBytes uint64 // the max total byte size of inflight messages
3739

0 commit comments

Comments
 (0)