@@ -579,24 +579,24 @@ func (r *raft) send(m pb.Message) {
579
579
}
580
580
}
581
581
582
- // sendAppend sends an append RPC with new entries (if any) and the
583
- // current commit index to the given peer.
584
- func (r * raft ) sendAppend (to uint64 ) {
585
- r .maybeSendAppend (to , true )
586
- }
587
-
588
- // maybeSendAppend sends an append RPC with new entries to the given peer,
589
- // if necessary. Returns true if a message was sent. The sendIfEmpty
590
- // argument controls whether messages with no entries will be sent
591
- // ("empty" messages are useful to convey updated Commit indexes, but
592
- // are undesirable when we're sending multiple messages in a batch).
582
+ // maybeSendAppend sends an append RPC with log entries (if any) that are not
583
+ // yet known to be replicated in the given peer's log, as well as the current
584
+ // commit index. Usually it sends a MsgApp message, but in some cases (e.g. the
585
+ // log has been compacted) it can send a MsgSnap.
586
+ //
587
+ // In some cases, the MsgApp message can have zero entries, and yet be sent.
588
+ // When the follower log is not fully up-to-date, we must send a MsgApp
589
+ // periodically so that eventually the flow is either accepted or rejected. Not
590
+ // doing so can result in replication stall, in cases when a MsgApp is dropped.
593
591
//
594
- // TODO(pav-kv): make invocation of maybeSendAppend stateless. The Progress
595
- // struct contains all the state necessary for deciding whether to send a
596
- // message.
597
- func (r * raft ) maybeSendAppend (to uint64 , sendIfEmpty bool ) bool {
592
+ // Returns true if a message was sent, or false otherwise. A message is not sent
593
+ // if the follower log and commit index are up-to-date, the flow is paused (for
594
+ // reasons like in-flight limits), or the message could not be constructed .
595
+ func (r * raft ) maybeSendAppend (to uint64 ) bool {
598
596
pr := r .trk .Progress [to ]
599
- if pr .IsPaused () {
597
+
598
+ last , commit := r .raftLog .lastIndex (), r .raftLog .committed
599
+ if ! pr .ShouldSendMsgApp (last , commit ) {
600
600
return false
601
601
}
602
602
@@ -608,36 +608,26 @@ func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
608
608
return r .maybeSendSnapshot (to , pr )
609
609
}
610
610
611
- var ents []pb.Entry
612
- // In a throttled StateReplicate only send empty MsgApp, to ensure progress.
613
- // Otherwise, if we had a full Inflights and all inflight messages were in
614
- // fact dropped, replication to that follower would stall. Instead, an empty
615
- // MsgApp will eventually reach the follower (heartbeats responses prompt the
616
- // leader to send an append), allowing it to be acked or rejected, both of
617
- // which will clear out Inflights.
618
- if pr .State != tracker .StateReplicate || ! pr .Inflights .Full () {
619
- ents , err = r .raftLog .entries (pr .Next , r .maxMsgSize )
620
- }
621
- if len (ents ) == 0 && ! sendIfEmpty {
622
- return false
623
- }
624
- // TODO(pav-kv): move this check up to where err is returned.
625
- if err != nil { // send a snapshot if we failed to get the entries
626
- return r .maybeSendSnapshot (to , pr )
611
+ var entries []pb.Entry
612
+ if pr .CanSendEntries (last ) {
613
+ if entries , err = r .raftLog .entries (pr .Next , r .maxMsgSize ); err != nil {
614
+ // Send a snapshot if we failed to get the entries.
615
+ return r .maybeSendSnapshot (to , pr )
616
+ }
627
617
}
628
618
629
- // Send the actual MsgApp otherwise , and update the progress accordingly.
619
+ // Send the MsgApp, and update the progress accordingly.
630
620
r .send (pb.Message {
631
621
To : to ,
632
622
Type : pb .MsgApp ,
633
623
Index : prevIndex ,
634
624
LogTerm : prevTerm ,
635
- Entries : ents ,
636
- Commit : r . raftLog . committed ,
625
+ Entries : entries ,
626
+ Commit : commit ,
637
627
Match : pr .Match ,
638
628
})
639
- pr .SentEntries (len (ents ), uint64 (payloadsSize (ents )))
640
- pr .SentCommit (r . raftLog . committed )
629
+ pr .SentEntries (len (entries ), uint64 (payloadsSize (entries )))
630
+ pr .SentCommit (commit )
641
631
return true
642
632
}
643
633
@@ -696,7 +686,7 @@ func (r *raft) bcastAppend() {
696
686
if id == r .id {
697
687
return
698
688
}
699
- r .sendAppend (id )
689
+ r .maybeSendAppend (id )
700
690
})
701
691
}
702
692
@@ -1450,7 +1440,7 @@ func stepLeader(r *raft, m pb.Message) error {
1450
1440
if pr .State == tracker .StateReplicate {
1451
1441
pr .BecomeProbe ()
1452
1442
}
1453
- r .sendAppend (m .From )
1443
+ r .maybeSendAppend (m .From )
1454
1444
}
1455
1445
} else {
1456
1446
// We want to update our tracking if the response updates our
@@ -1486,21 +1476,13 @@ func stepLeader(r *raft, m pb.Message) error {
1486
1476
1487
1477
if r .maybeCommit () {
1488
1478
r .bcastAppend ()
1489
- } else if r .id != m .From && pr .CanBumpCommit (r .raftLog .committed ) {
1490
- // This node may be missing the latest commit index, so send it.
1491
- // NB: this is not strictly necessary because the periodic heartbeat
1492
- // messages deliver commit indices too. However, a message sent now
1493
- // may arrive earlier than the next heartbeat fires.
1494
- r .sendAppend (m .From )
1495
1479
}
1496
- // We've updated flow control information above, which may
1497
- // allow us to send multiple (size-limited) in-flight messages
1498
- // at once (such as when transitioning from probe to
1499
- // replicate, or when freeTo() covers multiple messages). If
1500
- // we have more entries to send, send as many messages as we
1501
- // can (without sending empty messages for the commit index)
1480
+ // We've updated flow control information above, which may allow us to
1481
+ // send multiple (size-limited) in-flight messages at once (such as when
1482
+ // transitioning from probe to replicate, or when freeTo() covers
1483
+ // multiple messages). Send as many messages as we can.
1502
1484
if r .id != m .From {
1503
- for r .maybeSendAppend (m .From , false /* sendIfEmpty */ ) {
1485
+ for r .maybeSendAppend (m .From ) {
1504
1486
}
1505
1487
}
1506
1488
// Transfer leadership is in progress.
@@ -1512,24 +1494,8 @@ func stepLeader(r *raft, m pb.Message) error {
1512
1494
}
1513
1495
case pb .MsgHeartbeatResp :
1514
1496
pr .RecentActive = true
1515
- pr .MsgAppFlowPaused = false
1516
-
1517
- // NB: if the follower is paused (full Inflights), this will still send an
1518
- // empty append, allowing it to recover from situations in which all the
1519
- // messages that filled up Inflights in the first place were dropped. Note
1520
- // also that the outgoing heartbeat already communicated the commit index.
1521
- //
1522
- // If the follower is fully caught up but also in StateProbe (as can happen
1523
- // if ReportUnreachable was called), we also want to send an append (it will
1524
- // be empty) to allow the follower to transition back to StateReplicate once
1525
- // it responds.
1526
- //
1527
- // Note that StateSnapshot typically satisfies pr.Match < lastIndex, but
1528
- // `pr.Paused()` is always true for StateSnapshot, so sendAppend is a
1529
- // no-op.
1530
- if pr .Match < r .raftLog .lastIndex () || pr .State == tracker .StateProbe {
1531
- r .sendAppend (m .From )
1532
- }
1497
+ pr .MsgAppProbesPaused = false
1498
+ r .maybeSendAppend (m .From )
1533
1499
1534
1500
case pb .MsgSnapStatus :
1535
1501
if pr .State != tracker .StateSnapshot {
@@ -1548,7 +1514,7 @@ func stepLeader(r *raft, m pb.Message) error {
1548
1514
// If snapshot finish, wait for the MsgAppResp from the remote node before sending
1549
1515
// out the next MsgApp.
1550
1516
// If snapshot failure, wait for a heartbeat interval before next try
1551
- pr .MsgAppFlowPaused = true
1517
+ pr .MsgAppProbesPaused = true
1552
1518
case pb .MsgUnreachable :
1553
1519
// During optimistic replication, if the remote becomes unreachable,
1554
1520
// there is huge probability that a MsgApp is lost.
@@ -1585,7 +1551,8 @@ func stepLeader(r *raft, m pb.Message) error {
1585
1551
r .sendTimeoutNow (leadTransferee )
1586
1552
r .logger .Infof ("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log" , r .id , leadTransferee , leadTransferee )
1587
1553
} else {
1588
- r .sendAppend (leadTransferee )
1554
+ pr .MsgAppProbesPaused = false
1555
+ r .maybeSendAppend (leadTransferee )
1589
1556
}
1590
1557
}
1591
1558
return nil
@@ -1957,21 +1924,14 @@ func (r *raft) switchToConfig(cfg tracker.Config, trk tracker.ProgressMap) pb.Co
1957
1924
return cs
1958
1925
}
1959
1926
1960
- if r .maybeCommit () {
1961
- // If the configuration change means that more entries are committed now,
1962
- // broadcast/append to everyone in the updated config.
1963
- r .bcastAppend ()
1964
- } else {
1965
- // Otherwise, still probe the newly added replicas; there's no reason to
1966
- // let them wait out a heartbeat interval (or the next incoming
1967
- // proposal).
1968
- r .trk .Visit (func (id uint64 , pr * tracker.Progress ) {
1969
- if id == r .id {
1970
- return
1971
- }
1972
- r .maybeSendAppend (id , false /* sendIfEmpty */ )
1973
- })
1974
- }
1927
+ r .maybeCommit ()
1928
+ // If the configuration change means that more entries are committed now,
1929
+ // broadcast/append to everyone in the updated config.
1930
+ //
1931
+ // Otherwise, still probe the newly added replicas; there's no reason to let
1932
+ // them wait out a heartbeat interval (or the next incoming proposal).
1933
+ r .bcastAppend ()
1934
+
1975
1935
// If the leadTransferee was removed or demoted, abort the leadership transfer.
1976
1936
if _ , tOK := r .trk .Config .Voters .IDs ()[r .leadTransferee ]; ! tOK && r .leadTransferee != 0 {
1977
1937
r .abortLeaderTransfer ()
0 commit comments