diff --git a/confchange/testdata/joint_autoleave.txt b/confchange/testdata/joint_autoleave.txt index be138df3..a1370986 100644 --- a/confchange/testdata/joint_autoleave.txt +++ b/confchange/testdata/joint_autoleave.txt @@ -5,16 +5,16 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 # Autoleave is reflected in the config. enter-joint autoleave=true v2 v3 ---- voters=(1 2 3)&&(1) autoleave -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 -3: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 +3: StateProbe match=0 commit=0 next=1 # Can't enter-joint twice, even if autoleave changes. enter-joint autoleave=false @@ -24,6 +24,6 @@ config is already joint leave-joint ---- voters=(1 2 3) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 -3: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 +3: StateProbe match=0 commit=0 next=1 diff --git a/confchange/testdata/joint_idempotency.txt b/confchange/testdata/joint_idempotency.txt index a47f3a66..4f329982 100644 --- a/confchange/testdata/joint_idempotency.txt +++ b/confchange/testdata/joint_idempotency.txt @@ -5,19 +5,19 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 enter-joint r1 r2 r9 v2 v3 v4 v2 v3 v4 l2 l2 r4 r4 l1 l1 ---- voters=(3)&&(1) learners=(2) learners_next=(1) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 learner -3: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 learner +3: StateProbe match=0 commit=0 next=1 leave-joint ---- voters=(3) learners=(1 2) -1: StateProbe match=0 next=1 learner -2: StateProbe match=0 next=1 learner -3: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 learner +2: StateProbe match=0 commit=0 next=1 learner +3: StateProbe match=0 commit=0 next=1 diff --git a/confchange/testdata/joint_learners_next.txt b/confchange/testdata/joint_learners_next.txt index 6faddfe7..81a4bef8 100644 --- a/confchange/testdata/joint_learners_next.txt +++ b/confchange/testdata/joint_learners_next.txt @@ -8,17 +8,17 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 enter-joint v2 l1 ---- voters=(2)&&(1) learners_next=(1) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 leave-joint ---- voters=(2) learners=(1) -1: StateProbe match=0 next=1 learner -2: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 learner +2: StateProbe match=0 commit=0 next=1 diff --git a/confchange/testdata/joint_safety.txt b/confchange/testdata/joint_safety.txt index 75d11b19..7c418a8e 100644 --- a/confchange/testdata/joint_safety.txt +++ b/confchange/testdata/joint_safety.txt @@ -15,7 +15,7 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=3 +1: StateProbe match=0 commit=0 next=3 leave-joint ---- @@ -25,7 +25,7 @@ can't leave a non-joint config enter-joint ---- voters=(1)&&(1) -1: StateProbe match=0 next=3 +1: StateProbe match=0 commit=0 next=3 enter-joint ---- @@ -34,7 +34,7 @@ config is already joint leave-joint ---- voters=(1) -1: StateProbe match=0 next=3 +1: StateProbe match=0 commit=0 next=3 leave-joint ---- @@ -45,10 +45,10 @@ enter-joint r1 v2 v3 l4 ---- voters=(2 3)&&(1) learners=(4) -1: StateProbe match=0 next=3 -2: StateProbe match=0 next=9 -3: StateProbe match=0 next=9 -4: StateProbe match=0 next=9 learner +1: StateProbe match=0 commit=0 next=3 +2: StateProbe match=0 commit=0 next=9 +3: StateProbe match=0 commit=0 next=9 +4: StateProbe match=0 commit=0 next=9 learner enter-joint ---- @@ -67,15 +67,15 @@ can't apply simple config change in joint config leave-joint ---- voters=(2 3) learners=(4) -2: StateProbe match=0 next=9 -3: StateProbe match=0 next=9 -4: StateProbe match=0 next=9 learner +2: StateProbe match=0 commit=0 next=9 +3: StateProbe match=0 commit=0 next=9 +4: StateProbe match=0 commit=0 next=9 learner simple l9 ---- voters=(2 3) learners=(4 9) -2: StateProbe match=0 next=9 -3: StateProbe match=0 next=9 -4: StateProbe match=0 next=9 learner -9: StateProbe match=0 next=14 learner +2: StateProbe match=0 commit=0 next=9 +3: StateProbe match=0 commit=0 next=9 +4: StateProbe match=0 commit=0 next=9 learner +9: StateProbe match=0 commit=0 next=14 learner diff --git a/confchange/testdata/simple_idempotency.txt b/confchange/testdata/simple_idempotency.txt index e31c43b7..0eb7e2ba 100644 --- a/confchange/testdata/simple_idempotency.txt +++ b/confchange/testdata/simple_idempotency.txt @@ -2,68 +2,68 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 simple v2 ---- voters=(1 2) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=2 simple l1 ---- voters=(2) learners=(1) -1: StateProbe match=0 next=1 learner -2: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 learner +2: StateProbe match=0 commit=0 next=2 simple l1 ---- voters=(2) learners=(1) -1: StateProbe match=0 next=1 learner -2: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 learner +2: StateProbe match=0 commit=0 next=2 simple r1 ---- voters=(2) -2: StateProbe match=0 next=2 +2: StateProbe match=0 commit=0 next=2 simple r1 ---- voters=(2) -2: StateProbe match=0 next=2 +2: StateProbe match=0 commit=0 next=2 simple v3 ---- voters=(2 3) -2: StateProbe match=0 next=2 -3: StateProbe match=0 next=7 +2: StateProbe match=0 commit=0 next=2 +3: StateProbe match=0 commit=0 next=7 simple r3 ---- voters=(2) -2: StateProbe match=0 next=2 +2: StateProbe match=0 commit=0 next=2 simple r3 ---- voters=(2) -2: StateProbe match=0 next=2 +2: StateProbe match=0 commit=0 next=2 simple r4 ---- voters=(2) -2: StateProbe match=0 next=2 +2: StateProbe match=0 commit=0 next=2 diff --git a/confchange/testdata/simple_promote_demote.txt b/confchange/testdata/simple_promote_demote.txt index b4b770de..5e1eaefc 100644 --- a/confchange/testdata/simple_promote_demote.txt +++ b/confchange/testdata/simple_promote_demote.txt @@ -4,22 +4,22 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 simple v2 ---- voters=(1 2) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 simple v3 ---- voters=(1 2 3) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 -3: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 +3: StateProbe match=0 commit=0 next=2 # Can atomically demote and promote without a hitch. # This is pointless, but possible. @@ -27,18 +27,18 @@ simple l1 v1 ---- voters=(1 2 3) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 -3: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 +3: StateProbe match=0 commit=0 next=2 # Can demote a voter. simple l2 ---- voters=(1 3) learners=(2) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 learner -3: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 learner +3: StateProbe match=0 commit=0 next=2 # Can atomically promote and demote the same voter. # This is pointless, but possible. @@ -46,15 +46,15 @@ simple v2 l2 ---- voters=(1 3) learners=(2) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 learner -3: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 learner +3: StateProbe match=0 commit=0 next=2 # Can promote a voter. simple v2 ---- voters=(1 2 3) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 -3: StateProbe match=0 next=2 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 +3: StateProbe match=0 commit=0 next=2 diff --git a/confchange/testdata/simple_safety.txt b/confchange/testdata/simple_safety.txt index 6566c5fc..42f30350 100644 --- a/confchange/testdata/simple_safety.txt +++ b/confchange/testdata/simple_safety.txt @@ -7,15 +7,15 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 simple v2 l3 ---- voters=(1 2) learners=(3) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=2 -3: StateProbe match=0 next=2 learner +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=2 +3: StateProbe match=0 commit=0 next=2 learner simple r1 v5 @@ -46,11 +46,11 @@ simple l2 l3 l4 l5 ---- voters=(1) learners=(2 3 4 5) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=2 learner -3: StateProbe match=0 next=2 learner -4: StateProbe match=0 next=8 learner -5: StateProbe match=0 next=8 learner +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=2 learner +3: StateProbe match=0 commit=0 next=2 learner +4: StateProbe match=0 commit=0 next=8 learner +5: StateProbe match=0 commit=0 next=8 learner simple r1 @@ -61,4 +61,4 @@ simple r2 r3 r4 r5 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 diff --git a/confchange/testdata/update.txt b/confchange/testdata/update.txt index ac47bf3e..1220480e 100644 --- a/confchange/testdata/update.txt +++ b/confchange/testdata/update.txt @@ -6,18 +6,18 @@ simple v1 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 simple v2 u1 ---- voters=(1 2) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 simple u1 u2 u3 u1 u2 u3 ---- voters=(1 2) -1: StateProbe match=0 next=1 -2: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 +2: StateProbe match=0 commit=0 next=1 diff --git a/confchange/testdata/zero.txt b/confchange/testdata/zero.txt index 226ade08..068e5bec 100644 --- a/confchange/testdata/zero.txt +++ b/confchange/testdata/zero.txt @@ -3,4 +3,4 @@ simple v1 r0 v0 l0 ---- voters=(1) -1: StateProbe match=0 next=1 +1: StateProbe match=0 commit=0 next=1 diff --git a/raft.go b/raft.go index 7f591f26..1b52a163 100644 --- a/raft.go +++ b/raft.go @@ -599,6 +599,10 @@ func (r *raft) sendAppend(to uint64) { // 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). +// +// TODO(pav-kv): make invocation of maybeSendAppend stateless. The Progress +// struct contains all the state necessary for deciding whether to send a +// message. func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool { pr := r.trk.Progress[to] if pr.IsPaused() { @@ -640,7 +644,8 @@ func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool { Entries: ents, Commit: r.raftLog.committed, }) - pr.UpdateOnEntriesSend(len(ents), uint64(payloadsSize(ents))) + pr.SentEntries(len(ents), uint64(payloadsSize(ents))) + pr.SentCommit(r.raftLog.committed) return true } @@ -675,21 +680,21 @@ func (r *raft) maybeSendSnapshot(to uint64, pr *tracker.Progress) bool { // sendHeartbeat sends a heartbeat RPC to the given peer. func (r *raft) sendHeartbeat(to uint64, ctx []byte) { + pr := r.trk.Progress[to] // Attach the commit as min(to.matched, r.committed). // When the leader sends out heartbeat message, // the receiver(follower) might not be matched with the leader // or it might not have all the committed entries. // The leader MUST NOT forward the follower's commit to // an unmatched index. - commit := min(r.trk.Progress[to].Match, r.raftLog.committed) - m := pb.Message{ + commit := min(pr.Match, r.raftLog.committed) + r.send(pb.Message{ To: to, Type: pb.MsgHeartbeat, Commit: commit, Context: ctx, - } - - r.send(m) + }) + pr.SentCommit(commit) } // bcastAppend sends RPC, with entries to all peers that are not up-to-date @@ -822,7 +827,7 @@ func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) { // if r.maybeCommit() { // r.bcastAppend() // } - r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li}) + r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li, Commit: r.raftLog.committed}) return true } @@ -1480,7 +1485,6 @@ func stepLeader(r *raft, m pb.Message) error { r.sendAppend(m.From) } } else { - oldPaused := pr.IsPaused() // We want to update our tracking if the response updates our // matched index or if the response can move a probing peer back // into StateReplicate (see heartbeat_rep_recovers_from_probing.txt @@ -1489,7 +1493,9 @@ func stepLeader(r *raft, m pb.Message) error { // equals pr.Match we know we don't m.Index+1 in our log, so moving // back to replicating state is not useful; besides pr.PendingSnapshot // would prevent it. - if pr.MaybeUpdate(m.Index) || (pr.Match == m.Index && pr.State == tracker.StateProbe) { + updated := pr.MaybeUpdate(m.Index) + pr.UpdateCommit(m.Commit) + if updated || (pr.Match == m.Index && pr.State == tracker.StateProbe) { switch { case pr.State == tracker.StateProbe: pr.BecomeReplicate() @@ -1517,9 +1523,11 @@ func stepLeader(r *raft, m pb.Message) error { // to respond to pending read index requests releasePendingReadIndexMessages(r) r.bcastAppend() - } else if oldPaused { - // If we were paused before, this node may be missing the - // latest commit index, so send it. + } else if r.id != m.From && pr.CanBumpCommit(r.raftLog.committed) { + // This node may be missing the latest commit index, so send it. + // NB: this is not strictly necessary because the periodic heartbeat + // messages deliver commit indices too. However, a message sent now + // may arrive earlier than the next heartbeat fires. r.sendAppend(m.From) } // We've updated flow control information above, which may @@ -1759,12 +1767,12 @@ func (r *raft) handleAppendEntries(m pb.Message) { // message, and validate it before taking any action (e.g. bumping term). a := logSliceFromMsgApp(&m) - if a.prev.index < r.raftLog.committed { - r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed}) + if commit := r.raftLog.committed; a.prev.index < commit { + r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed, Commit: commit}) return } if mlastIndex, ok := r.raftLog.maybeAppend(a, m.Commit); ok { - r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex}) + r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex, Commit: r.raftLog.committed}) return } r.logger.Debugf("%x [logterm: %d, index: %d] rejected MsgApp [logterm: %d, index: %d] from %x", diff --git a/raft_paper_test.go b/raft_paper_test.go index eff31f63..5c72c280 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -28,6 +28,7 @@ package raft import ( "fmt" + "github.com/stretchr/testify/require" "reflect" "sort" "testing" @@ -606,20 +607,21 @@ func TestFollowerCheckMsgApp(t *testing.T) { term uint64 index uint64 windex uint64 + wcommit uint64 wreject bool wrejectHint uint64 wlogterm uint64 }{ // match with committed entries - {0, 0, 1, false, 0, 0}, - {ents[0].Term, ents[0].Index, 1, false, 0, 0}, + {0, 0, 1, 1, false, 0, 0}, + {ents[0].Term, ents[0].Index, 1, 1, false, 0, 0}, // match with uncommitted entries - {ents[1].Term, ents[1].Index, 2, false, 0, 0}, + {ents[1].Term, ents[1].Index, 2, 1, false, 0, 0}, // unmatch with existing entry - {ents[0].Term, ents[1].Index, ents[1].Index, true, 1, 1}, + {ents[0].Term, ents[1].Index, ents[1].Index, 0, true, 1, 1}, // unexisting entry - {ents[1].Term + 1, ents[1].Index + 1, ents[1].Index + 1, true, 2, 2}, + {ents[1].Term + 1, ents[1].Index + 1, ents[1].Index + 1, 0, true, 2, 2}, } for i, tt := range tests { storage := newTestMemoryStorage(withPeers(1, 2, 3)) @@ -630,13 +632,11 @@ func TestFollowerCheckMsgApp(t *testing.T) { r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, LogTerm: tt.term, Index: tt.index}) - msgs := r.readMessages() - wmsgs := []pb.Message{ - {From: 1, To: 2, Type: pb.MsgAppResp, Term: 2, Index: tt.windex, Reject: tt.wreject, RejectHint: tt.wrejectHint, LogTerm: tt.wlogterm}, - } - if !reflect.DeepEqual(msgs, wmsgs) { - t.Errorf("#%d: msgs = %+v, want %+v", i, msgs, wmsgs) + wmsg := pb.Message{ + From: 1, To: 2, Type: pb.MsgAppResp, Term: 2, Index: tt.windex, Commit: tt.wcommit, + Reject: tt.wreject, RejectHint: tt.wrejectHint, LogTerm: tt.wlogterm, } + require.Equal(t, []pb.Message{wmsg}, r.readMessages(), "#%d", i) } } diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt index 02e59e5a..9434e6c0 100644 --- a/testdata/async_storage_writes.txt +++ b/testdata/async_storage_writes.txt @@ -90,7 +90,7 @@ stabilize 1->2 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ - 1->1 MsgAppResp Term:1 Log:0/11 + 1->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 ] > 2 receiving messages @@ -101,7 +101,7 @@ stabilize Processing: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: - 1->1 MsgAppResp Term:1 Log:0/11 + 1->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 > 2 handling Ready Ready MustSync=true: @@ -110,7 +110,7 @@ stabilize 1/11 EntryNormal "" Messages: 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ - 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 ] > 3 handling Ready @@ -120,27 +120,27 @@ stabilize 1/11 EntryNormal "" Messages: 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ - 3->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 ] > 1 receiving messages - 1->1 MsgAppResp Term:1 Log:0/11 + 1->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: - 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: - 3->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 Commit:10 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/11 - 3->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:10 + 3->1 MsgAppResp Term:1 Log:0/11 Commit:10 > 2 receiving messages AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 > 3 receiving messages @@ -177,7 +177,7 @@ stabilize 1/11 EntryNormal "" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses:[ - 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:11 ] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] @@ -189,7 +189,7 @@ stabilize 1/11 EntryNormal "" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses:[ - 3->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 Commit:11 ] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] @@ -200,12 +200,12 @@ stabilize Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses: - 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:11 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses: - 3->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 Commit:11 > 2 processing apply thread Processing: 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] @@ -217,8 +217,8 @@ stabilize Responses: ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/11 - 3->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:11 + 3->1 MsgAppResp Term:1 Log:0/11 Commit:11 > 2 receiving messages ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 3 receiving messages @@ -238,7 +238,7 @@ process-ready 1 2 3 1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] 1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ - 1->1 MsgAppResp Term:1 Log:0/12 + 1->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 ] > 2 handling Ready @@ -261,7 +261,7 @@ process-ready 1 2 3 1/12 EntryNormal "prop_1" Messages: 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ - 2->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 ] > 3 handling Ready @@ -270,7 +270,7 @@ process-ready 1 2 3 1/12 EntryNormal "prop_1" Messages: 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ - 3->1 MsgAppResp Term:1 Log:0/12 + 3->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 ] @@ -288,7 +288,7 @@ process-ready 1 2 3 1->2 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_2"] 1->3 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_2"] 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ - 1->1 MsgAppResp Term:1 Log:0/13 + 1->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 ] > 2 handling Ready @@ -311,7 +311,7 @@ process-ready 1 2 3 1/13 EntryNormal "prop_2" Messages: 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ - 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 ] > 3 handling Ready @@ -320,7 +320,7 @@ process-ready 1 2 3 1/13 EntryNormal "prop_2" Messages: 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ - 3->1 MsgAppResp Term:1 Log:0/13 + 3->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 ] @@ -330,27 +330,27 @@ process-append-thread 1 2 3 Processing: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: - 1->1 MsgAppResp Term:1 Log:0/12 + 1->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: - 2->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: - 3->1 MsgAppResp Term:1 Log:0/12 + 3->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 deliver-msgs 1 2 3 ---- -1->1 MsgAppResp Term:1 Log:0/12 +1->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 -2->1 MsgAppResp Term:1 Log:0/12 -3->1 MsgAppResp Term:1 Log:0/12 +2->1 MsgAppResp Term:1 Log:0/12 Commit:11 +3->1 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 @@ -373,7 +373,7 @@ process-ready 1 2 3 1->2 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] 1->3 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses:[ - 1->1 MsgAppResp Term:1 Log:0/14 + 1->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 ] 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ @@ -404,8 +404,8 @@ process-ready 1 2 3 1/12 EntryNormal "prop_1" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses:[ - 2->1 MsgAppResp Term:1 Log:0/13 - 2->1 MsgAppResp Term:1 Log:0/14 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:12 + 2->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 ] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ @@ -420,8 +420,8 @@ process-ready 1 2 3 1/12 EntryNormal "prop_1" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses:[ - 3->1 MsgAppResp Term:1 Log:0/13 - 3->1 MsgAppResp Term:1 Log:0/14 + 3->1 MsgAppResp Term:1 Log:0/13 Commit:12 + 3->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 ] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ @@ -434,27 +434,27 @@ process-append-thread 1 2 3 Processing: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: - 1->1 MsgAppResp Term:1 Log:0/13 + 1->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: - 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: - 3->1 MsgAppResp Term:1 Log:0/13 + 3->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 deliver-msgs 1 2 3 ---- -1->1 MsgAppResp Term:1 Log:0/13 +1->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 -2->1 MsgAppResp Term:1 Log:0/13 -3->1 MsgAppResp Term:1 Log:0/13 +2->1 MsgAppResp Term:1 Log:0/13 Commit:11 +3->1 MsgAppResp Term:1 Log:0/13 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 @@ -477,7 +477,7 @@ process-ready 1 2 3 1->2 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] 1->3 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses:[ - 1->1 MsgAppResp Term:1 Log:0/15 + 1->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 ] 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ @@ -508,8 +508,8 @@ process-ready 1 2 3 1/13 EntryNormal "prop_2" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses:[ - 2->1 MsgAppResp Term:1 Log:0/14 - 2->1 MsgAppResp Term:1 Log:0/15 + 2->1 MsgAppResp Term:1 Log:0/14 Commit:13 + 2->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 ] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ @@ -524,8 +524,8 @@ process-ready 1 2 3 1/13 EntryNormal "prop_2" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses:[ - 3->1 MsgAppResp Term:1 Log:0/14 - 3->1 MsgAppResp Term:1 Log:0/15 + 3->1 MsgAppResp Term:1 Log:0/14 Commit:13 + 3->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 ] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ @@ -538,21 +538,21 @@ process-append-thread 1 2 3 Processing: 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses: - 1->1 MsgAppResp Term:1 Log:0/14 + 1->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses: - 2->1 MsgAppResp Term:1 Log:0/13 - 2->1 MsgAppResp Term:1 Log:0/14 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:12 + 2->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses: - 3->1 MsgAppResp Term:1 Log:0/13 - 3->1 MsgAppResp Term:1 Log:0/14 + 3->1 MsgAppResp Term:1 Log:0/13 Commit:12 + 3->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 process-apply-thread 1 2 3 @@ -575,12 +575,12 @@ process-apply-thread 1 2 3 deliver-msgs 1 2 3 ---- -1->1 MsgAppResp Term:1 Log:0/14 +1->1 MsgAppResp Term:1 Log:0/14 Commit:12 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 -2->1 MsgAppResp Term:1 Log:0/13 -2->1 MsgAppResp Term:1 Log:0/14 -3->1 MsgAppResp Term:1 Log:0/13 -3->1 MsgAppResp Term:1 Log:0/14 +2->1 MsgAppResp Term:1 Log:0/13 Commit:12 +2->1 MsgAppResp Term:1 Log:0/14 Commit:12 +3->1 MsgAppResp Term:1 Log:0/13 Commit:12 +3->1 MsgAppResp Term:1 Log:0/14 Commit:12 ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] @@ -624,7 +624,7 @@ process-ready 1 2 3 1/14 EntryNormal "prop_3" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[ - 2->1 MsgAppResp Term:1 Log:0/15 + 2->1 MsgAppResp Term:1 Log:0/15 Commit:14 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 ] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ @@ -637,7 +637,7 @@ process-ready 1 2 3 1/14 EntryNormal "prop_3" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[ - 3->1 MsgAppResp Term:1 Log:0/15 + 3->1 MsgAppResp Term:1 Log:0/15 Commit:14 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 ] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ @@ -650,21 +650,21 @@ process-append-thread 1 2 3 Processing: 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses: - 1->1 MsgAppResp Term:1 Log:0/15 + 1->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses: - 2->1 MsgAppResp Term:1 Log:0/14 - 2->1 MsgAppResp Term:1 Log:0/15 + 2->1 MsgAppResp Term:1 Log:0/14 Commit:13 + 2->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses: - 3->1 MsgAppResp Term:1 Log:0/14 - 3->1 MsgAppResp Term:1 Log:0/15 + 3->1 MsgAppResp Term:1 Log:0/14 Commit:13 + 3->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 process-apply-thread 1 2 3 @@ -687,12 +687,12 @@ process-apply-thread 1 2 3 deliver-msgs 1 2 3 ---- -1->1 MsgAppResp Term:1 Log:0/15 +1->1 MsgAppResp Term:1 Log:0/15 Commit:13 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 -2->1 MsgAppResp Term:1 Log:0/14 -2->1 MsgAppResp Term:1 Log:0/15 -3->1 MsgAppResp Term:1 Log:0/14 -3->1 MsgAppResp Term:1 Log:0/15 +2->1 MsgAppResp Term:1 Log:0/14 Commit:13 +2->1 MsgAppResp Term:1 Log:0/15 Commit:13 +3->1 MsgAppResp Term:1 Log:0/14 Commit:13 +3->1 MsgAppResp Term:1 Log:0/15 Commit:13 ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] @@ -734,7 +734,7 @@ process-ready 1 2 3 1/15 EntryNormal "prop_4" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses:[ - 2->1 MsgAppResp Term:1 Log:0/15 + 2->1 MsgAppResp Term:1 Log:0/15 Commit:15 ] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] @@ -746,7 +746,7 @@ process-ready 1 2 3 1/15 EntryNormal "prop_4" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses:[ - 3->1 MsgAppResp Term:1 Log:0/15 + 3->1 MsgAppResp Term:1 Log:0/15 Commit:15 ] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] @@ -758,13 +758,13 @@ process-append-thread 2 3 Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses: - 2->1 MsgAppResp Term:1 Log:0/15 + 2->1 MsgAppResp Term:1 Log:0/15 Commit:14 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses: - 3->1 MsgAppResp Term:1 Log:0/15 + 3->1 MsgAppResp Term:1 Log:0/15 Commit:14 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 process-apply-thread 1 2 3 @@ -787,8 +787,8 @@ process-apply-thread 1 2 3 deliver-msgs 1 2 3 ---- -2->1 MsgAppResp Term:1 Log:0/15 -3->1 MsgAppResp Term:1 Log:0/15 +2->1 MsgAppResp Term:1 Log:0/15 Commit:14 +3->1 MsgAppResp Term:1 Log:0/15 Commit:14 ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 INFO entry at index 15 missing from unstable log; ignoring @@ -812,12 +812,12 @@ process-append-thread 2 3 Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses: - 2->1 MsgAppResp Term:1 Log:0/15 + 2->1 MsgAppResp Term:1 Log:0/15 Commit:15 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses: - 3->1 MsgAppResp Term:1 Log:0/15 + 3->1 MsgAppResp Term:1 Log:0/15 Commit:15 process-apply-thread 1 2 3 ---- @@ -839,8 +839,8 @@ process-apply-thread 1 2 3 deliver-msgs 1 2 3 ---- -2->1 MsgAppResp Term:1 Log:0/15 -3->1 MsgAppResp Term:1 Log:0/15 +2->1 MsgAppResp Term:1 Log:0/15 Commit:15 +3->1 MsgAppResp Term:1 Log:0/15 Commit:15 ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] diff --git a/testdata/async_storage_writes_append_aba_race.txt b/testdata/async_storage_writes_append_aba_race.txt index 83964fe7..f2cbfb02 100644 --- a/testdata/async_storage_writes_append_aba_race.txt +++ b/testdata/async_storage_writes_append_aba_race.txt @@ -43,7 +43,7 @@ Messages: 2->6 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] 2->7 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[ - 2->2 MsgAppResp Term:1 Log:0/12 + 2->2 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 ] @@ -65,7 +65,7 @@ Entries: 1/12 EntryNormal "init_prop" Messages: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[ - 1->2 MsgAppResp Term:1 Log:0/12 + 1->2 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 ] @@ -198,7 +198,7 @@ Messages: 3->6 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] 3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[ - 3->3 MsgAppResp Term:2 Log:0/12 + 3->3 MsgAppResp Term:2 Log:0/12 Commit:11 AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12 ] @@ -229,7 +229,7 @@ Entries: Messages: 1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""] Responses:[ 1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0) - 1->3 MsgAppResp Term:2 Log:0/12 + 1->3 MsgAppResp Term:2 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12 ] @@ -355,7 +355,7 @@ Messages: 4->6 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] 4->7 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] 4->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[ - 4->4 MsgAppResp Term:3 Log:0/12 + 4->4 MsgAppResp Term:3 Log:0/12 Commit:11 AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12 ] @@ -425,7 +425,7 @@ Entries: 3/12 EntryNormal "" Messages: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[ - 1->4 MsgAppResp Term:3 Log:0/12 + 1->4 MsgAppResp Term:3 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 ] @@ -442,7 +442,7 @@ process-append-thread 1 Processing: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses: -1->2 MsgAppResp Term:1 Log:0/12 +1->2 MsgAppResp Term:1 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 raft-log 1 @@ -469,7 +469,7 @@ Processing: 1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""] Responses: 1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0) -1->3 MsgAppResp Term:2 Log:0/12 +1->3 MsgAppResp Term:2 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12 raft-log 1 @@ -504,7 +504,7 @@ process-append-thread 1 Processing: 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses: -1->4 MsgAppResp Term:3 Log:0/12 +1->4 MsgAppResp Term:3 Log:0/12 Commit:11 AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 raft-log 1 diff --git a/testdata/campaign.txt b/testdata/campaign.txt index 4eefab36..3538aa64 100644 --- a/testdata/campaign.txt +++ b/testdata/campaign.txt @@ -76,17 +76,17 @@ stabilize Entries: 1/3 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/3 + 2->1 MsgAppResp Term:1 Log:0/3 Commit:2 > 3 handling Ready Ready MustSync=true: Lead:1 State:StateFollower Entries: 1/3 EntryNormal "" Messages: - 3->1 MsgAppResp Term:1 Log:0/3 + 3->1 MsgAppResp Term:1 Log:0/3 Commit:2 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/3 - 3->1 MsgAppResp Term:1 Log:0/3 + 2->1 MsgAppResp Term:1 Log:0/3 Commit:2 + 3->1 MsgAppResp Term:1 Log:0/3 Commit:2 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:3 @@ -105,14 +105,14 @@ stabilize CommittedEntries: 1/3 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/3 + 2->1 MsgAppResp Term:1 Log:0/3 Commit:3 > 3 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:3 CommittedEntries: 1/3 EntryNormal "" Messages: - 3->1 MsgAppResp Term:1 Log:0/3 + 3->1 MsgAppResp Term:1 Log:0/3 Commit:3 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/3 - 3->1 MsgAppResp Term:1 Log:0/3 + 2->1 MsgAppResp Term:1 Log:0/3 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/3 Commit:3 diff --git a/testdata/campaign_learner_must_vote.txt b/testdata/campaign_learner_must_vote.txt index d05de374..f4d62888 100644 --- a/testdata/campaign_learner_must_vote.txt +++ b/testdata/campaign_learner_must_vote.txt @@ -113,7 +113,7 @@ stabilize 2 3 > 2 receiving messages 3->2 MsgAppResp Term:2 Log:1/4 Rejected (Hint: 3) DEBUG 2 received MsgAppResp(rejected, hint: (index 3, term 1)) from 3 for index 4 - DEBUG 2 decreased progress of 3 to [StateProbe match=0 next=4] + DEBUG 2 decreased progress of 3 to [StateProbe match=0 commit=0 next=4] > 2 handling Ready Ready MustSync=false: Messages: @@ -135,10 +135,10 @@ stabilize 2 3 CommittedEntries: 1/4 EntryConfChangeV2 v3 Messages: - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/5 Commit:4 INFO 3 switched to configuration voters=(1 2 3) > 2 receiving messages - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/5 Commit:4 > 2 handling Ready Ready MustSync=false: HardState Term:2 Vote:2 Commit:5 @@ -154,6 +154,6 @@ stabilize 2 3 CommittedEntries: 2/5 EntryNormal "" Messages: - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/5 Commit:5 > 2 receiving messages - 3->2 MsgAppResp Term:2 Log:0/5 + 3->2 MsgAppResp Term:2 Log:0/5 Commit:5 diff --git a/testdata/checkquorum.txt b/testdata/checkquorum.txt index b25c1e63..065e0754 100644 --- a/testdata/checkquorum.txt +++ b/testdata/checkquorum.txt @@ -193,7 +193,7 @@ stabilize Entries: 3/12 EntryNormal "" Messages: - 1->2 MsgAppResp Term:3 Log:0/12 + 1->2 MsgAppResp Term:3 Log:0/12 Commit:11 > 3 handling Ready Ready MustSync=true: Lead:2 State:StateFollower @@ -201,10 +201,10 @@ stabilize Entries: 3/12 EntryNormal "" Messages: - 3->2 MsgAppResp Term:3 Log:0/12 + 3->2 MsgAppResp Term:3 Log:0/12 Commit:11 > 2 receiving messages - 1->2 MsgAppResp Term:3 Log:0/12 - 3->2 MsgAppResp Term:3 Log:0/12 + 1->2 MsgAppResp Term:3 Log:0/12 Commit:11 + 3->2 MsgAppResp Term:3 Log:0/12 Commit:11 > 2 handling Ready Ready MustSync=false: HardState Term:3 Vote:2 Commit:12 @@ -223,14 +223,14 @@ stabilize CommittedEntries: 3/12 EntryNormal "" Messages: - 1->2 MsgAppResp Term:3 Log:0/12 + 1->2 MsgAppResp Term:3 Log:0/12 Commit:12 > 3 handling Ready Ready MustSync=false: HardState Term:3 Commit:12 CommittedEntries: 3/12 EntryNormal "" Messages: - 3->2 MsgAppResp Term:3 Log:0/12 + 3->2 MsgAppResp Term:3 Log:0/12 Commit:12 > 2 receiving messages - 1->2 MsgAppResp Term:3 Log:0/12 - 3->2 MsgAppResp Term:3 Log:0/12 + 1->2 MsgAppResp Term:3 Log:0/12 Commit:12 + 3->2 MsgAppResp Term:3 Log:0/12 Commit:12 diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index e54a183f..6d300c23 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -70,9 +70,9 @@ stabilize > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 2 for index 3 - DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4] + DEBUG 1 decreased progress of 2 to [StateProbe match=0 commit=0 next=1] + DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 commit=0 next=1] + DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -93,16 +93,4 @@ stabilize 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/4 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4] -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 -> 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 -> 2 handling Ready - Ready MustSync=false: - Messages: - 2->1 MsgAppResp Term:1 Log:0/4 -> 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 commit=0 next=5 paused pendingSnap=4] diff --git a/testdata/confchange_v1_remove_leader.txt b/testdata/confchange_v1_remove_leader.txt index cc91508a..fde99c82 100644 --- a/testdata/confchange_v1_remove_leader.txt +++ b/testdata/confchange_v1_remove_leader.txt @@ -70,8 +70,8 @@ stabilize 2 1/4 EntryConfChange r1 1/5 EntryNormal "foo" Messages: - 2->1 MsgAppResp Term:1 Log:0/4 - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:3 # Put another entry in n1's log. propose 1 bar @@ -92,8 +92,8 @@ stabilize 1 1->2 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"] 1->3 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"] > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:3 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:5 @@ -129,18 +129,18 @@ stabilize 2 1/4 EntryConfChange r1 1/5 EntryNormal "foo" Messages: - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 INFO 2 switched to configuration voters=(2 3) # ... which thankfully is what we see on the leader. stabilize 1 ---- > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 # When n3 responds, quorum is reached and everything falls into place. stabilize @@ -162,18 +162,18 @@ stabilize 1/4 EntryConfChange r1 1/5 EntryNormal "foo" Messages: - 3->1 MsgAppResp Term:1 Log:0/4 - 3->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 INFO 3 switched to configuration voters=(2 3) > 1 receiving messages - 3->1 MsgAppResp Term:1 Log:0/4 - 3->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:6 @@ -192,17 +192,17 @@ stabilize CommittedEntries: 1/6 EntryNormal "bar" Messages: - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:6 > 3 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:6 CommittedEntries: 1/6 EntryNormal "bar" Messages: - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:6 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:6 # However not all is well. n1 is still leader but unconditionally drops all # proposals on the floor, so we're effectively stuck if it still heartbeats diff --git a/testdata/confchange_v1_remove_leader_stepdown.txt b/testdata/confchange_v1_remove_leader_stepdown.txt index fe397650..b9e14ff0 100644 --- a/testdata/confchange_v1_remove_leader_stepdown.txt +++ b/testdata/confchange_v1_remove_leader_stepdown.txt @@ -71,8 +71,8 @@ stabilize 2 1/4 EntryConfChange r1 1/5 EntryNormal "foo" Messages: - 2->1 MsgAppResp Term:1 Log:0/4 - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:3 # Put another entry in n1's log. propose 1 bar @@ -91,8 +91,8 @@ stabilize 1 1->2 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"] 1->3 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"] > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:3 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:5 @@ -132,18 +132,18 @@ stabilize 2 1/4 EntryConfChange r1 1/5 EntryNormal "foo" Messages: - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 INFO 2 switched to configuration voters=(2 3) # ...because the old leader n1 ignores the append responses. stabilize 1 ---- > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 # When n3 responds, quorum is reached and everything falls into place. stabilize @@ -165,18 +165,18 @@ stabilize 1/4 EntryConfChange r1 1/5 EntryNormal "foo" Messages: - 3->1 MsgAppResp Term:1 Log:0/4 - 3->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 INFO 3 switched to configuration voters=(2 3) > 1 receiving messages - 3->1 MsgAppResp Term:1 Log:0/4 - 3->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:4 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 # n1 can no longer propose. propose 1 baz diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index bf1dfcbe..1adf9436 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -93,9 +93,9 @@ stabilize 1 2 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 2 for index 3 - DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4] + DEBUG 1 decreased progress of 2 to [StateProbe match=0 commit=0 next=1] + DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 commit=0 next=1] + DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -116,7 +116,7 @@ stabilize 1 2 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/4 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4] + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -128,9 +128,9 @@ stabilize 1 2 Entries: 1/5 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:5 @@ -147,10 +147,10 @@ stabilize 1 2 CommittedEntries: 1/5 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 INFO 2 switched to configuration voters=(1 2 3) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 # n3 immediately receives a snapshot in the final configuration. stabilize 1 3 @@ -169,9 +169,9 @@ stabilize 1 3 > 1 receiving messages 3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 3 for index 3 - DEBUG 1 decreased progress of 3 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 5] sent snapshot[index: 5, term: 1] to 3 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=1 paused pendingSnap=5] + DEBUG 1 decreased progress of 3 to [StateProbe match=0 commit=0 next=1] + DEBUG 1 [firstindex: 3, commit: 5] sent snapshot[index: 5, term: 1] to 3 [StateProbe match=0 commit=0 next=1] + DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 commit=0 next=6 paused pendingSnap=5] > 1 handling Ready Ready MustSync=false: Messages: @@ -192,19 +192,7 @@ stabilize 1 3 3->1 MsgAppResp Term:1 Log:0/5 > 1 receiving messages 3->1 MsgAppResp Term:1 Log:0/5 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=5 next=6 paused pendingSnap=5] -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->3 MsgApp Term:1 Log:1/5 Commit:5 -> 3 receiving messages - 1->3 MsgApp Term:1 Log:1/5 Commit:5 -> 3 handling Ready - Ready MustSync=false: - Messages: - 3->1 MsgAppResp Term:1 Log:0/5 -> 1 receiving messages - 3->1 MsgAppResp Term:1 Log:0/5 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=5 commit=0 next=6 paused pendingSnap=5] # Nothing else happens. stabilize @@ -242,13 +230,13 @@ stabilize 2 3 Entries: 1/6 EntryConfChangeV2 r2 r3 Messages: - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 > 3 handling Ready Ready MustSync=true: Entries: 1/6 EntryConfChangeV2 r2 r3 Messages: - 3->1 MsgAppResp Term:1 Log:0/6 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 # n1 gets some more proposals. This is part of a regression test: There used to # be a bug in which these proposals would prompt the leader to transition out of @@ -276,8 +264,8 @@ stabilize 1 1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] 1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 3->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:6 @@ -319,10 +307,10 @@ stabilize 2 3 CommittedEntries: 1/6 EntryConfChangeV2 r2 r3 Messages: - 2->1 MsgAppResp Term:1 Log:0/7 - 2->1 MsgAppResp Term:1 Log:0/8 - 2->1 MsgAppResp Term:1 Log:0/8 - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/8 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/8 Commit:6 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:6 INFO 2 switched to configuration voters=(1)&&(1 2 3) autoleave > 3 handling Ready Ready MustSync=true: @@ -334,10 +322,10 @@ stabilize 2 3 CommittedEntries: 1/6 EntryConfChangeV2 r2 r3 Messages: - 3->1 MsgAppResp Term:1 Log:0/7 - 3->1 MsgAppResp Term:1 Log:0/8 - 3->1 MsgAppResp Term:1 Log:0/8 - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/7 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/8 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/8 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:6 INFO 3 switched to configuration voters=(1)&&(1 2 3) autoleave # n2 and n3 also leave the joint config and the dust settles. We see at the very @@ -346,14 +334,14 @@ stabilize 2 3 stabilize ---- > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 - 2->1 MsgAppResp Term:1 Log:0/8 - 2->1 MsgAppResp Term:1 Log:0/8 - 2->1 MsgAppResp Term:1 Log:0/9 - 3->1 MsgAppResp Term:1 Log:0/7 - 3->1 MsgAppResp Term:1 Log:0/8 - 3->1 MsgAppResp Term:1 Log:0/8 - 3->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/8 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/8 Commit:6 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/7 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/8 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/8 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:6 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:9 @@ -385,9 +373,9 @@ stabilize 1/8 EntryNormal "bar" 1/9 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/9 - 2->1 MsgAppResp Term:1 Log:0/9 - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:7 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:8 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:9 INFO 2 switched to configuration voters=(1) > 3 handling Ready Ready MustSync=false: @@ -397,20 +385,20 @@ stabilize 1/8 EntryNormal "bar" 1/9 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/9 - 3->1 MsgAppResp Term:1 Log:0/9 - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:7 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:8 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:9 INFO 3 switched to configuration voters=(1) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:7 raft: cannot step as peer not found - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:8 raft: cannot step as peer not found - 2->1 MsgAppResp Term:1 Log:0/9 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:9 raft: cannot step as peer not found - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:7 raft: cannot step as peer not found - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:8 raft: cannot step as peer not found - 3->1 MsgAppResp Term:1 Log:0/9 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:9 raft: cannot step as peer not found diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index 536d66b8..3c7ffb8e 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -76,9 +76,9 @@ stabilize 1 2 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 2 for index 3 - DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4] + DEBUG 1 decreased progress of 2 to [StateProbe match=0 commit=0 next=1] + DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 commit=0 next=1] + DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -99,7 +99,7 @@ stabilize 1 2 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/4 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4] + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -111,9 +111,9 @@ stabilize 1 2 Entries: 1/5 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:5 @@ -130,7 +130,7 @@ stabilize 1 2 CommittedEntries: 1/5 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 INFO 2 switched to configuration voters=(1 2) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index 1c487da8..489b6485 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -71,9 +71,9 @@ stabilize > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 2 for index 3 - DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4] + DEBUG 1 decreased progress of 2 to [StateProbe match=0 commit=0 next=1] + DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 commit=0 next=1] + DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -94,16 +94,4 @@ stabilize 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/4 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4] -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 -> 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 -> 2 handling Ready - Ready MustSync=false: - Messages: - 2->1 MsgAppResp Term:1 Log:0/4 -> 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 commit=0 next=5 paused pendingSnap=4] diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index 123cd17a..2f2420f8 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -71,9 +71,9 @@ stabilize 1 2 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 2 for index 3 - DEBUG 1 decreased progress of 2 to [StateProbe match=0 next=1] - DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 next=1] - DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 next=1 paused pendingSnap=4] + DEBUG 1 decreased progress of 2 to [StateProbe match=0 commit=0 next=1] + DEBUG 1 [firstindex: 3, commit: 4] sent snapshot[index: 4, term: 1] to 2 [StateProbe match=0 commit=0 next=1] + DEBUG 1 paused sending replication messages to 2 [StateSnapshot match=0 commit=0 next=5 paused pendingSnap=4] > 1 handling Ready Ready MustSync=false: Messages: @@ -94,19 +94,7 @@ stabilize 1 2 2->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/4 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 next=5 paused pendingSnap=4] -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 -> 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 -> 2 handling Ready - Ready MustSync=false: - Messages: - 2->1 MsgAppResp Term:1 Log:0/4 -> 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 2 [StateSnapshot match=4 commit=0 next=5 paused pendingSnap=4] # Check that we're not allowed to change membership again while in the joint state. # This leads to an empty entry being proposed instead (index 5 in the stabilize block @@ -141,11 +129,11 @@ stabilize 1/5 EntryNormal "" 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:4 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:4 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:6 @@ -166,12 +154,12 @@ stabilize 1/5 EntryNormal "" 1/6 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:6 INFO 2 switched to configuration voters=(1 2) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 - 2->1 MsgAppResp Term:1 Log:0/6 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/6 Commit:6 # Check that trying to transition out again won't do anything. propose-conf-change 1 @@ -194,9 +182,9 @@ stabilize Entries: 1/7 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:6 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:6 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:7 @@ -212,6 +200,6 @@ stabilize CommittedEntries: 1/7 EntryNormal "" Messages: - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:7 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:7 diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index ae43ce21..a2b8425c 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -65,16 +65,16 @@ stabilize Entries: 1/4 EntryConfChangeV2 r1 v4 Messages: - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:3 > 3 handling Ready Ready MustSync=true: Entries: 1/4 EntryConfChangeV2 r1 v4 Messages: - 3->1 MsgAppResp Term:1 Log:0/4 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:3 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 - 3->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:3 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:3 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:4 @@ -98,7 +98,7 @@ stabilize CommittedEntries: 1/4 EntryConfChangeV2 r1 v4 Messages: - 2->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:4 INFO 2 switched to configuration voters=(2 3 4)&&(1 2 3) > 3 handling Ready Ready MustSync=false: @@ -106,11 +106,11 @@ stabilize CommittedEntries: 1/4 EntryConfChangeV2 r1 v4 Messages: - 3->1 MsgAppResp Term:1 Log:0/4 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:4 INFO 3 switched to configuration voters=(2 3 4)&&(1 2 3) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/4 - 3->1 MsgAppResp Term:1 Log:0/4 + 2->1 MsgAppResp Term:1 Log:0/4 Commit:4 + 3->1 MsgAppResp Term:1 Log:0/4 Commit:4 > 4 receiving messages 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4] INFO 4 [term: 0] received a MsgApp message with higher term from 1 [term: 1] @@ -143,18 +143,6 @@ stabilize 4->1 MsgAppResp Term:1 Log:0/4 > 1 receiving messages 4->1 MsgAppResp Term:1 Log:0/4 -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->4 MsgApp Term:1 Log:1/4 Commit:4 -> 4 receiving messages - 1->4 MsgApp Term:1 Log:1/4 Commit:4 -> 4 handling Ready - Ready MustSync=false: - Messages: - 4->1 MsgAppResp Term:1 Log:0/4 -> 1 receiving messages - 4->1 MsgAppResp Term:1 Log:0/4 # Transfer leadership while in the joint config. @@ -259,37 +247,35 @@ stabilize Entries: 2/5 EntryNormal "" Messages: - 1->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/5 Commit:4 > 2 handling Ready Ready MustSync=true: Lead:4 State:StateFollower Entries: 2/5 EntryNormal "" Messages: - 2->4 MsgAppResp Term:2 Log:0/5 + 2->4 MsgAppResp Term:2 Log:0/5 Commit:4 > 3 handling Ready Ready MustSync=true: Lead:4 State:StateFollower Entries: 2/5 EntryNormal "" Messages: - 3->4 MsgAppResp Term:2 Log:0/5 + 3->4 MsgAppResp Term:2 Log:0/5 Commit:4 > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/5 - 2->4 MsgAppResp Term:2 Log:0/5 - 3->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/5 Commit:4 + 2->4 MsgAppResp Term:2 Log:0/5 Commit:4 + 3->4 MsgAppResp Term:2 Log:0/5 Commit:4 > 4 handling Ready Ready MustSync=false: HardState Term:2 Vote:4 Commit:5 CommittedEntries: 2/5 EntryNormal "" Messages: - 4->1 MsgApp Term:2 Log:2/5 Commit:4 4->1 MsgApp Term:2 Log:2/5 Commit:5 4->2 MsgApp Term:2 Log:2/5 Commit:5 4->3 MsgApp Term:2 Log:2/5 Commit:5 > 1 receiving messages - 4->1 MsgApp Term:2 Log:2/5 Commit:4 4->1 MsgApp Term:2 Log:2/5 Commit:5 > 2 receiving messages 4->2 MsgApp Term:2 Log:2/5 Commit:5 @@ -301,27 +287,25 @@ stabilize CommittedEntries: 2/5 EntryNormal "" Messages: - 1->4 MsgAppResp Term:2 Log:0/5 - 1->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/5 Commit:5 > 2 handling Ready Ready MustSync=false: HardState Term:2 Vote:4 Commit:5 CommittedEntries: 2/5 EntryNormal "" Messages: - 2->4 MsgAppResp Term:2 Log:0/5 + 2->4 MsgAppResp Term:2 Log:0/5 Commit:5 > 3 handling Ready Ready MustSync=false: HardState Term:2 Vote:4 Commit:5 CommittedEntries: 2/5 EntryNormal "" Messages: - 3->4 MsgAppResp Term:2 Log:0/5 + 3->4 MsgAppResp Term:2 Log:0/5 Commit:5 > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/5 - 1->4 MsgAppResp Term:2 Log:0/5 - 2->4 MsgAppResp Term:2 Log:0/5 - 3->4 MsgAppResp Term:2 Log:0/5 + 1->4 MsgAppResp Term:2 Log:0/5 Commit:5 + 2->4 MsgAppResp Term:2 Log:0/5 Commit:5 + 3->4 MsgAppResp Term:2 Log:0/5 Commit:5 # Leadership transfer succeeded. raft-state @@ -358,23 +342,23 @@ stabilize Entries: 2/6 EntryConfChangeV2 Messages: - 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/6 Commit:5 > 2 handling Ready Ready MustSync=true: Entries: 2/6 EntryConfChangeV2 Messages: - 2->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/6 Commit:5 > 3 handling Ready Ready MustSync=true: Entries: 2/6 EntryConfChangeV2 Messages: - 3->4 MsgAppResp Term:2 Log:0/6 + 3->4 MsgAppResp Term:2 Log:0/6 Commit:5 > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/6 - 2->4 MsgAppResp Term:2 Log:0/6 - 3->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/6 Commit:5 + 2->4 MsgAppResp Term:2 Log:0/6 Commit:5 + 3->4 MsgAppResp Term:2 Log:0/6 Commit:5 > 4 handling Ready Ready MustSync=false: HardState Term:2 Vote:4 Commit:6 @@ -397,7 +381,7 @@ stabilize CommittedEntries: 2/6 EntryConfChangeV2 Messages: - 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/6 Commit:6 INFO 1 switched to configuration voters=(2 3 4) > 2 handling Ready Ready MustSync=false: @@ -405,7 +389,7 @@ stabilize CommittedEntries: 2/6 EntryConfChangeV2 Messages: - 2->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/6 Commit:6 INFO 2 switched to configuration voters=(2 3 4) > 3 handling Ready Ready MustSync=false: @@ -413,13 +397,13 @@ stabilize CommittedEntries: 2/6 EntryConfChangeV2 Messages: - 3->4 MsgAppResp Term:2 Log:0/6 + 3->4 MsgAppResp Term:2 Log:0/6 Commit:6 INFO 3 switched to configuration voters=(2 3 4) > 4 receiving messages - 1->4 MsgAppResp Term:2 Log:0/6 + 1->4 MsgAppResp Term:2 Log:0/6 Commit:6 raft: cannot step as peer not found - 2->4 MsgAppResp Term:2 Log:0/6 - 3->4 MsgAppResp Term:2 Log:0/6 + 2->4 MsgAppResp Term:2 Log:0/6 Commit:6 + 3->4 MsgAppResp Term:2 Log:0/6 Commit:6 # n1 is out of the configuration. raft-state diff --git a/testdata/confchange_v2_replace_leader_stepdown.txt b/testdata/confchange_v2_replace_leader_stepdown.txt index 62d01d23..ecad1d7c 100644 --- a/testdata/confchange_v2_replace_leader_stepdown.txt +++ b/testdata/confchange_v2_replace_leader_stepdown.txt @@ -93,23 +93,23 @@ stabilize Entries: 1/5 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 3 handling Ready Ready MustSync=true: Entries: 1/5 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 4 handling Ready Ready MustSync=true: Entries: 1/5 EntryConfChangeV2 Messages: - 4->1 MsgAppResp Term:1 Log:0/5 + 4->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/5 - 4->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:4 + 4->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:5 @@ -136,7 +136,7 @@ stabilize CommittedEntries: 1/5 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 INFO 2 switched to configuration voters=(2 3 4) > 3 handling Ready Ready MustSync=false: @@ -144,7 +144,7 @@ stabilize CommittedEntries: 1/5 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/5 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:5 INFO 3 switched to configuration voters=(2 3 4) > 4 handling Ready Ready MustSync=false: @@ -152,12 +152,12 @@ stabilize CommittedEntries: 1/5 EntryConfChangeV2 Messages: - 4->1 MsgAppResp Term:1 Log:0/5 + 4->1 MsgAppResp Term:1 Log:0/5 Commit:5 INFO 4 switched to configuration voters=(2 3 4) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/5 - 3->1 MsgAppResp Term:1 Log:0/5 - 4->1 MsgAppResp Term:1 Log:0/5 + 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/5 Commit:5 + 4->1 MsgAppResp Term:1 Log:0/5 Commit:5 # n1 is out of the configuration. raft-state diff --git a/testdata/forget_leader_prevote_checkquorum.txt b/testdata/forget_leader_prevote_checkquorum.txt index 9b3b80ff..d204b9d7 100644 --- a/testdata/forget_leader_prevote_checkquorum.txt +++ b/testdata/forget_leader_prevote_checkquorum.txt @@ -182,7 +182,7 @@ stabilize 2 Entries: 2/13 EntryNormal "prop_1" Messages: - 2->3 MsgAppResp Term:2 Log:0/13 + 2->3 MsgAppResp Term:2 Log:0/13 Commit:12 forget-leader 2 ---- diff --git a/testdata/heartbeat_resp_recovers_from_probing.txt b/testdata/heartbeat_resp_recovers_from_probing.txt index e606a155..e94a67a1 100644 --- a/testdata/heartbeat_resp_recovers_from_probing.txt +++ b/testdata/heartbeat_resp_recovers_from_probing.txt @@ -28,20 +28,20 @@ ok status 1 ---- -1: StateReplicate match=11 next=12 -2: StateReplicate match=11 next=12 -3: StateReplicate match=11 next=12 +1: StateReplicate match=11 commit=10 next=12 +2: StateReplicate match=11 commit=11 next=12 +3: StateReplicate match=11 commit=11 next=12 # On the first replica, report the second one as not reachable. report-unreachable 1 2 ---- -DEBUG 1 failed to send message to 2 because it is unreachable [StateProbe match=11 next=12] +DEBUG 1 failed to send message to 2 because it is unreachable [StateProbe match=11 commit=11 next=12] status 1 ---- -1: StateReplicate match=11 next=12 -2: StateProbe match=11 next=12 -3: StateReplicate match=11 next=12 +1: StateReplicate match=11 commit=10 next=12 +2: StateProbe match=11 commit=11 next=12 +3: StateReplicate match=11 commit=11 next=12 tick-heartbeat 1 ---- @@ -79,12 +79,12 @@ stabilize > 2 handling Ready Ready MustSync=false: Messages: - 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:11 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/11 + 2->1 MsgAppResp Term:1 Log:0/11 Commit:11 status 1 ---- -1: StateReplicate match=11 next=12 -2: StateReplicate match=11 next=12 -3: StateReplicate match=11 next=12 +1: StateReplicate match=11 commit=10 next=12 +2: StateReplicate match=11 commit=11 next=12 +3: StateReplicate match=11 commit=11 next=12 diff --git a/testdata/lagging_commit.txt b/testdata/lagging_commit.txt index 8f8ba336..66880ba4 100644 --- a/testdata/lagging_commit.txt +++ b/testdata/lagging_commit.txt @@ -51,15 +51,15 @@ Entries: 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" Messages: -3->1 MsgAppResp Term:1 Log:0/12 -3->1 MsgAppResp Term:1 Log:0/13 +3->1 MsgAppResp Term:1 Log:0/12 Commit:11 +3->1 MsgAppResp Term:1 Log:0/13 Commit:11 # Suppose there is a network blip which prevents the leader learning that the # follower 3 has appended the proposed entries to the log. deliver-msgs drop=(1) ---- -dropped: 3->1 MsgAppResp Term:1 Log:0/12 -dropped: 3->1 MsgAppResp Term:1 Log:0/13 +dropped: 3->1 MsgAppResp Term:1 Log:0/12 Commit:11 +dropped: 3->1 MsgAppResp Term:1 Log:0/13 Commit:11 # In the meantime, the entries are committed, and the leader sends the commit # index to all the followers. @@ -71,11 +71,11 @@ stabilize 1 2 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" Messages: - 2->1 MsgAppResp Term:1 Log:0/12 - 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/12 Commit:11 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:11 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/12 - 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/12 Commit:11 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:11 > 1 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:13 @@ -97,11 +97,11 @@ stabilize 1 2 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" Messages: - 2->1 MsgAppResp Term:1 Log:0/13 - 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:12 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:13 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/13 - 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:12 + 2->1 MsgAppResp Term:1 Log:0/13 Commit:13 # The network blip prevents the follower 3 from learning that the previously # appended entries are now committed. @@ -114,9 +114,9 @@ dropped: 1->3 MsgApp Term:1 Log:1/13 Commit:13 status 1 ---- -1: StateReplicate match=13 next=14 -2: StateReplicate match=13 next=14 -3: StateReplicate match=11 next=14 inflight=2 +1: StateReplicate match=13 commit=11 next=14 +2: StateReplicate match=13 commit=13 next=14 +3: StateReplicate match=11 commit=11 next=14 inflight=2 # The leader still observes that the entries are in-flight to the follower 3, # since it hasn't heard from it. Nothing triggers updating the follower's @@ -169,6 +169,6 @@ stabilize 1 3 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" Messages: - 3->1 MsgAppResp Term:1 Log:0/13 + 3->1 MsgAppResp Term:1 Log:0/13 Commit:13 > 1 receiving messages - 3->1 MsgAppResp Term:1 Log:0/13 + 3->1 MsgAppResp Term:1 Log:0/13 Commit:13 diff --git a/testdata/prevote.txt b/testdata/prevote.txt index db763d35..f4926d43 100644 --- a/testdata/prevote.txt +++ b/testdata/prevote.txt @@ -49,7 +49,7 @@ Ready MustSync=true: Entries: 1/12 EntryNormal "prop_1" Messages: -2->1 MsgAppResp Term:1 Log:0/12 +2->1 MsgAppResp Term:1 Log:0/12 Commit:11 # 3 is now behind on its log. Attempt to campaign. raft-log 3 @@ -75,7 +75,7 @@ INFO 3 has received 1 MsgPreVoteResp votes and 0 vote rejections deliver-msgs 1 2 ---- -2->1 MsgAppResp Term:1 Log:0/12 +2->1 MsgAppResp Term:1 Log:0/12 Commit:11 3->1 MsgPreVote Term:2 Log:1/11 INFO 1 [logterm: 1, index: 12, vote: 1] rejected MsgPreVote from 3 [logterm: 1, index: 11] at term 1 3->2 MsgPreVote Term:2 Log:1/11 @@ -111,7 +111,7 @@ stabilize CommittedEntries: 1/12 EntryNormal "prop_1" Messages: - 2->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 Commit:12 > 3 handling Ready Ready MustSync=true: Lead:1 State:StateFollower @@ -121,12 +121,12 @@ stabilize CommittedEntries: 1/12 EntryNormal "prop_1" Messages: - 3->1 MsgAppResp Term:1 Log:0/12 - 3->1 MsgAppResp Term:1 Log:0/12 + 3->1 MsgAppResp Term:1 Log:0/12 Commit:11 + 3->1 MsgAppResp Term:1 Log:0/12 Commit:12 > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/12 - 3->1 MsgAppResp Term:1 Log:0/12 - 3->1 MsgAppResp Term:1 Log:0/12 + 2->1 MsgAppResp Term:1 Log:0/12 Commit:12 + 3->1 MsgAppResp Term:1 Log:0/12 Commit:11 + 3->1 MsgAppResp Term:1 Log:0/12 Commit:12 # Let 2 campaign. It should succeed, since it's up-to-date on the log. campaign 2 @@ -223,17 +223,17 @@ stabilize Entries: 2/13 EntryNormal "" Messages: - 1->2 MsgAppResp Term:2 Log:0/13 + 1->2 MsgAppResp Term:2 Log:0/13 Commit:12 > 3 handling Ready Ready MustSync=true: Lead:2 State:StateFollower Entries: 2/13 EntryNormal "" Messages: - 3->2 MsgAppResp Term:2 Log:0/13 + 3->2 MsgAppResp Term:2 Log:0/13 Commit:12 > 2 receiving messages - 1->2 MsgAppResp Term:2 Log:0/13 - 3->2 MsgAppResp Term:2 Log:0/13 + 1->2 MsgAppResp Term:2 Log:0/13 Commit:12 + 3->2 MsgAppResp Term:2 Log:0/13 Commit:12 > 2 handling Ready Ready MustSync=false: HardState Term:2 Vote:2 Commit:13 @@ -252,14 +252,14 @@ stabilize CommittedEntries: 2/13 EntryNormal "" Messages: - 1->2 MsgAppResp Term:2 Log:0/13 + 1->2 MsgAppResp Term:2 Log:0/13 Commit:13 > 3 handling Ready Ready MustSync=false: HardState Term:2 Vote:2 Commit:13 CommittedEntries: 2/13 EntryNormal "" Messages: - 3->2 MsgAppResp Term:2 Log:0/13 + 3->2 MsgAppResp Term:2 Log:0/13 Commit:13 > 2 receiving messages - 1->2 MsgAppResp Term:2 Log:0/13 - 3->2 MsgAppResp Term:2 Log:0/13 + 1->2 MsgAppResp Term:2 Log:0/13 Commit:13 + 3->2 MsgAppResp Term:2 Log:0/13 Commit:13 diff --git a/testdata/prevote_checkquorum.txt b/testdata/prevote_checkquorum.txt index 6db6662b..a7cc57ae 100644 --- a/testdata/prevote_checkquorum.txt +++ b/testdata/prevote_checkquorum.txt @@ -147,17 +147,17 @@ stabilize Entries: 2/12 EntryNormal "" Messages: - 1->3 MsgAppResp Term:2 Log:0/12 + 1->3 MsgAppResp Term:2 Log:0/12 Commit:11 > 2 handling Ready Ready MustSync=true: Lead:3 State:StateFollower Entries: 2/12 EntryNormal "" Messages: - 2->3 MsgAppResp Term:2 Log:0/12 + 2->3 MsgAppResp Term:2 Log:0/12 Commit:11 > 3 receiving messages - 1->3 MsgAppResp Term:2 Log:0/12 - 2->3 MsgAppResp Term:2 Log:0/12 + 1->3 MsgAppResp Term:2 Log:0/12 Commit:11 + 2->3 MsgAppResp Term:2 Log:0/12 Commit:11 > 3 handling Ready Ready MustSync=false: HardState Term:2 Vote:3 Commit:12 @@ -176,17 +176,17 @@ stabilize CommittedEntries: 2/12 EntryNormal "" Messages: - 1->3 MsgAppResp Term:2 Log:0/12 + 1->3 MsgAppResp Term:2 Log:0/12 Commit:12 > 2 handling Ready Ready MustSync=false: HardState Term:2 Vote:3 Commit:12 CommittedEntries: 2/12 EntryNormal "" Messages: - 2->3 MsgAppResp Term:2 Log:0/12 + 2->3 MsgAppResp Term:2 Log:0/12 Commit:12 > 3 receiving messages - 1->3 MsgAppResp Term:2 Log:0/12 - 2->3 MsgAppResp Term:2 Log:0/12 + 1->3 MsgAppResp Term:2 Log:0/12 Commit:12 + 2->3 MsgAppResp Term:2 Log:0/12 Commit:12 # Node 3 is now the leader. Even though the leader is active, nodes 1 and 2 can # still win a prevote and election if they both explicitly campaign, since the @@ -302,7 +302,7 @@ stabilize Entries: 3/13 EntryNormal "" Messages: - 1->2 MsgAppResp Term:3 Log:0/13 + 1->2 MsgAppResp Term:3 Log:0/13 Commit:12 > 3 handling Ready Ready MustSync=true: Lead:2 State:StateFollower @@ -310,10 +310,10 @@ stabilize Entries: 3/13 EntryNormal "" Messages: - 3->2 MsgAppResp Term:3 Log:0/13 + 3->2 MsgAppResp Term:3 Log:0/13 Commit:12 > 2 receiving messages - 1->2 MsgAppResp Term:3 Log:0/13 - 3->2 MsgAppResp Term:3 Log:0/13 + 1->2 MsgAppResp Term:3 Log:0/13 Commit:12 + 3->2 MsgAppResp Term:3 Log:0/13 Commit:12 > 2 handling Ready Ready MustSync=false: HardState Term:3 Vote:2 Commit:13 @@ -332,14 +332,14 @@ stabilize CommittedEntries: 3/13 EntryNormal "" Messages: - 1->2 MsgAppResp Term:3 Log:0/13 + 1->2 MsgAppResp Term:3 Log:0/13 Commit:13 > 3 handling Ready Ready MustSync=false: HardState Term:3 Commit:13 CommittedEntries: 3/13 EntryNormal "" Messages: - 3->2 MsgAppResp Term:3 Log:0/13 + 3->2 MsgAppResp Term:3 Log:0/13 Commit:13 > 2 receiving messages - 1->2 MsgAppResp Term:3 Log:0/13 - 3->2 MsgAppResp Term:3 Log:0/13 + 1->2 MsgAppResp Term:3 Log:0/13 Commit:13 + 3->2 MsgAppResp Term:3 Log:0/13 Commit:13 diff --git a/testdata/probe_and_replicate.txt b/testdata/probe_and_replicate.txt index c4100e97..05f17a13 100644 --- a/testdata/probe_and_replicate.txt +++ b/testdata/probe_and_replicate.txt @@ -510,21 +510,9 @@ stabilize 1 2 6/20 EntryNormal "prop_6_20" 8/21 EntryNormal "" Messages: - 2->1 MsgAppResp Term:8 Log:0/21 + 2->1 MsgAppResp Term:8 Log:0/21 Commit:18 > 1 receiving messages - 2->1 MsgAppResp Term:8 Log:0/21 -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->2 MsgApp Term:8 Log:8/21 Commit:18 -> 2 receiving messages - 1->2 MsgApp Term:8 Log:8/21 Commit:18 -> 2 handling Ready - Ready MustSync=false: - Messages: - 2->1 MsgAppResp Term:8 Log:0/21 -> 1 receiving messages - 2->1 MsgAppResp Term:8 Log:0/21 + 2->1 MsgAppResp Term:8 Log:0/21 Commit:18 stabilize 1 3 ---- @@ -576,21 +564,9 @@ stabilize 1 3 5/17 EntryNormal "prop_5_17" 6/18 EntryNormal "" Messages: - 3->1 MsgAppResp Term:8 Log:0/21 -> 1 receiving messages - 3->1 MsgAppResp Term:8 Log:0/21 -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->3 MsgApp Term:8 Log:8/21 Commit:18 -> 3 receiving messages - 1->3 MsgApp Term:8 Log:8/21 Commit:18 -> 3 handling Ready - Ready MustSync=false: - Messages: - 3->1 MsgAppResp Term:8 Log:0/21 + 3->1 MsgAppResp Term:8 Log:0/21 Commit:18 > 1 receiving messages - 3->1 MsgAppResp Term:8 Log:0/21 + 3->1 MsgAppResp Term:8 Log:0/21 Commit:18 stabilize 1 4 ---- @@ -604,9 +580,9 @@ stabilize 1 4 Entries: 8/21 EntryNormal "" Messages: - 4->1 MsgAppResp Term:8 Log:0/21 + 4->1 MsgAppResp Term:8 Log:0/21 Commit:18 > 1 receiving messages - 4->1 MsgAppResp Term:8 Log:0/21 + 4->1 MsgAppResp Term:8 Log:0/21 Commit:18 > 1 handling Ready Ready MustSync=false: HardState Term:8 Vote:1 Commit:21 @@ -628,9 +604,9 @@ stabilize 1 4 6/20 EntryNormal "prop_6_20" 8/21 EntryNormal "" Messages: - 4->1 MsgAppResp Term:8 Log:0/21 + 4->1 MsgAppResp Term:8 Log:0/21 Commit:21 > 1 receiving messages - 4->1 MsgAppResp Term:8 Log:0/21 + 4->1 MsgAppResp Term:8 Log:0/21 Commit:21 stabilize 1 5 ---- @@ -671,21 +647,9 @@ stabilize 1 5 6/20 EntryNormal "prop_6_20" 8/21 EntryNormal "" Messages: - 5->1 MsgAppResp Term:8 Log:0/21 -> 1 receiving messages - 5->1 MsgAppResp Term:8 Log:0/21 -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->5 MsgApp Term:8 Log:8/21 Commit:21 -> 5 receiving messages - 1->5 MsgApp Term:8 Log:8/21 Commit:21 -> 5 handling Ready - Ready MustSync=false: - Messages: - 5->1 MsgAppResp Term:8 Log:0/21 + 5->1 MsgAppResp Term:8 Log:0/21 Commit:21 > 1 receiving messages - 5->1 MsgAppResp Term:8 Log:0/21 + 5->1 MsgAppResp Term:8 Log:0/21 Commit:21 stabilize 1 6 ---- @@ -738,21 +702,9 @@ stabilize 1 6 6/20 EntryNormal "prop_6_20" 8/21 EntryNormal "" Messages: - 6->1 MsgAppResp Term:8 Log:0/21 -> 1 receiving messages - 6->1 MsgAppResp Term:8 Log:0/21 -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->6 MsgApp Term:8 Log:8/21 Commit:21 -> 6 receiving messages - 1->6 MsgApp Term:8 Log:8/21 Commit:21 -> 6 handling Ready - Ready MustSync=false: - Messages: - 6->1 MsgAppResp Term:8 Log:0/21 + 6->1 MsgAppResp Term:8 Log:0/21 Commit:21 > 1 receiving messages - 6->1 MsgAppResp Term:8 Log:0/21 + 6->1 MsgAppResp Term:8 Log:0/21 Commit:21 stabilize 1 7 ---- @@ -813,18 +765,6 @@ stabilize 1 7 6/20 EntryNormal "prop_6_20" 8/21 EntryNormal "" Messages: - 7->1 MsgAppResp Term:8 Log:0/21 -> 1 receiving messages - 7->1 MsgAppResp Term:8 Log:0/21 -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->7 MsgApp Term:8 Log:8/21 Commit:21 -> 7 receiving messages - 1->7 MsgApp Term:8 Log:8/21 Commit:21 -> 7 handling Ready - Ready MustSync=false: - Messages: - 7->1 MsgAppResp Term:8 Log:0/21 + 7->1 MsgAppResp Term:8 Log:0/21 Commit:21 > 1 receiving messages - 7->1 MsgAppResp Term:8 Log:0/21 + 7->1 MsgAppResp Term:8 Log:0/21 Commit:21 diff --git a/testdata/replicate_pause.txt b/testdata/replicate_pause.txt index d9cee59f..b9a359bf 100644 --- a/testdata/replicate_pause.txt +++ b/testdata/replicate_pause.txt @@ -46,9 +46,9 @@ ok # Expect that in-flight tracking to nodes 2 and 3 is saturated. status 1 ---- -1: StateReplicate match=14 next=15 -2: StateReplicate match=11 next=15 paused inflight=3[full] -3: StateReplicate match=11 next=15 paused inflight=3[full] +1: StateReplicate match=14 commit=11 next=15 +2: StateReplicate match=11 commit=11 next=15 paused inflight=3[full] +3: StateReplicate match=11 commit=11 next=15 paused inflight=3[full] log-level none ---- @@ -66,9 +66,9 @@ ok # Expect that the entries are committed and stored on nodes 1 and 2. status 1 ---- -1: StateReplicate match=14 next=15 -2: StateReplicate match=14 next=15 -3: StateReplicate match=11 next=15 paused inflight=3[full] +1: StateReplicate match=14 commit=11 next=15 +2: StateReplicate match=14 commit=14 next=15 +3: StateReplicate match=11 commit=11 next=15 paused inflight=3[full] # Drop append messages to node 3. deliver-msgs drop=3 @@ -94,9 +94,9 @@ ok # In-flight tracking to nodes 2 and 3 is saturated, but node 3 is behind. status 1 ---- -1: StateReplicate match=14 next=15 -2: StateReplicate match=14 next=18 paused inflight=3[full] -3: StateReplicate match=11 next=15 paused inflight=3[full] +1: StateReplicate match=14 commit=11 next=15 +2: StateReplicate match=14 commit=14 next=18 paused inflight=3[full] +3: StateReplicate match=11 commit=11 next=15 paused inflight=3[full] log-level none ---- @@ -114,9 +114,9 @@ ok # Expect that the entries are committed and stored only on nodes 1 and 2. status 1 ---- -1: StateReplicate match=17 next=18 -2: StateReplicate match=17 next=18 -3: StateReplicate match=11 next=15 paused inflight=3[full] +1: StateReplicate match=17 commit=14 next=18 +2: StateReplicate match=17 commit=17 next=18 +3: StateReplicate match=11 commit=11 next=15 paused inflight=3[full] # Make a heartbeat roundtrip. tick-heartbeat 1 @@ -185,6 +185,6 @@ ok # Eventually all nodes catch up on the committed state. status 1 ---- -1: StateReplicate match=17 next=18 -2: StateReplicate match=17 next=18 -3: StateReplicate match=17 next=18 +1: StateReplicate match=17 commit=14 next=18 +2: StateReplicate match=17 commit=17 next=18 +3: StateReplicate match=17 commit=17 next=18 diff --git a/testdata/slow_follower_after_compaction.txt b/testdata/slow_follower_after_compaction.txt index 0d3d48c8..7d6ad29f 100644 --- a/testdata/slow_follower_after_compaction.txt +++ b/testdata/slow_follower_after_compaction.txt @@ -43,9 +43,9 @@ ok # All nodes up-to-date. status 1 ---- -1: StateReplicate match=14 next=15 -2: StateReplicate match=14 next=15 -3: StateReplicate match=14 next=15 +1: StateReplicate match=14 commit=11 next=15 +2: StateReplicate match=14 commit=14 next=15 +3: StateReplicate match=14 commit=14 next=15 log-level none ---- @@ -79,9 +79,9 @@ ok # Nodes 1 and 2 up-to-date, 3 is behind and MsgApp flow is throttled. status 1 ---- -1: StateReplicate match=18 next=19 -2: StateReplicate match=18 next=19 -3: StateReplicate match=14 next=17 paused inflight=2[full] +1: StateReplicate match=18 commit=14 next=19 +2: StateReplicate match=18 commit=18 next=19 +3: StateReplicate match=14 commit=14 next=17 paused inflight=2[full] # Break the MsgApp flow from the leader to node 3. deliver-msgs drop=3 @@ -116,6 +116,6 @@ ok # All nodes caught up. status 1 ---- -1: StateReplicate match=18 next=19 -2: StateReplicate match=18 next=19 -3: StateReplicate match=18 next=19 +1: StateReplicate match=18 commit=14 next=19 +2: StateReplicate match=18 commit=18 next=19 +3: StateReplicate match=18 commit=14 next=19 diff --git a/testdata/snapshot_succeed_via_app_resp.txt b/testdata/snapshot_succeed_via_app_resp.txt index 5c4b0c61..a015f9de 100644 --- a/testdata/snapshot_succeed_via_app_resp.txt +++ b/testdata/snapshot_succeed_via_app_resp.txt @@ -41,9 +41,9 @@ ok status 1 ---- -1: StateReplicate match=11 next=12 -2: StateReplicate match=11 next=12 -3: StateProbe match=0 next=11 paused inactive +1: StateReplicate match=11 commit=10 next=12 +2: StateReplicate match=11 commit=11 next=12 +3: StateProbe match=0 commit=0 next=11 paused inactive # Add the node that will receive a snapshot (it has no state at all, does not # even have a config). @@ -86,8 +86,8 @@ stabilize 1 ---- > 1 receiving messages 3->1 MsgHeartbeatResp Term:1 Log:0/0 - DEBUG 1 [firstindex: 12, commit: 11] sent snapshot[index: 11, term: 1] to 3 [StateProbe match=0 next=11] - DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=11 paused pendingSnap=11] + DEBUG 1 [firstindex: 12, commit: 11] sent snapshot[index: 11, term: 1] to 3 [StateProbe match=0 commit=0 next=11] + DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 commit=0 next=12 paused pendingSnap=11] > 1 handling Ready Ready MustSync=false: Messages: @@ -96,9 +96,9 @@ stabilize 1 status 1 ---- -1: StateReplicate match=11 next=12 -2: StateReplicate match=11 next=12 -3: StateSnapshot match=0 next=11 paused pendingSnap=11 +1: StateReplicate match=11 commit=10 next=12 +2: StateReplicate match=11 commit=11 next=12 +3: StateSnapshot match=0 commit=0 next=12 paused pendingSnap=11 # Follower applies the snapshot. Note how it reacts with a MsgAppResp upon completion. # The snapshot fully catches the follower up (i.e. there are no more log entries it @@ -126,33 +126,22 @@ stabilize 1 ---- > 1 receiving messages 3->1 MsgAppResp Term:1 Log:0/11 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=11 next=12 paused pendingSnap=11] -> 1 handling Ready - Ready MustSync=false: - Messages: - 1->3 MsgApp Term:1 Log:1/11 Commit:11 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=11 commit=0 next=12 paused pendingSnap=11] status 1 ---- -1: StateReplicate match=11 next=12 -2: StateReplicate match=11 next=12 -3: StateReplicate match=11 next=12 +1: StateReplicate match=11 commit=10 next=12 +2: StateReplicate match=11 commit=11 next=12 +3: StateReplicate match=11 commit=0 next=12 # Let things settle. stabilize ---- > 2 receiving messages 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11 -> 3 receiving messages - 1->3 MsgApp Term:1 Log:1/11 Commit:11 > 2 handling Ready Ready MustSync=false: Messages: 2->1 MsgHeartbeatResp Term:1 Log:0/0 -> 3 handling Ready - Ready MustSync=false: - Messages: - 3->1 MsgAppResp Term:1 Log:0/11 > 1 receiving messages 2->1 MsgHeartbeatResp Term:1 Log:0/0 - 3->1 MsgAppResp Term:1 Log:0/11 diff --git a/testdata/snapshot_succeed_via_app_resp_behind.txt b/testdata/snapshot_succeed_via_app_resp_behind.txt index b0f5883b..b27f19e8 100644 --- a/testdata/snapshot_succeed_via_app_resp_behind.txt +++ b/testdata/snapshot_succeed_via_app_resp_behind.txt @@ -48,9 +48,9 @@ raft-state status 1 ---- -1: StateReplicate match=11 next=12 -2: StateReplicate match=11 next=12 -3: StateProbe match=0 next=11 paused inactive +1: StateReplicate match=11 commit=10 next=12 +2: StateReplicate match=11 commit=11 next=12 +3: StateProbe match=0 commit=0 next=11 paused inactive raft-log 3 ---- @@ -83,9 +83,9 @@ ok status 1 ---- -1: StateReplicate match=12 next=13 -2: StateReplicate match=12 next=13 -3: StateProbe match=0 next=11 paused inactive +1: StateReplicate match=12 commit=11 next=13 +2: StateReplicate match=12 commit=12 next=13 +3: StateProbe match=0 commit=0 next=11 paused inactive # 3 now gets the first MsgApp the leader originally sent, trying to append entry # 11 but this is rejected because the follower's log started at index 5. @@ -122,9 +122,9 @@ stabilize 1 > 1 receiving messages 3->1 MsgAppResp Term:1 Log:1/10 Rejected (Hint: 5) DEBUG 1 received MsgAppResp(rejected, hint: (index 5, term 1)) from 3 for index 10 - DEBUG 1 decreased progress of 3 to [StateProbe match=0 next=6] - DEBUG 1 [firstindex: 11, commit: 12] sent snapshot[index: 12, term: 1] to 3 [StateProbe match=0 next=6] - DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=6 paused pendingSnap=12] + DEBUG 1 decreased progress of 3 to [StateProbe match=0 commit=0 next=6] + DEBUG 1 [firstindex: 11, commit: 12] sent snapshot[index: 12, term: 1] to 3 [StateProbe match=0 commit=0 next=6] + DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 commit=0 next=13 paused pendingSnap=12] > 1 handling Ready Ready MustSync=false: Messages: @@ -152,7 +152,7 @@ stabilize 1 ---- > 1 receiving messages 3->1 MsgAppResp Term:1 Log:0/11 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=11 next=12 paused pendingSnap=12] + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=11 commit=0 next=13 paused pendingSnap=12] > 1 handling Ready Ready MustSync=false: Messages: @@ -162,6 +162,6 @@ stabilize 1 # This is despite its PendingSnapshot having been 12. status 1 ---- -1: StateReplicate match=12 next=13 -2: StateReplicate match=12 next=13 -3: StateReplicate match=11 next=13 inflight=1 +1: StateReplicate match=12 commit=11 next=13 +2: StateReplicate match=12 commit=12 next=13 +3: StateReplicate match=11 commit=0 next=13 inflight=1 diff --git a/tracker/progress.go b/tracker/progress.go index cb4312a9..ebf52fb0 100644 --- a/tracker/progress.go +++ b/tracker/progress.go @@ -30,13 +30,31 @@ import ( type Progress struct { // Match is the index up to which the follower's log is known to match the // leader's. + // + // Match does not regress. Match uint64 // Next is the log index of the next entry to send to this follower. All // entries with indices in (Match, Next) interval are already in flight. // // Invariant: 0 <= Match < Next. + // NB: it follows that Next >= 1. + // + // In StateSnapshot, Next == PendingSnapshot + 1. Next uint64 + // Commit is the known commit index of the follower's log. + // + // Invariant: Commit <= Match. + // Invariant: Commit <= leader's commit index. + // Commit does not regress. + Commit uint64 + // sentCommit is the highest commit index in flight to the follower. + // + // Invariant: sentCommit >= Commit. + // + // In StateSnapshot, sentCommit == PendingSnapshot == Next-1. + sentCommit uint64 + // State defines how the leader should interact with the follower. // // When in StateProbe, leader sends at most one replication message @@ -128,6 +146,7 @@ func (pr *Progress) BecomeProbe() { pr.ResetState(StateProbe) pr.Next = pr.Match + 1 } + pr.sentCommit = min(pr.sentCommit, pr.Next-1) } // BecomeReplicate transitions into StateReplicate, resetting Next to Match+1. @@ -141,14 +160,16 @@ func (pr *Progress) BecomeReplicate() { func (pr *Progress) BecomeSnapshot(snapshoti uint64) { pr.ResetState(StateSnapshot) pr.PendingSnapshot = snapshoti + pr.Next = snapshoti + 1 + pr.sentCommit = snapshoti } -// UpdateOnEntriesSend updates the progress on the given number of consecutive -// entries being sent in a MsgApp, with the given total bytes size, appended at -// log indices >= pr.Next. +// SentEntries updates the progress on the given number of consecutive entries +// being sent in a MsgApp, with the given total bytes size, appended at log +// indices >= pr.Next. // // Must be used with StateProbe or StateReplicate. -func (pr *Progress) UpdateOnEntriesSend(entries int, bytes uint64) { +func (pr *Progress) SentEntries(entries int, bytes uint64) { switch pr.State { case StateReplicate: if entries > 0 { @@ -170,6 +191,26 @@ func (pr *Progress) UpdateOnEntriesSend(entries int, bytes uint64) { } } +// CanBumpCommit returns true if sending the given commit index can potentially +// advance the follower's commit index. +func (pr *Progress) CanBumpCommit(index uint64) bool { + // Sending the given commit index may bump the follower's commit index up to + // Next-1 in normal operation, or higher in some rare cases. Allow sending a + // commit index eagerly only if we haven't already sent one that bumps the + // follower's commit all the way to Next-1. + return index > pr.sentCommit && pr.sentCommit < pr.Next-1 +} + +// SentCommit updates the sentCommit. +func (pr *Progress) SentCommit(commit uint64) { + pr.sentCommit = commit +} + +// UpdateCommit moves the known commit index for this follower forward. +func (pr *Progress) UpdateCommit(index uint64) { + pr.Commit = max(pr.Commit, min(index, pr.Match)) +} + // MaybeUpdate is called when an MsgAppResp arrives from the follower, with the // index acked by it. The method returns false if the given n index comes from // an outdated message. Otherwise it updates the progress and returns true. @@ -205,6 +246,8 @@ func (pr *Progress) MaybeDecrTo(rejected, matchHint uint64) bool { // // TODO(tbg): why not use matchHint if it's larger? pr.Next = pr.Match + 1 + // Regress the sentCommit since it unlikely has been applied. + pr.sentCommit = min(pr.sentCommit, pr.Next-1) return true } @@ -216,6 +259,8 @@ func (pr *Progress) MaybeDecrTo(rejected, matchHint uint64) bool { } pr.Next = max(min(rejected, matchHint+1), pr.Match+1) + // Regress the sentCommit since it unlikely has been applied. + pr.sentCommit = min(pr.sentCommit, pr.Next-1) pr.MsgAppFlowPaused = false return true } @@ -241,7 +286,7 @@ func (pr *Progress) IsPaused() bool { func (pr *Progress) String() string { var buf strings.Builder - fmt.Fprintf(&buf, "%s match=%d next=%d", pr.State, pr.Match, pr.Next) + fmt.Fprintf(&buf, "%s match=%d commit=%d next=%d", pr.State, pr.Match, pr.Commit, pr.Next) if pr.IsLearner { fmt.Fprint(&buf, " learner") } diff --git a/tracker/progress_test.go b/tracker/progress_test.go index 49dedb53..702aef0d 100644 --- a/tracker/progress_test.go +++ b/tracker/progress_test.go @@ -24,8 +24,9 @@ func TestProgressString(t *testing.T) { ins := NewInflights(1, 0) ins.Add(123, 1) pr := &Progress{ - Match: 1, - Next: 2, + Match: 2, + Commit: 1, + Next: 3, State: StateSnapshot, PendingSnapshot: 123, RecentActive: false, @@ -33,7 +34,7 @@ func TestProgressString(t *testing.T) { IsLearner: true, Inflights: ins, } - const exp = `StateSnapshot match=1 next=2 learner paused pendingSnap=123 inactive inflight=1[full]` + const exp = `StateSnapshot match=2 commit=1 next=3 learner paused pendingSnap=123 inactive inflight=1[full]` assert.Equal(t, exp, pr.String()) }