From e9e525d0fe9469981586ba087daf6a7d6731774e Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 29 Sep 2022 17:33:35 -0400 Subject: [PATCH 01/13] raft: async leader vote This commit introduces an intermediate state that delays the acknowledgement of a node's self-vote during an election until that vote has been durably persisted (i.e. on the next call to Advance). This change can be viewed as the election counterpart to #14413. This is an intermediate state that limits code movement for the rest of the async storage writes change. Signed-off-by: Nathan VanBenschoten --- node_test.go | 23 +++- raft.go | 38 +++++-- raft_paper_test.go | 2 + raft_test.go | 101 ++++++++++++++++++ rawnode_test.go | 12 ++- testdata/campaign.txt | 3 +- testdata/campaign_learner_must_vote.txt | 3 +- testdata/confchange_v1_add_single.txt | 8 +- testdata/confchange_v2_add_double_auto.txt | 8 +- .../confchange_v2_add_double_implicit.txt | 8 +- testdata/confchange_v2_add_single_auto.txt | 8 +- .../confchange_v2_add_single_explicit.txt | 8 +- testdata/confchange_v2_replace_leader.txt | 3 +- testdata/probe_and_replicate.txt | 3 +- testdata/single_node.txt | 10 +- 15 files changed, 212 insertions(+), 26 deletions(-) diff --git a/node_test.go b/node_test.go index d0da8a2f..4343c234 100644 --- a/node_test.go +++ b/node_test.go @@ -393,7 +393,8 @@ func TestNodeProposeAddDuplicateNode(t *testing.T) { // know who is the current leader; node will accept proposal when it knows // who is the current leader. func TestBlockProposal(t *testing.T) { - rn := newTestRawNode(1, 10, 1, newTestMemoryStorage(withPeers(1))) + s := newTestMemoryStorage(withPeers(1)) + rn := newTestRawNode(1, 10, 1, s) n := newNode(rn) go n.run() defer n.Stop() @@ -412,6 +413,9 @@ func TestBlockProposal(t *testing.T) { } n.Campaign(context.TODO()) + rd := <-n.Ready() + s.Append(rd.Entries) + n.Advance() select { case err := <-errc: if err != nil { @@ -586,9 +590,14 @@ func TestNodeStart(t *testing.T) { } { + // Persist vote. rd := <-n.Ready() storage.Append(rd.Entries) n.Advance() + // Append empty entry. + rd = <-n.Ready() + storage.Append(rd.Entries) + n.Advance() } n.Propose(ctx, []byte("foo")) @@ -723,8 +732,12 @@ func TestNodeAdvance(t *testing.T) { defer cancel() n.Campaign(ctx) + // Persist vote. rd := readyWithTimeout(n) - // Commit empty entry. + storage.Append(rd.Entries) + n.Advance() + // Append empty entry. + rd = readyWithTimeout(n) storage.Append(rd.Entries) n.Advance() @@ -878,9 +891,15 @@ func TestCommitPagination(t *testing.T) { defer cancel() n.Campaign(ctx) + // Persist vote. rd := readyWithTimeout(n) s.Append(rd.Entries) n.Advance() + // Append empty entry. + rd = readyWithTimeout(n) + s.Append(rd.Entries) + n.Advance() + // Apply empty entry. rd = readyWithTimeout(n) if len(rd.CommittedEntries) != 1 { t.Fatalf("expected 1 (empty) entry, got %d", len(rd.CommittedEntries)) diff --git a/raft.go b/raft.go index fd7cfddf..1ba56772 100644 --- a/raft.go +++ b/raft.go @@ -278,6 +278,14 @@ type raft struct { msgs []pb.Message + // voteSelfOnAdvance is a marker that the local raft node should vote for + // itself upon its next call to advance. This is not meant to be the final + // approach towards handling self-votes, but it's a useful intermediate + // point to get all tests working and to write some additional tests that + // demonstrate possible race conditions when self-voting is asynchronous. + // This is replaced in a later commit. + voteSelfOnAdvance pb.Message + // the leader id lead uint64 // leadTransferee is id of the leader transfer target when its value is not zero. @@ -616,6 +624,7 @@ func (r *raft) advance(rd Ready) { if !IsEmptySnap(rd.Snapshot) { r.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index) } + r.maybeVoteForSelf() } // maybeCommit attempts to advance the commit index. Returns true if @@ -626,6 +635,22 @@ func (r *raft) maybeCommit() bool { return r.raftLog.maybeCommit(mci, r.Term) } +// maybeVoteForSelf attempts to inform a (pre-)candidate node that its +// vote for itself has been made durable and can now be counted towards +// the active election, if one is still ongoing. Returns true if the +// node was informed of a self-vote. +func (r *raft) maybeVoteForSelf() bool { + if r.voteSelfOnAdvance.Type == 0 { + return false + } + voteMsg := r.voteSelfOnAdvance + // NB: Clear the voteSelfOnAdvance marker before calling Step. + // Step may re-set the marker and cause us to loop. + r.voteSelfOnAdvance = pb.Message{} + _ = r.Step(voteMsg) + return true +} + func (r *raft) reset(term uint64) { if r.Term != term { r.Term = term @@ -840,16 +865,6 @@ func (r *raft) campaign(t CampaignType) { voteMsg = pb.MsgVote term = r.Term } - if _, _, res := r.poll(r.id, voteRespMsgType(voteMsg), true); res == quorum.VoteWon { - // We won the election after voting for ourselves (which must mean that - // this is a single-node cluster). Advance to the next state. - if t == campaignPreElection { - r.campaign(campaignElection) - } else { - r.becomeLeader() - } - return - } var ids []uint64 { idMap := r.prs.Voters.IDs() @@ -861,6 +876,7 @@ func (r *raft) campaign(t CampaignType) { } for _, id := range ids { if id == r.id { + r.voteSelfOnAdvance = pb.Message{To: id, From: id, Term: term, Type: voteRespMsgType(voteMsg)} continue } r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d", @@ -870,7 +886,7 @@ func (r *raft) campaign(t CampaignType) { if t == campaignTransfer { ctx = []byte(t) } - r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx}) + r.send(pb.Message{To: id, Term: term, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx}) } } diff --git a/raft_paper_test.go b/raft_paper_test.go index b8000e4e..f39fd561 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -162,6 +162,7 @@ func testNonleaderStartElection(t *testing.T, state StateType) { for i := 1; i < 2*et; i++ { r.tick() } + r.maybeVoteForSelf() if r.Term != 2 { t.Errorf("term = %d, want 2", r.Term) @@ -218,6 +219,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) { r := newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(idsBySize(tt.size)...))) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + r.maybeVoteForSelf() for id, vote := range tt.votes { r.Step(pb.Message{From: id, To: 1, Term: r.Term, Type: pb.MsgVoteResp, Reject: !vote}) } diff --git a/raft_test.go b/raft_test.go index 7f4f660c..030e59b5 100644 --- a/raft_test.go +++ b/raft_test.go @@ -49,6 +49,7 @@ func mustAppendEntry(r *raft, ents ...pb.Entry) { type stateMachine interface { Step(m pb.Message) error readMessages() []pb.Message + maybeVoteForSelf() bool } func (r *raft) readMessages() []pb.Message { @@ -382,6 +383,7 @@ func TestLearnerPromotion(t *testing.T) { setRandomizedElectionTimeout(n1, n1.electionTimeout) for i := 0; i < n1.electionTimeout; i++ { n1.tick() + n1.maybeVoteForSelf() } if n1.state != StateLeader { @@ -403,6 +405,7 @@ func TestLearnerPromotion(t *testing.T) { setRandomizedElectionTimeout(n2, n2.electionTimeout) for i := 0; i < n2.electionTimeout; i++ { n2.tick() + n2.maybeVoteForSelf() } nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat}) @@ -700,6 +703,7 @@ func TestLearnerLogReplication(t *testing.T) { setRandomizedElectionTimeout(n1, n1.electionTimeout) for i := 0; i < n1.electionTimeout; i++ { n1.tick() + n1.maybeVoteForSelf() } nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat}) @@ -1751,6 +1755,7 @@ func testCandidateResetTerm(t *testing.T, mt pb.MessageType) { c.resetRandomizedElectionTimeout() for i := 0; i < c.randomizedElectionTimeout; i++ { c.tick() + c.maybeVoteForSelf() } if c.state != StateCandidate { @@ -1773,6 +1778,95 @@ func testCandidateResetTerm(t *testing.T, mt pb.MessageType) { } } +// The following three tests exercise the behavior of a (pre-)candidate when its +// own self-vote is delivered back to itself after the peer has already learned +// that it has lost the election. The self-vote should be ignored in these cases. + +func TestCandidateSelfVoteAfterLostElection(t *testing.T) { + testCandidateSelfVoteAfterLostElection(t, false) +} + +func TestCandidateSelfVoteAfterLostElectionPreVote(t *testing.T) { + testCandidateSelfVoteAfterLostElection(t, true) +} + +func testCandidateSelfVoteAfterLostElection(t *testing.T, preVote bool) { + sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) + sm.preVote = preVote + + // n1 calls an election. + sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + voteMsg := sm.voteSelfOnAdvance + + // n1 hears that n2 already won the election before it has had a + // change to sync its vote to disk and account for its self-vote. + // Becomes a follower. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term, Type: pb.MsgHeartbeat}) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } + + // n1 remains a follower even after its self-vote is delivered. + sm.Step(voteMsg) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } + + // Its self-vote does not make its way to its ProgressTracker. + granted, _, _ := sm.prs.TallyVotes() + if granted != 0 { + t.Errorf("granted = %v, want %v", granted, 0) + } +} + +func TestCandidateDeliversPreCandidateSelfVoteAfterBecomingCandidate(t *testing.T) { + sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) + sm.preVote = true + + // n1 calls an election. + sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + preVoteMsg := sm.voteSelfOnAdvance + if sm.state != StatePreCandidate { + t.Errorf("state = %v, want %v", sm.state, StatePreCandidate) + } + + // n1 receives pre-candidate votes from both other peers before + // voting for itself. n1 becomes a candidate. + // NB: pre-vote messages carry the local term + 1. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term + 1, Type: pb.MsgPreVoteResp}) + sm.Step(pb.Message{From: 3, To: 1, Term: sm.Term + 1, Type: pb.MsgPreVoteResp}) + if sm.state != StateCandidate { + t.Errorf("state = %v, want %v", sm.state, StateCandidate) + } + + // n1 remains a candidate even after its delayed pre-vote self-vote is + // delivered. + sm.Step(preVoteMsg) + voteMsg := sm.voteSelfOnAdvance + if sm.state != StateCandidate { + t.Errorf("state = %v, want %v", sm.state, StateCandidate) + } + + // Its pre-vote self-vote does not make its way to its ProgressTracker. + granted, _, _ := sm.prs.TallyVotes() + if granted != 0 { + t.Errorf("granted = %v, want %v", granted, 0) + } + + // A single vote from n2 does not move n1 to the leader. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term, Type: pb.MsgVoteResp}) + if sm.state != StateCandidate { + t.Errorf("state = %v, want %v", sm.state, StateCandidate) + } + + // n1 becomes the leader once its self-vote is received because now + // quorum is reached. + sm.Step(voteMsg) + if sm.state != StateLeader { + t.Errorf("state = %v, want %v", sm.state, StateLeader) + } +} + func TestLeaderStepdownWhenQuorumActive(t *testing.T) { sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) @@ -3382,11 +3476,15 @@ func testCampaignWhileLeader(t *testing.T, preVote bool) { // We don't call campaign() directly because it comes after the check // for our current state. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + for r.maybeVoteForSelf() { + } if r.state != StateLeader { t.Errorf("expected single-node election to become leader but got %s", r.state) } term := r.Term r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + for r.maybeVoteForSelf() { + } if r.state != StateLeader { t.Errorf("expected to remain leader but got %s", r.state) } @@ -4754,6 +4852,8 @@ func (nw *network) send(msgs ...pb.Message) { nw.t.Log(DescribeMessage(m, nil)) } p.Step(m) + for p.maybeVoteForSelf() { + } msgs = append(msgs[1:], nw.filter(p.readMessages())...) } } @@ -4820,6 +4920,7 @@ type blackHole struct{} func (blackHole) Step(pb.Message) error { return nil } func (blackHole) readMessages() []pb.Message { return nil } +func (blackHole) maybeVoteForSelf() bool { return false } var nopStepper = &blackHole{} diff --git a/rawnode_test.go b/rawnode_test.go index 49f04c13..d9e8551b 100644 --- a/rawnode_test.go +++ b/rawnode_test.go @@ -490,6 +490,10 @@ func TestRawNodeJointAutoLeave(t *testing.T) { rd = rawNode.Ready() t.Log(DescribeReady(rd, nil)) s.Append(rd.Entries) + rawNode.Advance(rd) + rd = rawNode.Ready() + t.Log(DescribeReady(rd, nil)) + s.Append(rd.Entries) // Check that the right ConfChange comes out. if len(rd.Entries) != 1 || rd.Entries[0].Type != pb.EntryConfChangeV2 { t.Fatalf("expected exactly one more entry, got %+v", rd) @@ -743,11 +747,14 @@ func TestRawNodeStart(t *testing.T) { t.Fatalf("unexpected ready: %+v", rawNode.Ready()) } rawNode.Campaign() + rd := rawNode.Ready() + storage.Append(rd.Entries) + rawNode.Advance(rd) rawNode.Propose([]byte("foo")) if !rawNode.HasReady() { t.Fatal("expected a Ready") } - rd := rawNode.Ready() + rd = rawNode.Ready() if !reflect.DeepEqual(entries, rd.Entries) { t.Fatalf("expected to see entries\n%s, not\n%s", DescribeEntries(entries, nil), DescribeEntries(rd.Entries, nil)) } @@ -861,6 +868,9 @@ func TestRawNodeStatus(t *testing.T) { if err := rn.Campaign(); err != nil { t.Fatal(err) } + rd := rn.Ready() + s.Append(rd.Entries) + rn.Advance(rd) status := rn.Status() if status.Lead != 1 { t.Fatal("not lead") diff --git a/testdata/campaign.txt b/testdata/campaign.txt index c5deb2dc..4eefab36 100644 --- a/testdata/campaign.txt +++ b/testdata/campaign.txt @@ -18,7 +18,6 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 -INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 [logterm: 1, index: 2] sent MsgVote request to 2 at term 1 INFO 1 [logterm: 1, index: 2] sent MsgVote request to 3 at term 1 @@ -31,6 +30,8 @@ stabilize Messages: 1->2 MsgVote Term:1 Log:1/2 1->3 MsgVote Term:1 Log:1/2 + INFO 1 received MsgVoteResp from 1 at term 1 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections > 2 receiving messages 1->2 MsgVote Term:1 Log:1/2 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] diff --git a/testdata/campaign_learner_must_vote.txt b/testdata/campaign_learner_must_vote.txt index 55d42aa4..14530b55 100644 --- a/testdata/campaign_learner_must_vote.txt +++ b/testdata/campaign_learner_must_vote.txt @@ -54,7 +54,6 @@ campaign 2 ---- INFO 2 is starting a new election at term 1 INFO 2 became candidate at term 2 -INFO 2 received MsgVoteResp from 2 at term 2 INFO 2 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2 INFO 2 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 @@ -67,6 +66,8 @@ HardState Term:2 Vote:2 Commit:4 Messages: 2->1 MsgVote Term:2 Log:1/4 2->3 MsgVote Term:2 Log:1/4 +INFO 2 received MsgVoteResp from 2 at term 2 +INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections # n2 is now campaigning while n1 is down (does not respond). The latest config # has n3 as a voter, but n3 doesn't even have the corresponding conf change in diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index cd07af47..89bb2b98 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -11,7 +11,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 # Add v2 (with an auto transition). @@ -35,7 +42,6 @@ stabilize > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChange v2 diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index 0979bdd6..5b91716d 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -13,7 +13,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 propose-conf-change 1 transition=auto @@ -36,7 +43,6 @@ process-ready 1 ---- Ready MustSync=true: Lead:1 State:StateLeader -HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 v3 diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index 45dfc509..7484d7c8 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -15,7 +15,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 propose-conf-change 1 transition=implicit @@ -38,7 +45,6 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index 7ee3ab6c..52e80ae5 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -13,7 +13,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 # Add v2 (with an auto transition). @@ -36,7 +43,6 @@ stabilize > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index b4e6e3a8..ec44fa20 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -13,7 +13,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 # Add v2 with an explicit transition. @@ -36,7 +43,6 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index be7b573c..94892e2b 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -185,7 +185,6 @@ stabilize INFO 4 [term 1] received MsgTimeoutNow from 1 and starts an election to get leadership. INFO 4 is starting a new election at term 1 INFO 4 became candidate at term 2 - INFO 4 received MsgVoteResp from 4 at term 2 INFO 4 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2 INFO 4 [logterm: 1, index: 4] sent MsgVote request to 2 at term 2 INFO 4 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 @@ -197,6 +196,8 @@ stabilize 4->1 MsgVote Term:2 Log:1/4 4->2 MsgVote Term:2 Log:1/4 4->3 MsgVote Term:2 Log:1/4 + INFO 4 received MsgVoteResp from 4 at term 2 + INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections > 1 receiving messages 4->1 MsgVote Term:2 Log:1/4 INFO 1 [term: 1] received a MsgVote message with higher term from 4 [term: 2] diff --git a/testdata/probe_and_replicate.txt b/testdata/probe_and_replicate.txt index bebae6ef..8e61b618 100644 --- a/testdata/probe_and_replicate.txt +++ b/testdata/probe_and_replicate.txt @@ -357,7 +357,6 @@ campaign 1 ---- INFO 1 is starting a new election at term 7 INFO 1 became candidate at term 8 -INFO 1 received MsgVoteResp from 1 at term 8 INFO 1 [logterm: 6, index: 20] sent MsgVote request to 2 at term 8 INFO 1 [logterm: 6, index: 20] sent MsgVote request to 3 at term 8 INFO 1 [logterm: 6, index: 20] sent MsgVote request to 4 at term 8 @@ -379,6 +378,8 @@ stabilize 1 1->5 MsgVote Term:8 Log:6/20 1->6 MsgVote Term:8 Log:6/20 1->7 MsgVote Term:8 Log:6/20 + INFO 1 received MsgVoteResp from 1 at term 8 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections stabilize 2 3 4 5 6 7 ---- diff --git a/testdata/single_node.txt b/testdata/single_node.txt index 3b6e4f4c..f6aceb71 100644 --- a/testdata/single_node.txt +++ b/testdata/single_node.txt @@ -12,15 +12,19 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 -INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 became leader at term 1 stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:3 + INFO 1 received MsgVoteResp from 1 at term 1 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 1 became leader at term 1 +> 1 handling Ready + Ready MustSync=true: + Lead:1 State:StateLeader Entries: 1/4 EntryNormal "" > 1 handling Ready From a9b19847949ff6616320c3f0dcb8ae3650f914b0 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sun, 25 Sep 2022 21:17:24 -0400 Subject: [PATCH 02/13] raft: track in-progress log writes in unstable This commit adds a mechanism to the unstable struct to track "in-progress" log writes that are not yet stable. In-progress writes are still tracked in the unstable struct, which is necessary because they are not yet guaranteed to be present in `Storage` implementations. However, they are not included in the Entries field of future Ready structs, which avoids redundant raft log writes. The commit also does the same thing for the optional unstable snapshots. For now, entries and snapshots are immediately considered stable by `raft.advance`. A future commit will make it possible to accept multiple Ready structs without immediately stabilizing entries and snapshots from earlier Ready structs. This all works towards async Raft log writes, where log writes are decoupled from Ready iterations. Signed-off-by: Nathan VanBenschoten --- log.go | 46 ++- log_test.go | 8 +- log_unstable.go | 88 ++++- log_unstable_test.go | 353 +++++++++++++++--- node.go | 6 +- raft.go | 2 +- raft_paper_test.go | 6 +- rawnode.go | 7 +- testdata/confchange_v1_add_single.txt | 2 +- testdata/confchange_v2_add_double_auto.txt | 4 +- .../confchange_v2_add_double_implicit.txt | 2 +- testdata/confchange_v2_add_single_auto.txt | 2 +- .../confchange_v2_add_single_explicit.txt | 2 +- testdata/confchange_v2_replace_leader.txt | 2 +- testdata/snapshot_succeed_via_app_resp.txt | 2 +- 15 files changed, 431 insertions(+), 101 deletions(-) diff --git a/log.go b/log.go index 133df420..3359e08c 100644 --- a/log.go +++ b/log.go @@ -71,6 +71,7 @@ func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uin panic(err) // TODO(bdarnell) } log.unstable.offset = lastIndex + 1 + log.unstable.offsetInProgress = lastIndex + 1 log.unstable.logger = logger // Initialize our committed and applied pointers to the time of the last compaction. log.committed = firstIndex - 1 @@ -80,7 +81,8 @@ func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uin } func (l *raftLog) String() string { - return fmt.Sprintf("committed=%d, applied=%d, unstable.offset=%d, len(unstable.Entries)=%d", l.committed, l.applied, l.unstable.offset, len(l.unstable.entries)) + return fmt.Sprintf("committed=%d, applied=%d, unstable.offset=%d, unstable.offsetInProgress=%d, len(unstable.Entries)=%d", + l.committed, l.applied, l.unstable.offset, l.unstable.offsetInProgress, len(l.unstable.entries)) } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, @@ -170,18 +172,23 @@ func (l *raftLog) findConflictByTerm(index uint64, term uint64) uint64 { return index } -func (l *raftLog) unstableEntries() []pb.Entry { - if len(l.unstable.entries) == 0 { - return nil - } - return l.unstable.entries +// nextUnstableEnts returns all entries that are available to be written to the +// local stable log and are not already in-progress. +func (l *raftLog) nextUnstableEnts() []pb.Entry { + return l.unstable.nextEntries() +} + +// hasNextUnstableEnts returns if there are any entries that are available to be +// written to the local stable log and are not already in-progress. +func (l *raftLog) hasNextUnstableEnts() bool { + return len(l.nextUnstableEnts()) > 0 } // nextCommittedEnts returns all the available entries for execution. // If applied is smaller than the index of snapshot, it returns all committed // entries after the index of snapshot. func (l *raftLog) nextCommittedEnts() (ents []pb.Entry) { - if l.hasPendingSnapshot() { + if l.hasNextOrInProgressSnapshot() { // See comment in hasNextCommittedEnts. return nil } @@ -199,7 +206,7 @@ func (l *raftLog) nextCommittedEnts() (ents []pb.Entry) { // hasNextCommittedEnts returns if there is any available entries for execution. // This is a fast check without heavy raftLog.slice() in nextCommittedEnts(). func (l *raftLog) hasNextCommittedEnts() bool { - if l.hasPendingSnapshot() { + if l.hasNextOrInProgressSnapshot() { // If we have a snapshot to apply, don't also return any committed // entries. Doing so raises questions about what should be applied // first. @@ -208,8 +215,21 @@ func (l *raftLog) hasNextCommittedEnts() bool { return l.committed > l.applied } -// hasPendingSnapshot returns if there is pending snapshot waiting for applying. -func (l *raftLog) hasPendingSnapshot() bool { +// nextUnstableSnapshot returns the snapshot, if present, that is available to +// be applied to the local storage and is not already in-progress. +func (l *raftLog) nextUnstableSnapshot() *pb.Snapshot { + return l.unstable.nextSnapshot() +} + +// hasNextUnstableSnapshot returns if there is a snapshot that is available to +// be applied to the local storage and is not already in-progress. +func (l *raftLog) hasNextUnstableSnapshot() bool { + return l.unstable.nextSnapshot() != nil +} + +// hasNextOrInProgressSnapshot returns if there is pending snapshot waiting for +// applying or in the process of being applied. +func (l *raftLog) hasNextOrInProgressSnapshot() bool { return l.unstable.snapshot != nil } @@ -266,6 +286,12 @@ func (l *raftLog) stableTo(i, t uint64) { l.unstable.stableTo(i, t) } func (l *raftLog) stableSnapTo(i uint64) { l.unstable.stableSnapTo(i) } +// acceptUnstable indicates that the application has started persisting the +// unstable entries in storage, and that the current unstable entries are thus +// to be marked as being in-progress, to avoid returning them with future calls +// to Ready(). +func (l *raftLog) acceptUnstable() { l.unstable.acceptInProgress() } + func (l *raftLog) lastTerm() uint64 { t, err := l.term(l.lastIndex()) if err != nil { diff --git a/log_test.go b/log_test.go index d062831b..a5bfc4d8 100644 --- a/log_test.go +++ b/log_test.go @@ -284,7 +284,7 @@ func TestCompactionSideEffects(t *testing.T) { require.True(t, raftLog.matchTerm(j, j)) } - unstableEnts := raftLog.unstableEntries() + unstableEnts := raftLog.nextUnstableEnts() require.Equal(t, 250, len(unstableEnts)) require.Equal(t, uint64(751), unstableEnts[0].Index) @@ -378,9 +378,9 @@ func TestNextCommittedEnts(t *testing.T) { } } -// TestUnstableEnts ensures unstableEntries returns the unstable part of the +// TestNextUnstableEnts ensures unstableEntries returns the unstable part of the // entries correctly. -func TestUnstableEnts(t *testing.T) { +func TestNextUnstableEnts(t *testing.T) { previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { unstable uint64 @@ -400,7 +400,7 @@ func TestUnstableEnts(t *testing.T) { raftLog := newLog(storage, raftLogger) raftLog.append(previousEnts[tt.unstable-1:]...) - ents := raftLog.unstableEntries() + ents := raftLog.nextUnstableEnts() if l := len(ents); l > 0 { raftLog.stableTo(ents[l-1].Index, ents[l-1].Term) } diff --git a/log_unstable.go b/log_unstable.go index 0e4ba183..5caa433e 100644 --- a/log_unstable.go +++ b/log_unstable.go @@ -16,6 +16,16 @@ package raft import pb "go.etcd.io/raft/v3/raftpb" +// unstable contains "unstable" log entries and snapshot state that has +// not yet been written to Storage. The type serves two roles. First, it +// holds on to new log entries and an optional snapshot until they are +// handed to a Ready struct for persistence. Second, it continues to +// hold on to this state after it has been handed off to provide raftLog +// with a view of the in-progress log entries and snapshot until their +// writes have been stabilized and are guaranteed to be reflected in +// queries of Storage. After this point, the corresponding log entries +// and/or snapshot can be cleared from unstable. +// // unstable.entries[i] has raft log position i+unstable.offset. // Note that unstable.offset may be less than the highest log // position in storage; this means that the next write to storage @@ -25,7 +35,16 @@ type unstable struct { snapshot *pb.Snapshot // all entries that have not yet been written to storage. entries []pb.Entry - offset uint64 + // entries[i] has raft log position i+offset. + offset uint64 + + // if true, snapshot is being written to storage. + snapshotInProgress bool + // entries[:offsetInProgress-offset] are being written to storage. + // Like offset, offsetInProgress is exclusive, meaning that it + // contains the index following the largest in-progress entry. + // Invariant: offset <= offsetInProgress + offsetInProgress uint64 logger Logger } @@ -72,6 +91,42 @@ func (u *unstable) maybeTerm(i uint64) (uint64, bool) { return u.entries[i-u.offset].Term, true } +// nextEntries returns the unstable entries that are not already in the process +// of being written to storage. +func (u *unstable) nextEntries() []pb.Entry { + inProgress := int(u.offsetInProgress - u.offset) + if len(u.entries) == inProgress { + return nil + } + return u.entries[inProgress:] +} + +// nextSnapshot returns the unstable snapshot, if one exists that is not already +// in the process of being written to storage. +func (u *unstable) nextSnapshot() *pb.Snapshot { + if u.snapshot == nil || u.snapshotInProgress { + return nil + } + return u.snapshot +} + +// acceptInProgress marks all entries and the snapshot, if any, in the unstable +// as having begun the process of being written to storage. The entries/snapshot +// will no longer be returned from nextEntries/nextSnapshot. However, new +// entries/snapshots added after a call to acceptInProgress will be returned +// from those methods, until the next call to acceptInProgress. +func (u *unstable) acceptInProgress() { + if len(u.entries) > 0 { + // NOTE: +1 because offsetInProgress is exclusive, like offset. + u.offsetInProgress = u.entries[len(u.entries)-1].Index + 1 + } + if u.snapshot != nil { + u.snapshotInProgress = true + } +} + +// stableTo marks entries up to the entry with the specified (index, term) as +// being successfully written to stable storage. func (u *unstable) stableTo(i, t uint64) { gt, ok := u.maybeTerm(i) if !ok { @@ -84,10 +139,15 @@ func (u *unstable) stableTo(i, t uint64) { } if gt != t { // Term mismatch between unstable entry and specified entry. Ignore. + // This is possible if part or all of the unstable log was replaced + // between that time that a set of entries started to be written to + // stable storage and when they finished. return } - u.entries = u.entries[i+1-u.offset:] + num := int(i + 1 - u.offset) + u.entries = u.entries[num:] u.offset = i + 1 + u.offsetInProgress = max(u.offsetInProgress, u.offset) u.shrinkEntriesArray() } @@ -113,37 +173,47 @@ func (u *unstable) shrinkEntriesArray() { func (u *unstable) stableSnapTo(i uint64) { if u.snapshot != nil && u.snapshot.Metadata.Index == i { u.snapshot = nil + u.snapshotInProgress = false } } func (u *unstable) restore(s pb.Snapshot) { u.offset = s.Metadata.Index + 1 + u.offsetInProgress = u.offset u.entries = nil u.snapshot = &s + u.snapshotInProgress = false } func (u *unstable) truncateAndAppend(ents []pb.Entry) { + // TODO(nvanbenschoten): rename this variable to firstAppIndex. after := ents[0].Index switch { case after == u.offset+uint64(len(u.entries)): - // after is the next index in the u.entries - // directly append + // after is the next index in the u.entries, so append directly. u.entries = append(u.entries, ents...) case after <= u.offset: u.logger.Infof("replace the unstable entries from index %d", after) // The log is being truncated to before our current offset - // portion, so set the offset and replace the entries - u.offset = after + // portion, so set the offset and replace the entries. u.entries = ents + u.offset = after + u.offsetInProgress = u.offset default: - // truncate to after and copy to u.entries - // then append + // truncate to after and copy to u.entries then append. u.logger.Infof("truncate the unstable entries before index %d", after) - u.entries = append([]pb.Entry{}, u.slice(u.offset, after)...) + keep := u.slice(u.offset, after) + u.entries = append([]pb.Entry{}, keep...) u.entries = append(u.entries, ents...) + // Only in-progress entries before after are still considered to be + // in-progress. + u.offsetInProgress = min(u.offsetInProgress, after) } } +// slice returns the entries from the unstable log with indexes in the +// range [lo, hi). The entire range must be stored in the unstable log +// or the method will panic. func (u *unstable) slice(lo uint64, hi uint64) []pb.Entry { u.mustCheckOutOfBounds(lo, hi) return u.entries[lo-u.offset : hi-u.offset] diff --git a/log_unstable_test.go b/log_unstable_test.go index bf03f248..a0b4f8e3 100644 --- a/log_unstable_test.go +++ b/log_unstable_test.go @@ -53,7 +53,6 @@ func TestUnstableMaybeFirstIndex(t *testing.T) { } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ entries: tt.entries, @@ -99,7 +98,6 @@ func TestMaybeLastIndex(t *testing.T) { } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ entries: tt.entries, @@ -179,7 +177,6 @@ func TestUnstableMaybeTerm(t *testing.T) { } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ entries: tt.entries, @@ -196,98 +193,309 @@ func TestUnstableMaybeTerm(t *testing.T) { func TestUnstableRestore(t *testing.T) { u := unstable{ - entries: []pb.Entry{{Index: 5, Term: 1}}, - offset: 5, - snapshot: &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, - logger: raftLogger, + entries: []pb.Entry{{Index: 5, Term: 1}}, + offset: 5, + offsetInProgress: 6, + snapshot: &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + snapshotInProgress: true, + logger: raftLogger, } s := pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 6, Term: 2}} u.restore(s) require.Equal(t, s.Metadata.Index+1, u.offset) + require.Equal(t, s.Metadata.Index+1, u.offsetInProgress) require.Zero(t, len(u.entries)) require.Equal(t, &s, u.snapshot) + require.False(t, u.snapshotInProgress) +} + +func TestUnstableNextEntries(t *testing.T) { + tests := []struct { + entries []pb.Entry + offset uint64 + offsetInProgress uint64 + + wentries []pb.Entry + }{ + // nothing in progress + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 5, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, + }, + // partially in progress + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 6, + []pb.Entry{{Index: 6, Term: 1}}, + }, + // everything in progress + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 7, + nil, // nil, not empty slice + }, + } + + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + u := unstable{ + entries: tt.entries, + offset: tt.offset, + offsetInProgress: tt.offsetInProgress, + logger: raftLogger, + } + res := u.nextEntries() + require.Equal(t, tt.wentries, res) + }) + } +} + +func TestUnstableNextSnapshot(t *testing.T) { + s := &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}} + tests := []struct { + snapshot *pb.Snapshot + snapshotInProgress bool + + wsnapshot *pb.Snapshot + }{ + // snapshot not unstable + { + nil, false, + nil, + }, + // snapshot not in progress + { + s, false, + s, + }, + // snapshot in progress + { + s, true, + nil, + }, + } + + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + u := unstable{ + snapshot: tt.snapshot, + snapshotInProgress: tt.snapshotInProgress, + } + res := u.nextSnapshot() + require.Equal(t, tt.wsnapshot, res) + }) + } +} + +func TestUnstableAcceptInProgress(t *testing.T) { + tests := []struct { + entries []pb.Entry + snapshot *pb.Snapshot + offsetInProgress uint64 + snapshotInProgress bool + + woffsetInProgress uint64 + wsnapshotInProgress bool + }{ + { + []pb.Entry{}, nil, + 5, // no entries + false, // snapshot not already in progress + 5, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, nil, + 5, // entries not in progress + false, // snapshot not already in progress + 6, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, nil, + 5, // entries not in progress + false, // snapshot not already in progress + 7, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, nil, + 6, // in-progress to the first entry + false, // snapshot not already in progress + 7, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, nil, + 7, // in-progress to the second entry + false, // snapshot not already in progress + 7, false, + }, + // with snapshot + { + []pb.Entry{}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // no entries + false, // snapshot not already in progress + 5, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + false, // snapshot not already in progress + 6, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + false, // snapshot not already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 6, // in-progress to the first entry + false, // snapshot not already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 7, // in-progress to the second entry + false, // snapshot not already in progress + 7, true, + }, + { + []pb.Entry{}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + true, // snapshot already in progress + 5, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + true, // snapshot already in progress + 6, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + true, // snapshot already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 6, // in-progress to the first entry + true, // snapshot already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 7, // in-progress to the second entry + true, // snapshot already in progress + 7, true, + }, + } + + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + u := unstable{ + entries: tt.entries, + snapshot: tt.snapshot, + offsetInProgress: tt.offsetInProgress, + snapshotInProgress: tt.snapshotInProgress, + } + u.acceptInProgress() + require.Equal(t, tt.woffsetInProgress, u.offsetInProgress) + require.Equal(t, tt.wsnapshotInProgress, u.snapshotInProgress) + }) + } } func TestUnstableStableTo(t *testing.T) { tests := []struct { - entries []pb.Entry - offset uint64 - snap *pb.Snapshot - index, term uint64 + entries []pb.Entry + offset uint64 + offsetInProgress uint64 + snap *pb.Snapshot + index, term uint64 - woffset uint64 - wlen int + woffset uint64 + woffsetInProgress uint64 + wlen int }{ { - []pb.Entry{}, 0, nil, + []pb.Entry{}, 0, 0, nil, 5, 1, - 0, 0, + 0, 0, 0, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, 5, 1, // stable to the first entry - 6, 0, + 6, 6, 0, }, { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 6, nil, 5, 1, // stable to the first entry - 6, 1, + 6, 6, 1, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 7, nil, + 5, 1, // stable to the first entry and in-progress ahead + 6, 7, 1, }, { - []pb.Entry{{Index: 6, Term: 2}}, 6, nil, + []pb.Entry{{Index: 6, Term: 2}}, 6, 7, nil, 6, 1, // stable to the first entry and term mismatch - 6, 1, + 6, 7, 1, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, 4, 1, // stable to old entry - 5, 1, + 5, 6, 1, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, 4, 2, // stable to old entry - 5, 1, + 5, 6, 1, }, // with snapshot { - []pb.Entry{{Index: 5, Term: 1}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, 5, 1, // stable to the first entry - 6, 0, + 6, 6, 0, }, { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, 5, 1, // stable to the first entry - 6, 1, + 6, 6, 1, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 7, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, 1, // stable to the first entry and in-progress ahead + 6, 7, 1, }, { - []pb.Entry{{Index: 6, Term: 2}}, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 5, Term: 1}}, + []pb.Entry{{Index: 6, Term: 2}}, 6, 7, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 5, Term: 1}}, 6, 1, // stable to the first entry and term mismatch - 6, 1, + 6, 7, 1, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, 4, 1, // stable to snapshot - 5, 1, + 5, 6, 1, }, { - []pb.Entry{{Index: 5, Term: 2}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 2}}, + []pb.Entry{{Index: 5, Term: 2}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 2}}, 4, 1, // stable to old entry - 5, 1, + 5, 6, 1, }, } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ - entries: tt.entries, - offset: tt.offset, - snapshot: tt.snap, - logger: raftLogger, + entries: tt.entries, + offset: tt.offset, + offsetInProgress: tt.offsetInProgress, + snapshot: tt.snap, + logger: raftLogger, } u.stableTo(tt.index, tt.term) require.Equal(t, tt.woffset, u.offset) + require.Equal(t, tt.woffsetInProgress, u.offsetInProgress) require.Equal(t, tt.wlen, len(u.entries)) }) } @@ -295,55 +503,78 @@ func TestUnstableStableTo(t *testing.T) { func TestUnstableTruncateAndAppend(t *testing.T) { tests := []struct { - entries []pb.Entry - offset uint64 - snap *pb.Snapshot - toappend []pb.Entry + entries []pb.Entry + offset uint64 + offsetInProgress uint64 + snap *pb.Snapshot + toappend []pb.Entry - woffset uint64 - wentries []pb.Entry + woffset uint64 + woffsetInProgress uint64 + wentries []pb.Entry }{ // append to the end { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 5, nil, + []pb.Entry{{Index: 6, Term: 1}, {Index: 7, Term: 1}}, + 5, 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, []pb.Entry{{Index: 6, Term: 1}, {Index: 7, Term: 1}}, - 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, + 5, 6, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, }, // replace the unstable entries { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, - 5, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 4, Term: 2}, {Index: 5, Term: 2}, {Index: 6, Term: 2}}, - 4, []pb.Entry{{Index: 4, Term: 2}, {Index: 5, Term: 2}, {Index: 6, Term: 2}}, + 4, 4, []pb.Entry{{Index: 4, Term: 2}, {Index: 5, Term: 2}, {Index: 6, Term: 2}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, + []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, }, // truncate the existing entries and append { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 6, Term: 2}}, - 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, }, { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 7, Term: 2}, {Index: 8, Term: 2}}, - 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 2}, {Index: 8, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 2}, {Index: 8, Term: 2}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 6, nil, + []pb.Entry{{Index: 6, Term: 2}}, + 5, 6, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 7, nil, + []pb.Entry{{Index: 6, Term: 2}}, + 5, 6, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, }, } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ - entries: tt.entries, - offset: tt.offset, - snapshot: tt.snap, - logger: raftLogger, + entries: tt.entries, + offset: tt.offset, + offsetInProgress: tt.offsetInProgress, + snapshot: tt.snap, + logger: raftLogger, } u.truncateAndAppend(tt.toappend) require.Equal(t, tt.woffset, u.offset) + require.Equal(t, tt.woffsetInProgress, u.offsetInProgress) require.Equal(t, tt.wentries, u.entries) }) } diff --git a/node.go b/node.go index 62288a17..8318cc24 100644 --- a/node.go +++ b/node.go @@ -561,7 +561,7 @@ func (n *node) ReadIndex(ctx context.Context, rctx []byte) error { func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { rd := Ready{ - Entries: r.raftLog.unstableEntries(), + Entries: r.raftLog.nextUnstableEnts(), CommittedEntries: r.raftLog.nextCommittedEnts(), Messages: r.msgs, } @@ -571,8 +571,8 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { if hardSt := r.hardState(); !isHardStateEqual(hardSt, prevHardSt) { rd.HardState = hardSt } - if r.raftLog.unstable.snapshot != nil { - rd.Snapshot = *r.raftLog.unstable.snapshot + if r.raftLog.hasNextUnstableSnapshot() { + rd.Snapshot = *r.raftLog.nextUnstableSnapshot() } if len(r.readStates) != 0 { rd.ReadStates = r.readStates diff --git a/raft.go b/raft.go index 1ba56772..e279baeb 100644 --- a/raft.go +++ b/raft.go @@ -1675,7 +1675,7 @@ func (r *raft) restore(s pb.Snapshot) bool { // which is true when its own id is in progress list. func (r *raft) promotable() bool { pr := r.prs.Progress[r.id] - return pr != nil && !pr.IsLearner && !r.raftLog.hasPendingSnapshot() + return pr != nil && !pr.IsLearner && !r.raftLog.hasNextOrInProgressSnapshot() } func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState { diff --git a/raft_paper_test.go b/raft_paper_test.go index f39fd561..c7124e6e 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -423,7 +423,7 @@ func TestLeaderStartReplication(t *testing.T) { if !reflect.DeepEqual(msgs, wmsgs) { t.Errorf("msgs = %+v, want %+v", msgs, wmsgs) } - if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextUnstableEnts(); !reflect.DeepEqual(g, wents) { t.Errorf("ents = %+v, want %+v", g, wents) } } @@ -691,7 +691,7 @@ func TestFollowerAppendEntries(t *testing.T) { if g := r.raftLog.allEntries(); !reflect.DeepEqual(g, tt.wents) { t.Errorf("#%d: ents = %+v, want %+v", i, g, tt.wents) } - if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, tt.wunstable) { + if g := r.raftLog.nextUnstableEnts(); !reflect.DeepEqual(g, tt.wunstable) { t.Errorf("#%d: unstableEnts = %+v, want %+v", i, g, tt.wunstable) } } @@ -926,7 +926,7 @@ func commitNoopEntry(r *raft, s *MemoryStorage) { } // ignore further messages to refresh followers' commit index r.readMessages() - s.Append(r.raftLog.unstableEntries()) + s.Append(r.raftLog.nextUnstableEnts()) r.raftLog.appliedTo(r.raftLog.committed) r.raftLog.stableTo(r.raftLog.lastIndex(), r.raftLog.lastTerm()) } diff --git a/rawnode.go b/rawnode.go index 66f345f2..dcabdd16 100644 --- a/rawnode.go +++ b/rawnode.go @@ -148,6 +148,9 @@ func (rn *RawNode) acceptReady(rd Ready) { rn.raft.readStates = nil } rn.raft.msgs = nil + // NB: this does not do anything yet, as entries and snapshots are always + // stabilized on the next Advance. + rn.raft.raftLog.acceptUnstable() } // HasReady called when RawNode user need to check if any Ready pending. @@ -159,10 +162,10 @@ func (rn *RawNode) HasReady() bool { if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) { return true } - if r.raftLog.hasPendingSnapshot() { + if r.raftLog.hasNextUnstableSnapshot() { return true } - if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextCommittedEnts() { + if len(r.msgs) > 0 || r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts() { return true } if len(r.readStates) != 0 { diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index 89bb2b98..f33fb310 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -79,7 +79,7 @@ stabilize 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index 5b91716d..51ee9b22 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -102,7 +102,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2 3)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] @@ -176,7 +176,7 @@ stabilize 1 3 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 5, lastindex: 5, lastterm: 1] restored snapshot [index: 5, term: 1] INFO 3 [commit: 5] restored snapshot [index: 5, term: 1] diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index 7484d7c8..fbd2f930 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -85,7 +85,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index 52e80ae5..e68653ae 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -80,7 +80,7 @@ stabilize 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index ec44fa20..48f5efa4 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -80,7 +80,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index 94892e2b..5e19fb1a 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -133,7 +133,7 @@ stabilize 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false > 4 receiving messages 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 4 switched to configuration voters=(2 3 4)&&(1 2 3) INFO 4 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 4 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/snapshot_succeed_via_app_resp.txt b/testdata/snapshot_succeed_via_app_resp.txt index dbbd5ce1..da2175ba 100644 --- a/testdata/snapshot_succeed_via_app_resp.txt +++ b/testdata/snapshot_succeed_via_app_resp.txt @@ -107,7 +107,7 @@ stabilize 3 ---- > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] + INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 11, lastindex: 11, lastterm: 1] restored snapshot [index: 11, term: 1] INFO 3 [commit: 11] restored snapshot [index: 11, term: 1] From 7302ee6f8351076b10a13aa00fb6200de8693e58 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 18 Oct 2022 19:34:12 -0400 Subject: [PATCH 03/13] raft: track in-progress log application in log This commit adds a mechanism to the raft log struct to track in progress log application that has not yet completed. For now, committed entries are immediately considered applied by `raft.advance`. A future commit will make it possible to accept multiple Ready structs without immediately applying committed entries from earlier Ready structs. This all works towards async Raft log writes, where log application is decoupled from Ready iterations. Signed-off-by: Nathan VanBenschoten --- diff_test.go | 1 + log.go | 31 +++++++++++----- log_test.go | 35 +++++++++++-------- rawnode.go | 7 +++- testdata/confchange_v1_add_single.txt | 2 +- testdata/confchange_v2_add_double_auto.txt | 4 +-- .../confchange_v2_add_double_implicit.txt | 2 +- testdata/confchange_v2_add_single_auto.txt | 2 +- .../confchange_v2_add_single_explicit.txt | 2 +- testdata/confchange_v2_replace_leader.txt | 2 +- testdata/snapshot_succeed_via_app_resp.txt | 2 +- 11 files changed, 58 insertions(+), 32 deletions(-) diff --git a/diff_test.go b/diff_test.go index 60305279..9eef22f2 100644 --- a/diff_test.go +++ b/diff_test.go @@ -57,6 +57,7 @@ func mustTemp(pre, body string) string { func ltoa(l *raftLog) string { s := fmt.Sprintf("lastIndex: %d\n", l.lastIndex()) s += fmt.Sprintf("applied: %d\n", l.applied) + s += fmt.Sprintf("applying: %d\n", l.applying) for i, e := range l.allEntries() { s += fmt.Sprintf("#%d: %+v\n", i, e) } diff --git a/log.go b/log.go index 3359e08c..c8b678b4 100644 --- a/log.go +++ b/log.go @@ -32,8 +32,14 @@ type raftLog struct { // committed is the highest log position that is known to be in // stable storage on a quorum of nodes. committed uint64 + // applying is the highest log position that the application has + // been instructed to apply to its state machine. Some of these + // entries may be in the process of applying and have not yet + // reached applied. + // Invariant: applied <= applying && applying <= committed + applying uint64 // applied is the highest log position that the application has - // been instructed to apply to its state machine. + // successfully applied to its state machine. // Invariant: applied <= committed applied uint64 @@ -76,13 +82,14 @@ func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uin // Initialize our committed and applied pointers to the time of the last compaction. log.committed = firstIndex - 1 log.applied = firstIndex - 1 + log.applying = firstIndex - 1 return log } func (l *raftLog) String() string { - return fmt.Sprintf("committed=%d, applied=%d, unstable.offset=%d, unstable.offsetInProgress=%d, len(unstable.Entries)=%d", - l.committed, l.applied, l.unstable.offset, l.unstable.offsetInProgress, len(l.unstable.entries)) + return fmt.Sprintf("committed=%d, applied=%d, applying=%d, unstable.offset=%d, unstable.offsetInProgress=%d, len(unstable.Entries)=%d", + l.committed, l.applied, l.applying, l.unstable.offset, l.unstable.offsetInProgress, len(l.unstable.entries)) } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, @@ -192,8 +199,9 @@ func (l *raftLog) nextCommittedEnts() (ents []pb.Entry) { // See comment in hasNextCommittedEnts. return nil } - if l.committed > l.applied { - lo, hi := l.applied+1, l.committed+1 // [lo, hi) + if l.committed > l.applying { + lo, hi := l.applying+1, l.committed+1 // [lo, hi) + // TODO: handle pagination correctly. ents, err := l.slice(lo, hi, l.maxNextCommittedEntsSize) if err != nil { l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err) @@ -212,7 +220,7 @@ func (l *raftLog) hasNextCommittedEnts() bool { // first. return false } - return l.committed > l.applied + return l.committed > l.applying } // nextUnstableSnapshot returns the snapshot, if present, that is available to @@ -273,13 +281,18 @@ func (l *raftLog) commitTo(tocommit uint64) { } func (l *raftLog) appliedTo(i uint64) { - if i == 0 { - return - } if l.committed < i || i < l.applied { l.logger.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed) } l.applied = i + l.applying = max(l.applying, i) +} + +func (l *raftLog) acceptApplying(i uint64) { + if l.committed < i { + l.logger.Panicf("applying(%d) is out of range [prevApplying(%d), committed(%d)]", i, l.applying, l.committed) + } + l.applying = i } func (l *raftLog) stableTo(i, t uint64) { l.unstable.stableTo(i, t) } diff --git a/log_test.go b/log_test.go index a5bfc4d8..cb3f75d3 100644 --- a/log_test.go +++ b/log_test.go @@ -308,15 +308,18 @@ func TestHasNextCommittedEnts(t *testing.T) { } tests := []struct { applied uint64 + applying uint64 snap bool whasNext bool }{ - {applied: 0, snap: false, whasNext: true}, - {applied: 3, snap: false, whasNext: true}, - {applied: 4, snap: false, whasNext: true}, - {applied: 5, snap: false, whasNext: false}, + {applied: 3, applying: 3, snap: false, whasNext: true}, + {applied: 3, applying: 4, snap: false, whasNext: true}, + {applied: 3, applying: 5, snap: false, whasNext: false}, + {applied: 4, applying: 4, snap: false, whasNext: true}, + {applied: 4, applying: 5, snap: false, whasNext: false}, + {applied: 5, applying: 5, snap: false, whasNext: false}, // With snapshot. - {applied: 3, snap: true, whasNext: false}, + {applied: 3, applying: 3, snap: true, whasNext: false}, } for i, tt := range tests { t.Run(fmt.Sprint(i), func(t *testing.T) { @@ -327,6 +330,7 @@ func TestHasNextCommittedEnts(t *testing.T) { raftLog.append(ents...) raftLog.maybeCommit(5, 1) raftLog.appliedTo(tt.applied) + raftLog.acceptApplying(tt.applying) if tt.snap { newSnap := snap newSnap.Metadata.Index++ @@ -347,16 +351,19 @@ func TestNextCommittedEnts(t *testing.T) { {Term: 1, Index: 6}, } tests := []struct { - applied uint64 - snap bool - wents []pb.Entry + applied uint64 + applying uint64 + snap bool + wents []pb.Entry }{ - {applied: 0, snap: false, wents: ents[:2]}, - {applied: 3, snap: false, wents: ents[:2]}, - {applied: 4, snap: false, wents: ents[1:2]}, - {applied: 5, snap: false, wents: nil}, + {applied: 3, applying: 3, snap: false, wents: ents[:2]}, + {applied: 3, applying: 4, snap: false, wents: ents[1:2]}, + {applied: 3, applying: 5, snap: false, wents: nil}, + {applied: 4, applying: 4, snap: false, wents: ents[1:2]}, + {applied: 4, applying: 5, snap: false, wents: nil}, + {applied: 5, applying: 5, snap: false, wents: nil}, // With snapshot. - {applied: 3, snap: true, wents: nil}, + {applied: 3, applying: 3, snap: true, wents: nil}, } for i, tt := range tests { t.Run(fmt.Sprint(i), func(t *testing.T) { @@ -367,6 +374,7 @@ func TestNextCommittedEnts(t *testing.T) { raftLog.append(ents...) raftLog.maybeCommit(5, 1) raftLog.appliedTo(tt.applied) + raftLog.acceptApplying(tt.applying) if tt.snap { newSnap := snap newSnap.Metadata.Index++ @@ -374,7 +382,6 @@ func TestNextCommittedEnts(t *testing.T) { } require.Equal(t, tt.wents, raftLog.nextCommittedEnts()) }) - } } diff --git a/rawnode.go b/rawnode.go index dcabdd16..af274ace 100644 --- a/rawnode.go +++ b/rawnode.go @@ -149,8 +149,13 @@ func (rn *RawNode) acceptReady(rd Ready) { } rn.raft.msgs = nil // NB: this does not do anything yet, as entries and snapshots are always - // stabilized on the next Advance. + // stabilized on the next Advance and committed entries are always applied + // by the next Advance. rn.raft.raftLog.acceptUnstable() + if len(rd.CommittedEntries) > 0 { + ents := rd.CommittedEntries + rn.raft.raftLog.acceptApplying(ents[len(ents)-1].Index) + } } // HasReady called when RawNode user need to check if any Ready pending. diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index f33fb310..0419d28d 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -79,7 +79,7 @@ stabilize 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index 51ee9b22..5cf4f24c 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -102,7 +102,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2 3)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] @@ -176,7 +176,7 @@ stabilize 1 3 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 5, lastindex: 5, lastterm: 1] restored snapshot [index: 5, term: 1] INFO 3 [commit: 5] restored snapshot [index: 5, term: 1] diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index fbd2f930..81b3f0de 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -85,7 +85,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index e68653ae..fe127bf5 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -80,7 +80,7 @@ stabilize 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index 48f5efa4..c51900f0 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -80,7 +80,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index 5e19fb1a..8074188d 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -133,7 +133,7 @@ stabilize 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false > 4 receiving messages 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 4 switched to configuration voters=(2 3 4)&&(1 2 3) INFO 4 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 4 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/snapshot_succeed_via_app_resp.txt b/testdata/snapshot_succeed_via_app_resp.txt index da2175ba..a52a683e 100644 --- a/testdata/snapshot_succeed_via_app_resp.txt +++ b/testdata/snapshot_succeed_via_app_resp.txt @@ -107,7 +107,7 @@ stabilize 3 ---- > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 11, lastindex: 11, lastterm: 1] restored snapshot [index: 11, term: 1] INFO 3 [commit: 11] restored snapshot [index: 11, term: 1] From 75d65df5fc15a6a0b5f55122678f0b734e6559fe Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 19 Oct 2022 22:30:08 -0400 Subject: [PATCH 04/13] raft: add allowUnstable param to raftLog.nextCommittedEnts This allows callers to configure whether they want to allow entries that are not already in stable storage to be returned from the method. This will be used in a future commit. Signed-off-by: Nathan VanBenschoten --- log.go | 38 ++++++++++++++++---------- log_test.go | 68 ++++++++++++++++++++++++++++++---------------- node.go | 2 +- raft_paper_test.go | 6 ++-- rawnode.go | 2 +- 5 files changed, 73 insertions(+), 43 deletions(-) diff --git a/log.go b/log.go index c8b678b4..3aad898c 100644 --- a/log.go +++ b/log.go @@ -192,35 +192,45 @@ func (l *raftLog) hasNextUnstableEnts() bool { } // nextCommittedEnts returns all the available entries for execution. -// If applied is smaller than the index of snapshot, it returns all committed -// entries after the index of snapshot. -func (l *raftLog) nextCommittedEnts() (ents []pb.Entry) { +// Entries can be committed even when the local raft instance has not durably +// appended them to the local raft log yet. If allowUnstable is true, committed +// entries from the unstable log may be returned; otherwise, only entries known +// to reside locally on stable storage will be returned. +func (l *raftLog) nextCommittedEnts(allowUnstable bool) (ents []pb.Entry) { if l.hasNextOrInProgressSnapshot() { // See comment in hasNextCommittedEnts. return nil } - if l.committed > l.applying { - lo, hi := l.applying+1, l.committed+1 // [lo, hi) - // TODO: handle pagination correctly. - ents, err := l.slice(lo, hi, l.maxNextCommittedEntsSize) - if err != nil { - l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err) - } - return ents + lo, hi := l.applying+1, l.committed+1 // [lo, hi) + if !allowUnstable { + hi = min(hi, l.unstable.offset) } - return nil + if lo >= hi { + // Nothing to apply. + return nil + } + // TODO: handle pagination correctly. + ents, err := l.slice(lo, hi, l.maxNextCommittedEntsSize) + if err != nil { + l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err) + } + return ents } // hasNextCommittedEnts returns if there is any available entries for execution. // This is a fast check without heavy raftLog.slice() in nextCommittedEnts(). -func (l *raftLog) hasNextCommittedEnts() bool { +func (l *raftLog) hasNextCommittedEnts(allowUnstable bool) bool { if l.hasNextOrInProgressSnapshot() { // If we have a snapshot to apply, don't also return any committed // entries. Doing so raises questions about what should be applied // first. return false } - return l.committed > l.applying + lo, hi := l.applying+1, l.committed+1 // [lo, hi) + if !allowUnstable { + hi = min(hi, l.unstable.offset) + } + return lo < hi } // nextUnstableSnapshot returns the snapshot, if present, that is available to diff --git a/log_test.go b/log_test.go index cb3f75d3..3e03c7a2 100644 --- a/log_test.go +++ b/log_test.go @@ -307,27 +307,37 @@ func TestHasNextCommittedEnts(t *testing.T) { {Term: 1, Index: 6}, } tests := []struct { - applied uint64 - applying uint64 - snap bool - whasNext bool + applied uint64 + applying uint64 + allowUnstable bool + snap bool + whasNext bool }{ - {applied: 3, applying: 3, snap: false, whasNext: true}, - {applied: 3, applying: 4, snap: false, whasNext: true}, - {applied: 3, applying: 5, snap: false, whasNext: false}, - {applied: 4, applying: 4, snap: false, whasNext: true}, - {applied: 4, applying: 5, snap: false, whasNext: false}, - {applied: 5, applying: 5, snap: false, whasNext: false}, + {applied: 3, applying: 3, allowUnstable: true, snap: false, whasNext: true}, + {applied: 3, applying: 4, allowUnstable: true, snap: false, whasNext: true}, + {applied: 3, applying: 5, allowUnstable: true, snap: false, whasNext: false}, + {applied: 4, applying: 4, allowUnstable: true, snap: false, whasNext: true}, + {applied: 4, applying: 5, allowUnstable: true, snap: false, whasNext: false}, + {applied: 5, applying: 5, allowUnstable: true, snap: false, whasNext: false}, + // Don't allow unstable entries. + {applied: 3, applying: 3, allowUnstable: false, snap: false, whasNext: true}, + {applied: 3, applying: 4, allowUnstable: false, snap: false, whasNext: false}, + {applied: 3, applying: 5, allowUnstable: false, snap: false, whasNext: false}, + {applied: 4, applying: 4, allowUnstable: false, snap: false, whasNext: false}, + {applied: 4, applying: 5, allowUnstable: false, snap: false, whasNext: false}, + {applied: 5, applying: 5, allowUnstable: false, snap: false, whasNext: false}, // With snapshot. - {applied: 3, applying: 3, snap: true, whasNext: false}, + {applied: 3, applying: 3, allowUnstable: true, snap: true, whasNext: false}, } for i, tt := range tests { t.Run(fmt.Sprint(i), func(t *testing.T) { storage := NewMemoryStorage() require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) raftLog := newLog(storage, raftLogger) raftLog.append(ents...) + raftLog.stableTo(4, 1) raftLog.maybeCommit(5, 1) raftLog.appliedTo(tt.applied) raftLog.acceptApplying(tt.applying) @@ -336,7 +346,7 @@ func TestHasNextCommittedEnts(t *testing.T) { newSnap.Metadata.Index++ raftLog.restore(newSnap) } - require.Equal(t, tt.whasNext, raftLog.hasNextCommittedEnts()) + require.Equal(t, tt.whasNext, raftLog.hasNextCommittedEnts(tt.allowUnstable)) }) } } @@ -351,27 +361,37 @@ func TestNextCommittedEnts(t *testing.T) { {Term: 1, Index: 6}, } tests := []struct { - applied uint64 - applying uint64 - snap bool - wents []pb.Entry + applied uint64 + applying uint64 + allowUnstable bool + snap bool + wents []pb.Entry }{ - {applied: 3, applying: 3, snap: false, wents: ents[:2]}, - {applied: 3, applying: 4, snap: false, wents: ents[1:2]}, - {applied: 3, applying: 5, snap: false, wents: nil}, - {applied: 4, applying: 4, snap: false, wents: ents[1:2]}, - {applied: 4, applying: 5, snap: false, wents: nil}, - {applied: 5, applying: 5, snap: false, wents: nil}, + {applied: 3, applying: 3, allowUnstable: true, snap: false, wents: ents[:2]}, + {applied: 3, applying: 4, allowUnstable: true, snap: false, wents: ents[1:2]}, + {applied: 3, applying: 5, allowUnstable: true, snap: false, wents: nil}, + {applied: 4, applying: 4, allowUnstable: true, snap: false, wents: ents[1:2]}, + {applied: 4, applying: 5, allowUnstable: true, snap: false, wents: nil}, + {applied: 5, applying: 5, allowUnstable: true, snap: false, wents: nil}, + // Don't allow unstable entries. + {applied: 3, applying: 3, allowUnstable: false, snap: false, wents: ents[:1]}, + {applied: 3, applying: 4, allowUnstable: false, snap: false, wents: nil}, + {applied: 3, applying: 5, allowUnstable: false, snap: false, wents: nil}, + {applied: 4, applying: 4, allowUnstable: false, snap: false, wents: nil}, + {applied: 4, applying: 5, allowUnstable: false, snap: false, wents: nil}, + {applied: 5, applying: 5, allowUnstable: false, snap: false, wents: nil}, // With snapshot. - {applied: 3, applying: 3, snap: true, wents: nil}, + {applied: 3, applying: 3, allowUnstable: true, snap: true, wents: nil}, } for i, tt := range tests { t.Run(fmt.Sprint(i), func(t *testing.T) { storage := NewMemoryStorage() require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) raftLog := newLog(storage, raftLogger) raftLog.append(ents...) + raftLog.stableTo(4, 1) raftLog.maybeCommit(5, 1) raftLog.appliedTo(tt.applied) raftLog.acceptApplying(tt.applying) @@ -380,7 +400,7 @@ func TestNextCommittedEnts(t *testing.T) { newSnap.Metadata.Index++ raftLog.restore(newSnap) } - require.Equal(t, tt.wents, raftLog.nextCommittedEnts()) + require.Equal(t, tt.wents, raftLog.nextCommittedEnts(tt.allowUnstable)) }) } } diff --git a/node.go b/node.go index 8318cc24..8ef1ae05 100644 --- a/node.go +++ b/node.go @@ -562,7 +562,7 @@ func (n *node) ReadIndex(ctx context.Context, rctx []byte) error { func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { rd := Ready{ Entries: r.raftLog.nextUnstableEnts(), - CommittedEntries: r.raftLog.nextCommittedEnts(), + CommittedEntries: r.raftLog.nextCommittedEnts(true /* allowUnstable */), Messages: r.msgs, } if softSt := r.softState(); !softSt.equal(prevSoftSt) { diff --git a/raft_paper_test.go b/raft_paper_test.go index c7124e6e..72942c08 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -452,7 +452,7 @@ func TestLeaderCommitEntry(t *testing.T) { t.Errorf("committed = %d, want %d", g, li+1) } wents := []pb.Entry{{Index: li + 1, Term: 1, Data: []byte("some data")}} - if g := r.raftLog.nextCommittedEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextCommittedEnts(true); !reflect.DeepEqual(g, wents) { t.Errorf("nextCommittedEnts = %+v, want %+v", g, wents) } msgs := r.readMessages() @@ -540,7 +540,7 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) { li := uint64(len(tt)) wents := append(tt, pb.Entry{Term: 3, Index: li + 1}, pb.Entry{Term: 3, Index: li + 2, Data: []byte("some data")}) - if g := r.raftLog.nextCommittedEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextCommittedEnts(true); !reflect.DeepEqual(g, wents) { t.Errorf("#%d: ents = %+v, want %+v", i, g, wents) } } @@ -592,7 +592,7 @@ func TestFollowerCommitEntry(t *testing.T) { t.Errorf("#%d: committed = %d, want %d", i, g, tt.commit) } wents := tt.ents[:int(tt.commit)] - if g := r.raftLog.nextCommittedEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextCommittedEnts(true); !reflect.DeepEqual(g, wents) { t.Errorf("#%d: nextCommittedEnts = %v, want %v", i, g, wents) } } diff --git a/rawnode.go b/rawnode.go index af274ace..82f2b0bf 100644 --- a/rawnode.go +++ b/rawnode.go @@ -170,7 +170,7 @@ func (rn *RawNode) HasReady() bool { if r.raftLog.hasNextUnstableSnapshot() { return true } - if len(r.msgs) > 0 || r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts() { + if len(r.msgs) > 0 || r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(true /* allowUnstable */) { return true } if len(r.readStates) != 0 { From 35cb355aed2069cc84190b30784ba31ca6426b23 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 21 Oct 2022 19:48:44 +0000 Subject: [PATCH 05/13] raft: add new proto fields and message types for async storage writes This commit adds new proto fields and message types for the upcoming async storage writes functionality. These proto changes are not yet used. Signed-off-by: Nathan VanBenschoten --- raftpb/raft.pb.go | 344 ++++++++++++++++++++++++++++++-------------- raftpb/raft.proto | 58 +++++--- raftpb/raft_test.go | 2 +- 3 files changed, 275 insertions(+), 129 deletions(-) diff --git a/raftpb/raft.pb.go b/raftpb/raft.pb.go index 1829dfc5..e1033908 100644 --- a/raftpb/raft.pb.go +++ b/raftpb/raft.pb.go @@ -72,25 +72,29 @@ func (EntryType) EnumDescriptor() ([]byte, []int) { type MessageType int32 const ( - MsgHup MessageType = 0 - MsgBeat MessageType = 1 - MsgProp MessageType = 2 - MsgApp MessageType = 3 - MsgAppResp MessageType = 4 - MsgVote MessageType = 5 - MsgVoteResp MessageType = 6 - MsgSnap MessageType = 7 - MsgHeartbeat MessageType = 8 - MsgHeartbeatResp MessageType = 9 - MsgUnreachable MessageType = 10 - MsgSnapStatus MessageType = 11 - MsgCheckQuorum MessageType = 12 - MsgTransferLeader MessageType = 13 - MsgTimeoutNow MessageType = 14 - MsgReadIndex MessageType = 15 - MsgReadIndexResp MessageType = 16 - MsgPreVote MessageType = 17 - MsgPreVoteResp MessageType = 18 + MsgHup MessageType = 0 + MsgBeat MessageType = 1 + MsgProp MessageType = 2 + MsgApp MessageType = 3 + MsgAppResp MessageType = 4 + MsgVote MessageType = 5 + MsgVoteResp MessageType = 6 + MsgSnap MessageType = 7 + MsgHeartbeat MessageType = 8 + MsgHeartbeatResp MessageType = 9 + MsgUnreachable MessageType = 10 + MsgSnapStatus MessageType = 11 + MsgCheckQuorum MessageType = 12 + MsgTransferLeader MessageType = 13 + MsgTimeoutNow MessageType = 14 + MsgReadIndex MessageType = 15 + MsgReadIndexResp MessageType = 16 + MsgPreVote MessageType = 17 + MsgPreVoteResp MessageType = 18 + MsgStorageAppend MessageType = 19 + MsgStorageAppendResp MessageType = 20 + MsgStorageApply MessageType = 21 + MsgStorageApplyResp MessageType = 22 ) var MessageType_name = map[int32]string{ @@ -113,28 +117,36 @@ var MessageType_name = map[int32]string{ 16: "MsgReadIndexResp", 17: "MsgPreVote", 18: "MsgPreVoteResp", + 19: "MsgStorageAppend", + 20: "MsgStorageAppendResp", + 21: "MsgStorageApply", + 22: "MsgStorageApplyResp", } var MessageType_value = map[string]int32{ - "MsgHup": 0, - "MsgBeat": 1, - "MsgProp": 2, - "MsgApp": 3, - "MsgAppResp": 4, - "MsgVote": 5, - "MsgVoteResp": 6, - "MsgSnap": 7, - "MsgHeartbeat": 8, - "MsgHeartbeatResp": 9, - "MsgUnreachable": 10, - "MsgSnapStatus": 11, - "MsgCheckQuorum": 12, - "MsgTransferLeader": 13, - "MsgTimeoutNow": 14, - "MsgReadIndex": 15, - "MsgReadIndexResp": 16, - "MsgPreVote": 17, - "MsgPreVoteResp": 18, + "MsgHup": 0, + "MsgBeat": 1, + "MsgProp": 2, + "MsgApp": 3, + "MsgAppResp": 4, + "MsgVote": 5, + "MsgVoteResp": 6, + "MsgSnap": 7, + "MsgHeartbeat": 8, + "MsgHeartbeatResp": 9, + "MsgUnreachable": 10, + "MsgSnapStatus": 11, + "MsgCheckQuorum": 12, + "MsgTransferLeader": 13, + "MsgTimeoutNow": 14, + "MsgReadIndex": 15, + "MsgReadIndexResp": 16, + "MsgPreVote": 17, + "MsgPreVoteResp": 18, + "MsgStorageAppend": 19, + "MsgStorageAppendResp": 20, + "MsgStorageApply": 21, + "MsgStorageApplyResp": 22, } func (x MessageType) Enum() *MessageType { @@ -386,14 +398,20 @@ type Message struct { From uint64 `protobuf:"varint,3,opt,name=from" json:"from"` Term uint64 `protobuf:"varint,4,opt,name=term" json:"term"` // logTerm is generally used for appending Raft logs to followers. For example, - // (type=MsgApp,index=100,logTerm=5) means leader appends entries starting at - // index=101, and the term of entry at index 100 is 5. + // (type=MsgApp,index=100,logTerm=5) means the leader appends entries starting + // at index=101, and the term of the entry at index 100 is 5. // (type=MsgAppResp,reject=true,index=100,logTerm=5) means follower rejects some // entries from its leader as it already has an entry with term 5 at index 100. + // (type=MsgStorageAppendResp,index=100,logTerm=5) means the local node wrote + // entries up to index=100 in stable storage, and the term of the entry at index + // 100 was 5. LogTerm uint64 `protobuf:"varint,5,opt,name=logTerm" json:"logTerm"` Index uint64 `protobuf:"varint,6,opt,name=index" json:"index"` Entries []Entry `protobuf:"bytes,7,rep,name=entries" json:"entries"` Commit uint64 `protobuf:"varint,8,opt,name=commit" json:"commit"` + // hardState can be non-nil for MsgStorageAppend messages and is nil for all + // other message types. + HardState *HardState `protobuf:"bytes,13,opt,name=hardState" json:"hardState,omitempty"` // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a // non-nil, empty value for the snapshot field of non-MsgSnap messages. Code @@ -402,6 +420,10 @@ type Message struct { Reject bool `protobuf:"varint,10,opt,name=reject" json:"reject"` RejectHint uint64 `protobuf:"varint,11,opt,name=rejectHint" json:"rejectHint"` Context []byte `protobuf:"bytes,12,opt,name=context" json:"context,omitempty"` + // responses are populated by a raft node to instruct storage threads on how + // to respond and who to respond to when the work associated with a message + // is complete. Populated for MsgStorageAppend and MsgStorageApply messages. + Responses []Message `protobuf:"bytes,14,rep,name=responses" json:"responses"` } func (m *Message) Reset() { *m = Message{} } @@ -699,72 +721,76 @@ func init() { func init() { proto.RegisterFile("raft.proto", fileDescriptor_b042552c306ae59b) } var fileDescriptor_b042552c306ae59b = []byte{ - // 1028 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcd, 0x6e, 0xdb, 0x46, - 0x17, 0xe5, 0x50, 0xb4, 0x7e, 0xae, 0x64, 0x79, 0x7c, 0xe3, 0x2f, 0x20, 0x0c, 0x43, 0xd1, 0xa7, - 0xa4, 0x88, 0xe0, 0x22, 0x6e, 0xa1, 0x45, 0x51, 0x74, 0xe7, 0x9f, 0x00, 0x76, 0x61, 0xb9, 0xa9, - 0xec, 0x78, 0x51, 0xa0, 0x30, 0xc6, 0xe2, 0x88, 0x66, 0x2b, 0x72, 0x08, 0x72, 0xe4, 0xda, 0x9b, - 0xa2, 0xe8, 0x13, 0x74, 0xd9, 0x4d, 0xb6, 0x7d, 0x80, 0x3e, 0x85, 0x97, 0x06, 0xba, 0xe9, 0x2a, - 0x68, 0xec, 0x17, 0x29, 0x66, 0x38, 0x94, 0x28, 0xd9, 0xc8, 0xa2, 0xbb, 0x99, 0x73, 0xcf, 0xdc, - 0x39, 0xe7, 0xde, 0xcb, 0x21, 0x40, 0xc2, 0x46, 0x72, 0x2b, 0x4e, 0x84, 0x14, 0x58, 0x56, 0xeb, - 0xf8, 0x7c, 0x7d, 0xcd, 0x17, 0xbe, 0xd0, 0xd0, 0x67, 0x6a, 0x95, 0x45, 0x3b, 0x3f, 0xc3, 0xd2, - 0xeb, 0x48, 0x26, 0xd7, 0xe8, 0x82, 0x73, 0xc2, 0x93, 0xd0, 0xb5, 0xdb, 0xa4, 0xeb, 0xec, 0x38, - 0x37, 0xef, 0x9f, 0x59, 0x03, 0x8d, 0xe0, 0x3a, 0x2c, 0x1d, 0x44, 0x1e, 0xbf, 0x72, 0x4b, 0x85, - 0x50, 0x06, 0xe1, 0xa7, 0xe0, 0x9c, 0x5c, 0xc7, 0xdc, 0x25, 0x6d, 0xd2, 0x6d, 0xf6, 0x56, 0xb7, - 0xb2, 0xbb, 0xb6, 0x74, 0x4a, 0x15, 0x98, 0x26, 0xba, 0x8e, 0x39, 0x22, 0x38, 0x7b, 0x4c, 0x32, - 0xd7, 0x69, 0x93, 0x6e, 0x63, 0xa0, 0xd7, 0x9d, 0x5f, 0x08, 0xd0, 0xe3, 0x88, 0xc5, 0xe9, 0x85, - 0x90, 0x7d, 0x2e, 0x99, 0xc7, 0x24, 0xc3, 0x2f, 0x00, 0x86, 0x22, 0x1a, 0x9d, 0xa5, 0x92, 0xc9, - 0x2c, 0x77, 0x7d, 0x96, 0x7b, 0x57, 0x44, 0xa3, 0x63, 0x15, 0x30, 0xb9, 0x6b, 0xc3, 0x1c, 0x50, - 0x4a, 0x03, 0xad, 0xb4, 0x68, 0x22, 0x83, 0x94, 0x3f, 0xa9, 0xfc, 0x15, 0x4d, 0x68, 0xa4, 0xf3, - 0x1d, 0x54, 0x73, 0x05, 0x4a, 0xa2, 0x52, 0xa0, 0xef, 0x6c, 0x0c, 0xf4, 0x1a, 0xbf, 0x82, 0x6a, - 0x68, 0x94, 0xe9, 0xc4, 0xf5, 0x9e, 0x9b, 0x6b, 0x59, 0x54, 0x6e, 0xf2, 0x4e, 0xf9, 0x9d, 0x77, - 0x25, 0xa8, 0xf4, 0x79, 0x9a, 0x32, 0x9f, 0xe3, 0x2b, 0x70, 0xe4, 0xac, 0x56, 0x4f, 0xf2, 0x1c, - 0x26, 0x5c, 0xac, 0x96, 0xa2, 0xe1, 0x1a, 0xd8, 0x52, 0xcc, 0x39, 0xb1, 0xa5, 0x50, 0x36, 0x46, - 0x89, 0x58, 0xb0, 0xa1, 0x90, 0xa9, 0x41, 0x67, 0xd1, 0x20, 0xb6, 0xa0, 0x32, 0x16, 0xbe, 0xee, - 0xee, 0x52, 0x21, 0x98, 0x83, 0xb3, 0xb2, 0x95, 0x1f, 0x96, 0xed, 0x15, 0x54, 0x78, 0x24, 0x93, - 0x80, 0xa7, 0x6e, 0xa5, 0x5d, 0xea, 0xd6, 0x7b, 0xcb, 0x73, 0x3d, 0xce, 0x53, 0x19, 0x0e, 0x6e, - 0x40, 0x79, 0x28, 0xc2, 0x30, 0x90, 0x6e, 0xb5, 0x90, 0xcb, 0x60, 0xd8, 0x83, 0x6a, 0x6a, 0x2a, - 0xe6, 0xd6, 0x74, 0x25, 0xe9, 0x62, 0x25, 0xf5, 0x09, 0x32, 0x98, 0xf2, 0x54, 0xc6, 0x84, 0xff, - 0xc0, 0x87, 0xd2, 0x85, 0x36, 0xe9, 0x56, 0xf3, 0x8c, 0x19, 0x86, 0x2f, 0x00, 0xb2, 0xd5, 0x7e, - 0x10, 0x49, 0xb7, 0x5e, 0xb8, 0xb3, 0x80, 0xa3, 0x0b, 0x95, 0xa1, 0x88, 0x24, 0xbf, 0x92, 0x6e, - 0x43, 0x37, 0x36, 0xdf, 0x76, 0xbe, 0x87, 0xda, 0x3e, 0x4b, 0xbc, 0x6c, 0x7c, 0xf2, 0x0a, 0x92, - 0x07, 0x15, 0x74, 0xc1, 0xb9, 0x14, 0x92, 0xcf, 0x7f, 0x1c, 0x0a, 0x29, 0x18, 0x2e, 0x3d, 0x34, - 0xdc, 0xf9, 0x93, 0x40, 0x6d, 0x3a, 0xaf, 0xf8, 0x14, 0xca, 0xea, 0x4c, 0x92, 0xba, 0xa4, 0x5d, - 0xea, 0x3a, 0x03, 0xb3, 0xc3, 0x75, 0xa8, 0x8e, 0x39, 0x4b, 0x22, 0x15, 0xb1, 0x75, 0x64, 0xba, - 0xc7, 0x97, 0xb0, 0x92, 0xb1, 0xce, 0xc4, 0x44, 0xfa, 0x22, 0x88, 0x7c, 0xb7, 0xa4, 0x29, 0xcd, - 0x0c, 0xfe, 0xc6, 0xa0, 0xf8, 0x1c, 0x96, 0xf3, 0x43, 0x67, 0x91, 0x72, 0xea, 0x68, 0x5a, 0x23, - 0x07, 0x8f, 0xf8, 0x95, 0xc4, 0xe7, 0x00, 0x6c, 0x22, 0xc5, 0xd9, 0x98, 0xb3, 0x4b, 0xae, 0x87, - 0x21, 0x2f, 0x68, 0x4d, 0xe1, 0x87, 0x0a, 0xee, 0xbc, 0x23, 0x00, 0x4a, 0xf4, 0xee, 0x05, 0x8b, - 0x7c, 0x8e, 0x9f, 0x9b, 0xb1, 0xb5, 0xf5, 0xd8, 0x3e, 0x2d, 0x7e, 0x86, 0x19, 0xe3, 0xc1, 0xe4, - 0xbe, 0x84, 0x4a, 0x24, 0x3c, 0x7e, 0x16, 0x78, 0xa6, 0x28, 0x4d, 0x15, 0xbc, 0x7b, 0xff, 0xac, - 0x7c, 0x24, 0x3c, 0x7e, 0xb0, 0x37, 0x28, 0xab, 0xf0, 0x81, 0x57, 0xec, 0x8b, 0x33, 0xd7, 0x17, - 0x5c, 0x07, 0x3b, 0xf0, 0x4c, 0x23, 0xc0, 0x9c, 0xb6, 0x0f, 0xf6, 0x06, 0x76, 0xe0, 0x75, 0x42, - 0xa0, 0xb3, 0xcb, 0x8f, 0x83, 0xc8, 0x1f, 0xcf, 0x44, 0x92, 0xff, 0x22, 0xd2, 0xfe, 0x98, 0xc8, - 0xce, 0x1f, 0x04, 0x1a, 0xb3, 0x3c, 0xa7, 0x3d, 0xdc, 0x01, 0x90, 0x09, 0x8b, 0xd2, 0x40, 0x06, - 0x22, 0x32, 0x37, 0x6e, 0x3c, 0x72, 0xe3, 0x94, 0x93, 0x4f, 0xe4, 0xec, 0x14, 0x7e, 0x09, 0x95, - 0xa1, 0x66, 0x65, 0x1d, 0x2f, 0x3c, 0x29, 0x8b, 0xd6, 0xf2, 0x2f, 0xcc, 0xd0, 0x8b, 0x35, 0x2b, - 0xcd, 0xd5, 0x6c, 0x73, 0x1f, 0x6a, 0xd3, 0x77, 0x17, 0x57, 0xa0, 0xae, 0x37, 0x47, 0x22, 0x09, - 0xd9, 0x98, 0x5a, 0xf8, 0x04, 0x56, 0x34, 0x30, 0xcb, 0x4f, 0x09, 0xfe, 0x0f, 0x56, 0x17, 0xc0, - 0xd3, 0x1e, 0xb5, 0x37, 0xff, 0xb2, 0xa1, 0x5e, 0x78, 0x96, 0x10, 0xa0, 0xdc, 0x4f, 0xfd, 0xfd, - 0x49, 0x4c, 0x2d, 0xac, 0x43, 0xa5, 0x9f, 0xfa, 0x3b, 0x9c, 0x49, 0x4a, 0xcc, 0xe6, 0x4d, 0x22, - 0x62, 0x6a, 0x1b, 0xd6, 0x76, 0x1c, 0xd3, 0x12, 0x36, 0x01, 0xb2, 0xf5, 0x80, 0xa7, 0x31, 0x75, - 0x0c, 0xf1, 0x54, 0x48, 0x4e, 0x97, 0x94, 0x36, 0xb3, 0xd1, 0xd1, 0xb2, 0x89, 0xaa, 0x27, 0x80, - 0x56, 0x90, 0x42, 0x43, 0x5d, 0xc6, 0x59, 0x22, 0xcf, 0xd5, 0x2d, 0x55, 0x5c, 0x03, 0x5a, 0x44, - 0xf4, 0xa1, 0x1a, 0x22, 0x34, 0xfb, 0xa9, 0xff, 0x36, 0x4a, 0x38, 0x1b, 0x5e, 0xb0, 0xf3, 0x31, - 0xa7, 0x80, 0xab, 0xb0, 0x6c, 0x12, 0xa9, 0x2f, 0x6e, 0x92, 0xd2, 0xba, 0xa1, 0xed, 0x5e, 0xf0, - 0xe1, 0x8f, 0xdf, 0x4e, 0x44, 0x32, 0x09, 0x69, 0x43, 0xd9, 0xee, 0xa7, 0xbe, 0x6e, 0xd0, 0x88, - 0x27, 0x87, 0x9c, 0x79, 0x3c, 0xa1, 0xcb, 0xe6, 0xf4, 0x49, 0x10, 0x72, 0x31, 0x91, 0x47, 0xe2, - 0x27, 0xda, 0x34, 0x62, 0x06, 0x9c, 0x79, 0xfa, 0x7f, 0x47, 0x57, 0x8c, 0x98, 0x29, 0xa2, 0xc5, - 0x50, 0xe3, 0xf7, 0x4d, 0xc2, 0xb5, 0xc5, 0x55, 0x73, 0xab, 0xd9, 0x6b, 0x0e, 0x6e, 0xfe, 0x4a, - 0x60, 0xed, 0xb1, 0xf1, 0xc0, 0x0d, 0x70, 0x1f, 0xc3, 0xb7, 0x27, 0x52, 0x50, 0x0b, 0x3f, 0x81, - 0xff, 0x3f, 0x16, 0xfd, 0x5a, 0x04, 0x91, 0x3c, 0x08, 0xe3, 0x71, 0x30, 0x0c, 0x54, 0x2b, 0x3e, - 0x46, 0x7b, 0x7d, 0x65, 0x68, 0xf6, 0xe6, 0x35, 0x34, 0xe7, 0x3f, 0x0a, 0x55, 0x8c, 0x19, 0xb2, - 0xed, 0x79, 0x6a, 0xfc, 0xa9, 0x85, 0x6e, 0x51, 0xec, 0x80, 0x87, 0xe2, 0x92, 0xeb, 0x08, 0x99, - 0x8f, 0xbc, 0x8d, 0x3d, 0x26, 0xb3, 0x88, 0x3d, 0x6f, 0x64, 0xdb, 0xf3, 0x0e, 0xb3, 0xb7, 0x47, - 0x47, 0x4b, 0x3b, 0x2f, 0x6e, 0x3e, 0xb4, 0xac, 0xdb, 0x0f, 0x2d, 0xeb, 0xe6, 0xae, 0x45, 0x6e, - 0xef, 0x5a, 0xe4, 0x9f, 0xbb, 0x16, 0xf9, 0xed, 0xbe, 0x65, 0xfd, 0x7e, 0xdf, 0xb2, 0x6e, 0xef, - 0x5b, 0xd6, 0xdf, 0xf7, 0x2d, 0xeb, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc9, 0x35, 0x94, 0xd2, - 0xbb, 0x08, 0x00, 0x00, + // 1100 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcd, 0x6e, 0x23, 0x45, + 0x17, 0xed, 0x6e, 0x77, 0xfc, 0x73, 0xed, 0x38, 0x95, 0x8a, 0x27, 0x5f, 0x2b, 0x8a, 0x3c, 0xfe, + 0x3c, 0x83, 0xc6, 0x0a, 0x9a, 0x80, 0x8c, 0x40, 0x88, 0x5d, 0x7e, 0x46, 0x4a, 0x50, 0x1c, 0x06, + 0x27, 0x93, 0x05, 0x12, 0x8a, 0x2a, 0xee, 0x4a, 0xa7, 0xc1, 0xae, 0x6a, 0x55, 0x97, 0x43, 0xb2, + 0x41, 0x88, 0x17, 0x80, 0x25, 0x1b, 0xb6, 0xec, 0xe1, 0x29, 0xb2, 0xcc, 0x92, 0xd5, 0x88, 0x49, + 0x5e, 0x04, 0x55, 0x75, 0xf5, 0x8f, 0x9d, 0x68, 0x16, 0xec, 0xaa, 0xce, 0x39, 0x75, 0xeb, 0xdc, + 0x7b, 0xfb, 0x56, 0x03, 0x08, 0x72, 0x2e, 0x37, 0x23, 0xc1, 0x25, 0xc7, 0x65, 0xb5, 0x8e, 0xce, + 0xd6, 0x5a, 0x01, 0x0f, 0xb8, 0x86, 0x3e, 0x52, 0xab, 0x84, 0xed, 0xfe, 0x08, 0x0b, 0xaf, 0x98, + 0x14, 0xd7, 0xd8, 0x03, 0xf7, 0x98, 0x8a, 0x89, 0xe7, 0x74, 0xec, 0x9e, 0xbb, 0xed, 0xde, 0xbc, + 0x7d, 0x6a, 0x0d, 0x35, 0x82, 0xd7, 0x60, 0x61, 0x9f, 0xf9, 0xf4, 0xca, 0x2b, 0x15, 0xa8, 0x04, + 0xc2, 0x1f, 0x82, 0x7b, 0x7c, 0x1d, 0x51, 0xcf, 0xee, 0xd8, 0xbd, 0x66, 0x7f, 0x79, 0x33, 0xb9, + 0x6b, 0x53, 0x87, 0x54, 0x44, 0x16, 0xe8, 0x3a, 0xa2, 0x18, 0x83, 0xbb, 0x4b, 0x24, 0xf1, 0xdc, + 0x8e, 0xdd, 0x6b, 0x0c, 0xf5, 0xba, 0xfb, 0x93, 0x0d, 0xe8, 0x88, 0x91, 0x28, 0xbe, 0xe0, 0x72, + 0x40, 0x25, 0xf1, 0x89, 0x24, 0xf8, 0x33, 0x80, 0x11, 0x67, 0xe7, 0xa7, 0xb1, 0x24, 0x32, 0x89, + 0x5d, 0xcf, 0x63, 0xef, 0x70, 0x76, 0x7e, 0xa4, 0x08, 0x13, 0xbb, 0x36, 0x4a, 0x01, 0xe5, 0x34, + 0xd4, 0x4e, 0x8b, 0x49, 0x24, 0x90, 0xca, 0x4f, 0xaa, 0xfc, 0x8a, 0x49, 0x68, 0xa4, 0xfb, 0x0d, + 0x54, 0x53, 0x07, 0xca, 0xa2, 0x72, 0xa0, 0xef, 0x6c, 0x0c, 0xf5, 0x1a, 0x7f, 0x01, 0xd5, 0x89, + 0x71, 0xa6, 0x03, 0xd7, 0xfb, 0x5e, 0xea, 0x65, 0xde, 0xb9, 0x89, 0x9b, 0xe9, 0xbb, 0xbf, 0xb8, + 0x50, 0x19, 0xd0, 0x38, 0x26, 0x01, 0xc5, 0x2f, 0xc1, 0x95, 0x79, 0xad, 0x56, 0xd2, 0x18, 0x86, + 0x2e, 0x56, 0x4b, 0xc9, 0x70, 0x0b, 0x1c, 0xc9, 0x67, 0x32, 0x71, 0x24, 0x57, 0x69, 0x9c, 0x0b, + 0x3e, 0x97, 0x86, 0x42, 0xb2, 0x04, 0xdd, 0xf9, 0x04, 0x71, 0x1b, 0x2a, 0x63, 0x1e, 0xe8, 0xee, + 0x2e, 0x14, 0xc8, 0x14, 0xcc, 0xcb, 0x56, 0x7e, 0x58, 0xb6, 0x97, 0x50, 0xa1, 0x4c, 0x8a, 0x90, + 0xc6, 0x5e, 0xa5, 0x53, 0xea, 0xd5, 0xfb, 0x8b, 0x33, 0x3d, 0x4e, 0x43, 0x19, 0x0d, 0x5e, 0x87, + 0xf2, 0x88, 0x4f, 0x26, 0xa1, 0xf4, 0xaa, 0x85, 0x58, 0x06, 0xc3, 0x9f, 0x42, 0xed, 0x82, 0x08, + 0x5f, 0x37, 0xcb, 0x5b, 0x9c, 0x6d, 0xeb, 0x5e, 0x4a, 0xe8, 0x33, 0xf6, 0x30, 0x57, 0xe2, 0x3e, + 0x54, 0x63, 0x53, 0x68, 0xaf, 0xa6, 0x4f, 0xa1, 0xf9, 0x06, 0x98, 0x43, 0x99, 0x4e, 0x19, 0x11, + 0xf4, 0x3b, 0x3a, 0x92, 0x1e, 0x74, 0xec, 0x5e, 0x35, 0x35, 0x92, 0x60, 0xf8, 0x39, 0x40, 0xb2, + 0xda, 0x0b, 0x99, 0xf4, 0xea, 0x05, 0xab, 0x05, 0x5c, 0xd5, 0x6d, 0xc4, 0x99, 0xa4, 0x57, 0xd2, + 0x6b, 0xa8, 0xef, 0xc1, 0x5c, 0x92, 0x82, 0xf8, 0x13, 0xa8, 0x09, 0x1a, 0x47, 0x9c, 0xc5, 0x34, + 0xf6, 0x9a, 0xba, 0x3a, 0x4b, 0x73, 0x5d, 0x4d, 0xbf, 0xd1, 0x4c, 0xd7, 0xfd, 0x16, 0x6a, 0x59, + 0xaa, 0x59, 0xcf, 0xec, 0x07, 0x3d, 0xf3, 0xc0, 0xbd, 0xe4, 0x92, 0xce, 0x8e, 0xa3, 0x42, 0x0a, + 0x25, 0x2e, 0x3d, 0x2c, 0x71, 0xf7, 0x2f, 0x1b, 0x6a, 0xd9, 0x84, 0xe0, 0x55, 0x28, 0xab, 0x33, + 0x22, 0xf6, 0xec, 0x4e, 0xa9, 0xe7, 0x0e, 0xcd, 0x0e, 0xaf, 0x41, 0x75, 0x4c, 0x89, 0x60, 0x8a, + 0x71, 0x34, 0x93, 0xed, 0xf1, 0x0b, 0x58, 0x4a, 0x54, 0xa7, 0x7c, 0x2a, 0x03, 0x1e, 0xb2, 0xc0, + 0x2b, 0x69, 0x49, 0x33, 0x81, 0xbf, 0x32, 0x28, 0x7e, 0x06, 0x8b, 0xe9, 0xa1, 0x53, 0xa6, 0x8a, + 0xe4, 0x6a, 0x59, 0x23, 0x05, 0x0f, 0x55, 0x8d, 0x9e, 0x01, 0x90, 0xa9, 0xe4, 0xa7, 0x63, 0x4a, + 0x2e, 0xa9, 0xfe, 0xfc, 0xd2, 0x5e, 0xd4, 0x14, 0x7e, 0xa0, 0xe0, 0xee, 0xef, 0x36, 0x80, 0x32, + 0xbd, 0x73, 0x41, 0x58, 0x40, 0xf1, 0xc7, 0x66, 0x50, 0x1c, 0x3d, 0x28, 0xab, 0xc5, 0xc1, 0x4f, + 0x14, 0x0f, 0x66, 0xe5, 0x05, 0x54, 0x18, 0xf7, 0xe9, 0x69, 0xe8, 0x9b, 0xa2, 0x34, 0x15, 0x79, + 0xf7, 0xf6, 0x69, 0xf9, 0x90, 0xfb, 0x74, 0x7f, 0x77, 0x58, 0x56, 0xf4, 0xbe, 0x8f, 0xbd, 0xbc, + 0xa5, 0xc9, 0x2b, 0x94, 0x35, 0x73, 0x0d, 0x9c, 0xd0, 0x37, 0x8d, 0x00, 0x73, 0xda, 0xd9, 0xdf, + 0x1d, 0x3a, 0xa1, 0xdf, 0x9d, 0x00, 0xca, 0x2f, 0x3f, 0x0a, 0x59, 0x30, 0xce, 0x4d, 0xda, 0xff, + 0xc5, 0xa4, 0xf3, 0x3e, 0x93, 0xdd, 0x3f, 0x6c, 0x68, 0xe4, 0x71, 0x4e, 0xfa, 0x78, 0x1b, 0x40, + 0x0a, 0xc2, 0xe2, 0x50, 0x86, 0x9c, 0x99, 0x1b, 0xd7, 0x1f, 0xb9, 0x31, 0xd3, 0xa4, 0x1f, 0x73, + 0x7e, 0x0a, 0x7f, 0x0e, 0x95, 0x91, 0x56, 0x25, 0x1d, 0x2f, 0x3c, 0x62, 0xf3, 0xa9, 0xa5, 0x33, + 0x6d, 0xe4, 0xc5, 0x9a, 0x95, 0x66, 0x6a, 0xb6, 0xb1, 0x07, 0xb5, 0xec, 0xa5, 0xc7, 0x4b, 0x50, + 0xd7, 0x9b, 0x43, 0x2e, 0x26, 0x64, 0x8c, 0x2c, 0xbc, 0x02, 0x4b, 0x1a, 0xc8, 0xe3, 0x23, 0x1b, + 0x3f, 0x81, 0xe5, 0x39, 0xf0, 0xa4, 0x8f, 0x9c, 0x8d, 0x3f, 0x4b, 0x50, 0x2f, 0x3c, 0x84, 0x18, + 0xa0, 0x3c, 0x88, 0x83, 0xbd, 0x69, 0x84, 0x2c, 0x5c, 0x87, 0xca, 0x20, 0x0e, 0xb6, 0x29, 0x91, + 0xc8, 0x36, 0x9b, 0xd7, 0x82, 0x47, 0xc8, 0x31, 0xaa, 0xad, 0x28, 0x42, 0x25, 0xdc, 0x04, 0x48, + 0xd6, 0x43, 0x1a, 0x47, 0xc8, 0x35, 0xc2, 0x13, 0x2e, 0x29, 0x5a, 0x50, 0xde, 0xcc, 0x46, 0xb3, + 0x65, 0xc3, 0xaa, 0xd7, 0x03, 0x55, 0x30, 0x82, 0x86, 0xba, 0x8c, 0x12, 0x21, 0xcf, 0xd4, 0x2d, + 0x55, 0xdc, 0x02, 0x54, 0x44, 0xf4, 0xa1, 0x1a, 0xc6, 0xd0, 0x1c, 0xc4, 0xc1, 0x1b, 0x26, 0x28, + 0x19, 0x5d, 0x90, 0xb3, 0x31, 0x45, 0x80, 0x97, 0x61, 0xd1, 0x04, 0x52, 0x13, 0x37, 0x8d, 0x51, + 0xdd, 0xc8, 0x76, 0x2e, 0xe8, 0xe8, 0xfb, 0xaf, 0xa7, 0x5c, 0x4c, 0x27, 0xa8, 0xa1, 0xd2, 0x1e, + 0xc4, 0x81, 0x6e, 0xd0, 0x39, 0x15, 0x07, 0x94, 0xf8, 0x54, 0xa0, 0x45, 0x73, 0xfa, 0x38, 0x9c, + 0x50, 0x3e, 0x95, 0x87, 0xfc, 0x07, 0xd4, 0x34, 0x66, 0x86, 0x94, 0xf8, 0xfa, 0x0f, 0x8b, 0x96, + 0x8c, 0x99, 0x0c, 0xd1, 0x66, 0x90, 0xc9, 0xf7, 0xb5, 0xa0, 0x3a, 0xc5, 0x65, 0x73, 0xab, 0xd9, + 0x6b, 0x0d, 0x36, 0x27, 0x8f, 0x24, 0x17, 0x24, 0xa0, 0x5b, 0x51, 0x44, 0x99, 0x8f, 0x56, 0xb0, + 0x07, 0xad, 0x79, 0x54, 0xeb, 0x5b, 0xaa, 0x63, 0x33, 0xcc, 0xf8, 0x1a, 0x3d, 0xc1, 0xff, 0x83, + 0x95, 0x39, 0x50, 0xab, 0x57, 0x37, 0x7e, 0xb6, 0xa1, 0xf5, 0xd8, 0xc7, 0x87, 0xd7, 0xc1, 0x7b, + 0x0c, 0xdf, 0x9a, 0x4a, 0x8e, 0x2c, 0xfc, 0x01, 0xfc, 0xff, 0x31, 0xf6, 0x4b, 0x1e, 0x32, 0xb9, + 0x3f, 0x89, 0xc6, 0xe1, 0x28, 0x54, 0x8d, 0x7e, 0x9f, 0xec, 0xd5, 0x95, 0x91, 0x39, 0x1b, 0xd7, + 0xd0, 0x9c, 0x1d, 0x39, 0x55, 0xea, 0x1c, 0xd9, 0xf2, 0x7d, 0x35, 0x5c, 0xc8, 0x52, 0x59, 0xe7, + 0xf0, 0x90, 0x4e, 0xf8, 0x25, 0xd5, 0x8c, 0x3d, 0xcb, 0xbc, 0x89, 0x7c, 0x22, 0x13, 0xc6, 0x99, + 0x4d, 0x64, 0xcb, 0xf7, 0x0f, 0x92, 0x97, 0x4d, 0xb3, 0xa5, 0xed, 0xe7, 0x37, 0xef, 0xda, 0xd6, + 0xed, 0xbb, 0xb6, 0x75, 0x73, 0xd7, 0xb6, 0x6f, 0xef, 0xda, 0xf6, 0x3f, 0x77, 0x6d, 0xfb, 0xd7, + 0xfb, 0xb6, 0xf5, 0xdb, 0x7d, 0xdb, 0xba, 0xbd, 0x6f, 0x5b, 0x7f, 0xdf, 0xb7, 0xad, 0x7f, 0x03, + 0x00, 0x00, 0xff, 0xff, 0x17, 0x42, 0x8c, 0x08, 0x8b, 0x09, 0x00, 0x00, } func (m *Entry) Marshal() (dAtA []byte, err error) { @@ -905,6 +931,32 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.Responses) > 0 { + for iNdEx := len(m.Responses) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Responses[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRaft(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x72 + } + } + if m.HardState != nil { + { + size, err := m.HardState.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRaft(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x6a + } if m.Context != nil { i -= len(m.Context) copy(dAtA[i:], m.Context) @@ -1263,6 +1315,16 @@ func (m *Message) Size() (n int) { l = len(m.Context) n += 1 + l + sovRaft(uint64(l)) } + if m.HardState != nil { + l = m.HardState.Size() + n += 1 + l + sovRaft(uint64(l)) + } + if len(m.Responses) > 0 { + for _, e := range m.Responses { + l = e.Size() + n += 1 + l + sovRaft(uint64(l)) + } + } return n } @@ -2045,6 +2107,76 @@ func (m *Message) Unmarshal(dAtA []byte) error { m.Context = []byte{} } iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HardState", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRaft + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HardState == nil { + m.HardState = &HardState{} + } + if err := m.HardState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRaft + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Responses = append(m.Responses, Message{}) + if err := m.Responses[len(m.Responses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) diff --git a/raftpb/raft.proto b/raftpb/raft.proto index b3b49e58..35c763f1 100644 --- a/raftpb/raft.proto +++ b/raftpb/raft.proto @@ -41,25 +41,29 @@ message Snapshot { // For description of different message types, see: // https://pkg.go.dev/go.etcd.io/raft/v3#hdr-MessageType enum MessageType { - MsgHup = 0; - MsgBeat = 1; - MsgProp = 2; - MsgApp = 3; - MsgAppResp = 4; - MsgVote = 5; - MsgVoteResp = 6; - MsgSnap = 7; - MsgHeartbeat = 8; - MsgHeartbeatResp = 9; - MsgUnreachable = 10; - MsgSnapStatus = 11; - MsgCheckQuorum = 12; - MsgTransferLeader = 13; - MsgTimeoutNow = 14; - MsgReadIndex = 15; - MsgReadIndexResp = 16; - MsgPreVote = 17; - MsgPreVoteResp = 18; + MsgHup = 0; + MsgBeat = 1; + MsgProp = 2; + MsgApp = 3; + MsgAppResp = 4; + MsgVote = 5; + MsgVoteResp = 6; + MsgSnap = 7; + MsgHeartbeat = 8; + MsgHeartbeatResp = 9; + MsgUnreachable = 10; + MsgSnapStatus = 11; + MsgCheckQuorum = 12; + MsgTransferLeader = 13; + MsgTimeoutNow = 14; + MsgReadIndex = 15; + MsgReadIndexResp = 16; + MsgPreVote = 17; + MsgPreVoteResp = 18; + MsgStorageAppend = 19; + MsgStorageAppendResp = 20; + MsgStorageApply = 21; + MsgStorageApplyResp = 22; // NOTE: when adding new message types, remember to update the isLocalMsg and // isResponseMsg arrays in raft/util.go and update the corresponding tests in // raft/util_test.go. @@ -71,14 +75,20 @@ message Message { optional uint64 from = 3 [(gogoproto.nullable) = false]; optional uint64 term = 4 [(gogoproto.nullable) = false]; // logTerm is generally used for appending Raft logs to followers. For example, - // (type=MsgApp,index=100,logTerm=5) means leader appends entries starting at - // index=101, and the term of entry at index 100 is 5. + // (type=MsgApp,index=100,logTerm=5) means the leader appends entries starting + // at index=101, and the term of the entry at index 100 is 5. // (type=MsgAppResp,reject=true,index=100,logTerm=5) means follower rejects some // entries from its leader as it already has an entry with term 5 at index 100. + // (type=MsgStorageAppendResp,index=100,logTerm=5) means the local node wrote + // entries up to index=100 in stable storage, and the term of the entry at index + // 100 was 5. optional uint64 logTerm = 5 [(gogoproto.nullable) = false]; optional uint64 index = 6 [(gogoproto.nullable) = false]; repeated Entry entries = 7 [(gogoproto.nullable) = false]; optional uint64 commit = 8 [(gogoproto.nullable) = false]; + // hardState can be non-nil for MsgStorageAppend messages and is nil for all + // other message types. + optional HardState hardState = 13 [(gogoproto.nullable) = true]; // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a // non-nil, empty value for the snapshot field of non-MsgSnap messages. Code @@ -86,7 +96,11 @@ message Message { optional Snapshot snapshot = 9 [(gogoproto.nullable) = true]; optional bool reject = 10 [(gogoproto.nullable) = false]; optional uint64 rejectHint = 11 [(gogoproto.nullable) = false]; - optional bytes context = 12; + optional bytes context = 12 [(gogoproto.nullable) = true]; + // responses are populated by a raft node to instruct storage threads on how + // to respond and who to respond to when the work associated with a message + // is complete. Populated for MsgStorageAppend and MsgStorageApply messages. + repeated Message responses = 14 [(gogoproto.nullable) = false]; } message HardState { diff --git a/raftpb/raft_test.go b/raftpb/raft_test.go index 7057446c..7f5be88b 100644 --- a/raftpb/raft_test.go +++ b/raftpb/raft_test.go @@ -45,7 +45,7 @@ func TestProtoMemorySizes(t *testing.T) { assert(unsafe.Sizeof(s), if64Bit(144, 80), "Snapshot") var m Message - assert(unsafe.Sizeof(m), if64Bit(128, 92), "Message") + assert(unsafe.Sizeof(m), if64Bit(160, 108), "Message") var hs HardState assert(unsafe.Sizeof(hs), 24, "HardState") From 5c02c257e06481ccd5322a6c72b544c775e09f1a Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 19 Oct 2022 23:26:07 -0400 Subject: [PATCH 06/13] raft: support asynchronous storage writes Fixes #12257. This change adds opt-in support to raft to perform local storage writes asynchronously from the raft state machine handling loop. A new AsyncStorageWrites configuration instructs the raft node to write to its local storage (raft log and state machine) using a request/response message passing interface instead of the default `Ready`/`Advance` function call interface. Local storage messages can be pipelined and processed asynchronously (with respect to `Ready` iteration), facilitating reduced interference between Raft proposals and increased batching of log appends and state machine application. As a result, use of asynchronous storage writes can reduce end-to-end commit latency and increase maximum throughput. When AsyncStorageWrites is enabled, the `Ready.Message` slice will include new `MsgStorageAppend` and `MsgStorageApply` messages. The messages will target a `LocalAppendThread` and a `LocalApplyThread`, respectively. Messages to the same target must be reliably processed in order. In other words, they can't be dropped (like messages over the network) and those targeted at the same thread can't be reordered. Messages to different targets can be processed in any order. `MsgStorageAppend` carries Raft log entries to append, election votes to persist, and snapshots to apply. All writes performed in response to a `MsgStorageAppend` are expected to be durable. The message assumes the role of the Entries, HardState, and Snapshot fields in Ready. `MsgStorageApply` carries committed entries to apply. The message assumes the role of the CommittedEntries field in Ready. Local messages each carry one or more response messages which should be delivered after the corresponding storage write has been completed. These responses may target the same node or may target other nodes. The storage threads are not responsible for understanding the response messages, only for delivering them to the correct target after performing the storage write. \## Design Considerations - There must be no regression for existing users that do not enable `AsyncStorageWrites`. For instance, CommittedEntries must not wait on unstable entries to be stabilized in cases where a follower is given committed entries in a MsgApp. - Asynchronous storage work should use a message passing interface, like the rest of this library. - The Raft leader and followers should behave symmetrically. Both should be able to use asynchronous storage writes for log appends and entry application. - The LocalAppendThread on a follower should be able to send MsgAppResp messages directly to the leader without passing back through the raft state machine handling loop. - The `unstable` log should remain true to its name. It should hold entries until they are stable and should not rely on an intermediate reliable cache. - Pseudo-targets should be assigned to messages that target the local storage systems to denote required ordering guarantees. - Code should be maximally unified across `AsyncStorageWrites=false` and `AsyncStorageWrites=true`. `AsyncStorageWrites=false` should be a special case of `AsyncStorageWrites=true` where the library hides the possibility of asynchrony. - It should be possible to apply snapshots asynchronously, even though a snapshot touches both the Raft log state and the state machine. The library should make this easy for users to handle by delaying all committed entries until after the snapshot has applied, so snapshot application can be handled by 1) flushing the apply thread, 2) sending the `MsgStorageAppend` that contains a snapshot to the `LocalAppendThread` to be applied. \## Usage When asynchronous storage writes is enabled, the responsibility of code using the library is different from what is presented in raft/doc.go (which has been updated to include a section about async storage writes). Users still read from the Node.Ready() channel. However, they process the updates it contains in a different manner. Users no longer consult the HardState, Entries, and Snapshot fields (steps 1 and 3 in doc.go). They also no longer call Node.Advance() to indicate that they have processed all entries in the Ready (step 4 in doc.go). Instead, all local storage operations are also communicated through messages present in the Ready.Message slice. The local storage messages come in two flavors. The first flavor is log append messages, which target a LocalAppendThread and carry Entries, HardState, and a Snapshot. The second flavor is entry application messages, which target a LocalApplyThread and carry CommittedEntries. Messages to the same target must be reliably processed in order. Messages to different targets can be processed in any order. Each local storage message carries a slice of response messages that must delivered after the corresponding storage write has been completed. With Asynchronous Storage Writes enabled, the total state machine handling loop will look something like this: ```go for { select { case <-s.Ticker: n.Tick() case rd := <-s.Node.Ready(): for _, m := range rd.Messages { switch m.To { case raft.LocalAppendThread: toAppend <- m case raft.LocalApplyThread: toApply <-m default: sendOverNetwork(m) } } case <-s.done: return } } ``` Usage of Asynchronous Storage Writes will typically also contain a pair of storage handler threads, one for log writes (append) and one for entry application to the local state machine (apply). Those will look something like: ```go // append thread go func() { for { select { case m := <-toAppend: saveToStorage(m.State, m.Entries, m.Snapshot) send(m.Responses) case <-s.done: return } } } // apply thread go func() { for { select { case m := <-toApply: for _, entry := range m.CommittedEntries { process(entry) if entry.Type == raftpb.EntryConfChange { var cc raftpb.ConfChange cc.Unmarshal(entry.Data) s.Node.ApplyConfChange(cc) } } send(m.Responses) case <-s.done: return } } } ``` \## Compatibility The library remains backwards compatible with existing users and the change does not introduce any breaking changes. Users that do not set `AsyncStorageWrites` to true in the `Config` struct will not notice a difference with this change. This is despite the fact that the existing "synchronous storage writes" interface was adapted to share a majority of the same code. For instance, `Node.Advance` has been adapted to transparently acknowledge an asynchronous log append attempt and an asynchronous state machine application attempt, internally using the same message passing mechanism introduced in this change. The change has no cross-version compatibility concerns. All changes are local to a process and nodes using asynchronous storage writes appear to behave no differently from the outside. Clusters are free to mix nodes running with and without asynchronous storage writes. \## Performance The bulk of the performance evaluation of this functionality thus far has been done with [rafttoy](https://github.com/nvanbenschoten/rafttoy), a benchmarking harness developed to experiment with Raft proposal pipeline optimization. The harness can be used to run single-node benchmarks or multi-node benchmarks. It supports plugable raft logs, storage engines, network transports, and pipeline implementations. To evaluate this change, we fixed the raft log (`etcd/wal`), storage engine (`pebble`), and network transport (`grpc`). We then built (https://github.com/nvanbenschoten/rafttoy/pull/3) a pipeline implementation on top of the new asynchronous storage writes functionality and compared it against two other pipeline implementations. The three pipeline implementations we compared were: - **basic** (P1): baseline stock raft usage, similar to the code in `doc.go` - **parallel append + early ack** (P2): CockroachDB's current pipeline, which includes two significant variations to the basic pipeline. The first is that it sends MsgApp messages to followers before writing to local Raft log (see [commit](https://github.com/cockroachdb/cockroach/commit/b67eb696f6a064d6a3a18517c1ed5525faa931e8) for explanation), allowing log appends to occur in parallel across replicas. The second is that it acknowledges committed log entries before applying them (see [commit](https://github.com/cockroachdb/cockroach/commit/87aaea70d68903ad04ecae1adf2910333aa07a78) for explanation). - **async append + async apply + early ack** (P3): A pipelining using asynchronous storage writes with a separate append thread and a separate apply thread. Also uses the same early acknowledgement optimization from above to ack committed entries before handing them to the apply thread. All testing was performed on a 3 node AWS cluster of m5.4xlarge instances with gp3 EBS volumes (16000 IOPS, 1GB/s throughput). ![Throughput vs latency of Raft proposal pipeline implementations](https://user-images.githubusercontent.com/5438456/197925200-11352c09-569b-460c-ae42-effbf407c4e5.svg) The comparison demonstrates two different benefits of asynchronous storage writes. The first is that it reduces end-to-end latency of proposals by 20-25%. For instance, when serving 16MB/s of write traffic, P1's average latency was 13.2ms, P2's average latency was 7.3ms, and P3's average latency was 5.24ms. This is a reduction in average latency of 28% from the optimized pipeline that does not use asynchronous storage writes. This matches expectations outlined in https://github.com/cockroachdb/cockroach/issues/17500. The second is that it increases the maximum throughput at saturation. This is because asynchronous storage writes can improve batching for both log appends and log application. In this experiment, we saw the average append batch size under saturation increase from 928 to 1542, which is a similar ratio to the increase in peak throughput. We see a similar difference for apply batch sizes. There is more benchmarking to do. For instance, we'll need to thoroughly verify that this change does not negatively impact the performance of users of this library that do not use asynchronous storage writes. Signed-off-by: Nathan VanBenschoten --- doc.go | 100 +++ log.go | 7 + log_unstable.go | 4 + node.go | 285 ++++++- raft.go | 308 ++++--- raft_paper_test.go | 17 +- raft_test.go | 171 ++-- rafttest/interaction_env.go | 6 +- rafttest/interaction_env_handler.go | 12 +- rafttest/interaction_env_handler_add_nodes.go | 2 + ...ction_env_handler_process_append_thread.go | 97 +++ ...action_env_handler_process_apply_thread.go | 111 +++ .../interaction_env_handler_process_ready.go | 72 +- rafttest/interaction_env_handler_stabilize.go | 47 +- rawnode.go | 61 +- rawnode_test.go | 2 + testdata/async_storage_writes.txt | 785 ++++++++++++++++++ util.go | 61 +- util_test.go | 8 + 19 files changed, 1869 insertions(+), 287 deletions(-) create mode 100644 rafttest/interaction_env_handler_process_append_thread.go create mode 100644 rafttest/interaction_env_handler_process_apply_thread.go create mode 100644 testdata/async_storage_writes.txt diff --git a/doc.go b/doc.go index 4febfe60..06253f4e 100644 --- a/doc.go +++ b/doc.go @@ -169,6 +169,94 @@ given ID MUST be used only once even if the old node has been removed. This means that for example IP addresses make poor node IDs since they may be reused. Node IDs must be non-zero. +# Usage with Asynchronous Storage Writes + +The library can be configured with an alternate interface for local storage +writes that can provide better performance in the presence of high proposal +concurrency by minimizing interference between proposals. This feature is called +AsynchronousStorageWrites, and can be enabled using the flag on the Config +struct with the same name. + +When Asynchronous Storage Writes is enabled, the responsibility of code using +the library is different from what was presented above. Users still read from +the Node.Ready() channel. However, they process the updates it contains in a +different manner. Users no longer consult the HardState, Entries, and Snapshot +fields (steps 1 and 3 above). They also no longer call Node.Advance() to +indicate that they have processed all entries in the Ready (step 4 above). +Instead, all local storage operations are also communicated through messages +present in the Ready.Message slice. + +The local storage messages come in two flavors. The first flavor is log append +messages, which target a LocalAppendThread and carry Entries, HardState, and a +Snapshot. The second flavor is entry application messages, which target a +LocalApplyThread and carry CommittedEntries. Messages to the same target must be +reliably processed in order. Messages to different targets can be processed in +any order. + +Each local storage message carries a slice of response messages that must +delivered after the corresponding storage write has been completed. These +responses may target the same node or may target other nodes. + +With Asynchronous Storage Writes enabled, the total state machine handling loop +will look something like this: + + for { + select { + case <-s.Ticker: + n.Tick() + case rd := <-s.Node.Ready(): + for _, m := range rd.Messages { + switch m.To { + case raft.LocalAppendThread: + toAppend <- m + case raft.LocalApplyThread: + toApply <-m + default: + sendOverNetwork(m) + } + } + case <-s.done: + return + } + } + +Usage of Asynchronous Storage Writes will typically also contain a pair of +storage handler threads, one for log writes (append) and one for entry +application to the local state machine (apply). Those will look something like: + + // append thread + go func() { + for { + select { + case m := <-toAppend: + saveToStorage(m.State, m.Entries, m.Snapshot) + send(m.Responses) + case <-s.done: + return + } + } + } + + // apply thread + go func() { + for { + select { + case m := <-toApply: + for _, entry := range m.CommittedEntries { + process(entry) + if entry.Type == raftpb.EntryConfChange { + var cc raftpb.ConfChange + cc.Unmarshal(entry.Data) + s.Node.ApplyConfChange(cc) + } + } + send(m.Responses) + case <-s.done: + return + } + } + } + # Implementation notes This implementation is up to date with the final Raft thesis @@ -295,5 +383,17 @@ stale log entries: that the follower that sent this 'MsgUnreachable' is not reachable, often indicating 'MsgApp' is lost. When follower's progress state is replicate, the leader sets it back to probe. + + 'MsgStorageAppend' is a message from a node to its local append storage + thread to write entries, hard state, and/or a snapshot to stable storage. + The message will carry one or more responses, one of which will be a + 'MsgStorageAppendResp' back to itself. The responses can also contain + 'MsgAppResp', 'MsgVoteResp', and 'MsgPreVoteResp' messages. Used with + AsynchronousStorageWrites. + + 'MsgStorageApply' is a message from a node to its local apply storage + thread to apply committed entries. The message will carry one response, + which will be a 'MsgStorageApplyResp' back to itself. Used with + AsynchronousStorageWrites. */ package raft diff --git a/log.go b/log.go index 3aad898c..82efda40 100644 --- a/log.go +++ b/log.go @@ -191,6 +191,13 @@ func (l *raftLog) hasNextUnstableEnts() bool { return len(l.nextUnstableEnts()) > 0 } +// hasNextOrInProgressUnstableEnts returns if there are any entries that are +// available to be written to the local stable log or in the process of being +// written to the local stable log. +func (l *raftLog) hasNextOrInProgressUnstableEnts() bool { + return len(l.unstable.entries) > 0 +} + // nextCommittedEnts returns all the available entries for execution. // Entries can be committed even when the local raft instance has not durably // appended them to the local raft log yet. If allowUnstable is true, committed diff --git a/log_unstable.go b/log_unstable.go index 5caa433e..6e4298de 100644 --- a/log_unstable.go +++ b/log_unstable.go @@ -127,6 +127,10 @@ func (u *unstable) acceptInProgress() { // stableTo marks entries up to the entry with the specified (index, term) as // being successfully written to stable storage. +// +// The method should only be called when the caller can attest that the entries +// can not be overwritten by an in-progress log append. See the related comment +// in newStorageAppendRespMsg. func (u *unstable) stableTo(i, t uint64) { gt, ok := u.maybeTerm(i) if !ok { diff --git a/node.go b/node.go index 8ef1ae05..5afeb44d 100644 --- a/node.go +++ b/node.go @@ -57,7 +57,12 @@ type Ready struct { // The current state of a Node to be saved to stable storage BEFORE // Messages are sent. + // // HardState will be equal to empty state if there is no update. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageAppend message in the + // Messages slice. pb.HardState // ReadStates can be used for node to serve linearizable read requests locally @@ -68,24 +73,44 @@ type Ready struct { // Entries specifies entries to be saved to stable storage BEFORE // Messages are sent. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageAppend message in the + // Messages slice. Entries []pb.Entry // Snapshot specifies the snapshot to be saved to stable storage. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageAppend message in the + // Messages slice. Snapshot pb.Snapshot // CommittedEntries specifies entries to be committed to a - // store/state-machine. These have previously been committed to stable - // store. + // store/state-machine. These have previously been appended to stable + // storage. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageApply message in the + // Messages slice. CommittedEntries []pb.Entry - // Messages specifies outbound messages to be sent AFTER Entries are - // committed to stable storage. + // Messages specifies outbound messages. + // + // If async storage writes are not enabled, these messages must be sent + // AFTER Entries are appended to stable storage. + // + // If async storage writes are enabled, these messages can be sent + // immediately as the messages that have the completion of the async writes + // as a precondition are attached to the individual MsgStorage{Append,Apply} + // messages instead. + // // If it contains a MsgSnap message, the application MUST report back to raft // when the snapshot has been received or has failed by calling ReportSnapshot. Messages []pb.Message - // MustSync indicates whether the HardState and Entries must be synchronously - // written to disk or if an asynchronous write is permissible. + // MustSync indicates whether the HardState and Entries must be durably + // written to disk or if a non-durable write is permissible. MustSync bool } @@ -103,19 +128,6 @@ func IsEmptySnap(sp pb.Snapshot) bool { return sp.Metadata.Index == 0 } -// appliedCursor extracts from the Ready the highest index the client has -// applied (once the Ready is confirmed via Advance). If no information is -// contained in the Ready, returns zero. -func (rd Ready) appliedCursor() uint64 { - if n := len(rd.CommittedEntries); n > 0 { - return rd.CommittedEntries[n-1].Index - } - if index := rd.Snapshot.Metadata.Index; index > 0 { - return index - } - return 0 -} - // Node represents a node in a raft cluster. type Node interface { // Tick increments the internal logical clock for the Node by a single tick. Election @@ -144,7 +156,8 @@ type Node interface { Step(ctx context.Context, msg pb.Message) error // Ready returns a channel that returns the current point-in-time state. - // Users of the Node must call Advance after retrieving the state returned by Ready. + // Users of the Node must call Advance after retrieving the state returned by Ready (unless + // async storage writes is enabled, in which case it should never be called). // // NOTE: No committed entries from the next Ready may be applied until all committed entries // and snapshots from the previous one have finished. @@ -159,6 +172,9 @@ type Node interface { // commands. For example. when the last Ready contains a snapshot, the application might take // a long time to apply the snapshot data. To continue receiving Ready without blocking raft // progress, it can call Advance before finishing applying the last ready. + // + // NOTE: Advance must not be called when using AsyncStorageWrites. Response messages from the + // local append and apply threads take its place. Advance() // ApplyConfChange applies a config change (previously passed to // ProposeConfChange) to the node. This must be called whenever a config @@ -309,9 +325,7 @@ func (n *node) run() { lead := None for { - if advancec != nil { - readyc = nil - } else if n.rn.HasReady() { + if advancec == nil && n.rn.HasReady() { // Populate a Ready. Note that this Ready is not guaranteed to // actually be handled. We will arm readyc, but there's no guarantee // that we will actually send on it. It's possible that we will @@ -352,10 +366,11 @@ func (n *node) run() { close(pm.result) } case m := <-n.recvc: - // filter out response message from unknown From. - if pr := r.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) { - r.Step(m) + if IsResponseMsg(m.Type) && !IsLocalMsgTarget(m.From) && r.prs.Progress[m.From] == nil { + // Filter out response message from unknown From. + break } + r.Step(m) case cc := <-n.confc: _, okBefore := r.prs.Progress[r.id] cs := r.applyConfChange(cc) @@ -393,7 +408,12 @@ func (n *node) run() { n.rn.Tick() case readyc <- rd: n.rn.acceptReady(rd) - advancec = n.advancec + if !n.rn.raft.asyncStorageWrites { + advancec = n.advancec + } else { + rd = Ready{} + } + readyc = nil case <-advancec: n.rn.Advance(rd) rd = Ready{} @@ -425,8 +445,8 @@ func (n *node) Propose(ctx context.Context, data []byte) error { } func (n *node) Step(ctx context.Context, m pb.Message) error { - // ignore unexpected local messages receiving over network - if IsLocalMsg(m.Type) { + // Ignore unexpected local messages receiving over network. + if IsLocalMsg(m.Type) && !IsLocalMsgTarget(m.From) { // TODO: return an error? return nil } @@ -562,7 +582,7 @@ func (n *node) ReadIndex(ctx context.Context, rctx []byte) error { func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { rd := Ready{ Entries: r.raftLog.nextUnstableEnts(), - CommittedEntries: r.raftLog.nextCommittedEnts(true /* allowUnstable */), + CommittedEntries: r.raftLog.nextCommittedEnts(!r.asyncStorageWrites), Messages: r.msgs, } if softSt := r.softState(); !softSt.equal(prevSoftSt) { @@ -578,6 +598,30 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { rd.ReadStates = r.readStates } rd.MustSync = MustSync(r.hardState(), prevHardSt, len(rd.Entries)) + + if r.asyncStorageWrites { + // If async storage writes are enabled, enqueue messages to + // local storage threads, where applicable. + if needStorageAppend(rd, len(r.msgsAfterAppend) > 0) { + m := newStorageAppendMsg(r, rd) + rd.Messages = append(rd.Messages, m) + } + if needStorageApply(rd) { + m := newStorageApplyMsg(r, rd) + rd.Messages = append(rd.Messages, m) + } + } else { + // If async storage writes are disabled, immediately enqueue + // msgsAfterAppend to be sent out. The Ready struct contract + // mandates that Messages cannot be sent until after Entries + // are written to stable storage. + for _, m := range r.msgsAfterAppend { + if m.To != r.id { + rd.Messages = append(rd.Messages, m) + } + } + } + return rd } @@ -591,3 +635,184 @@ func MustSync(st, prevst pb.HardState, entsnum int) bool { // log entries[] return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term } + +func needStorageAppend(rd Ready, haveMsgsAfterAppend bool) bool { + // Return true if log entries, hard state, or a snapshot need to be written + // to stable storage. Also return true if any messages are contingent on all + // prior MsgStorageAppend being processed. + return len(rd.Entries) > 0 || + !IsEmptyHardState(rd.HardState) || + !IsEmptySnap(rd.Snapshot) || + haveMsgsAfterAppend +} + +// newStorageAppendMsg creates the message that should be sent to the local +// append thread to instruct it to append log entries, write an updated hard +// state, and apply a snapshot. The message also carries a set of responses +// that should be delivered after the rest of the message is processed. Used +// with AsyncStorageWrites. +func newStorageAppendMsg(r *raft, rd Ready) pb.Message { + m := pb.Message{ + Type: pb.MsgStorageAppend, + To: LocalAppendThread, + From: r.id, + Term: r.Term, + Entries: rd.Entries, + } + if !IsEmptyHardState(rd.HardState) { + hs := rd.HardState + m.HardState = &hs + } + if !IsEmptySnap(rd.Snapshot) { + snap := rd.Snapshot + m.Snapshot = &snap + } + // Attach all messages in msgsAfterAppend as responses to be delivered after + // the message is processed, along with a self-directed MsgStorageAppendResp + // to acknowledge the entry stability. + // + // NB: it is important for performance that MsgStorageAppendResp message be + // handled after self-directed MsgAppResp messages on the leader (which will + // be contained in msgsAfterAppend). This ordering allows the MsgAppResp + // handling to use a fast-path in r.raftLog.term() before the newly appended + // entries are removed from the unstable log. + m.Responses = r.msgsAfterAppend + m.Responses = append(m.Responses, newStorageAppendRespMsg(r, rd)) + return m +} + +// newStorageAppendRespMsg creates the message that should be returned to node +// after the unstable log entries, hard state, and snapshot in the current Ready +// (along with those in all prior Ready structs) have been saved to stable +// storage. +func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { + m := pb.Message{ + Type: pb.MsgStorageAppendResp, + To: r.id, + From: LocalAppendThread, + // Dropped after term change, see below. + Term: r.Term, + } + if r.raftLog.hasNextOrInProgressUnstableEnts() { + // If the raft log has unstable entries, attach the last index and term to the + // response message. This (index, term) tuple will be handed back and consulted + // when the stability of those log entries is signaled to the unstable. If the + // (index, term) match the unstable log by the time the response is received, + // the unstable log can be truncated. + // + // However, with just this logic, there would be an ABA problem that could lead + // to the unstable log and the stable log getting out of sync temporarily and + // leading to an inconsistent view. Consider the following example with 5 nodes, + // A B C D E: + // + // 1. A is the leader. + // 2. A proposes some log entries but only B receives these entries. + // 3. B gets the Ready and the entries are appended asynchronously. + // 4. A crashes and C becomes leader after getting a vote from D and E. + // 5. C proposes some log entries and B receives these entries, overwriting the + // previous unstable log entries that are in the process of being appended. + // The entries have a larger term than the previous entries but the same + // indexes. It begins appending these new entries asynchronously. + // 6. C crashes and A restarts and becomes leader again after getting the vote + // from D and E. + // 7. B receives the entries from A which are the same as the ones from step 2, + // overwriting the previous unstable log entries that are in the process of + // being appended from step 5. The entries have the original terms and + // indexes from step 2. Recall that log entries retain their original term + // numbers when a leader replicates entries from previous terms. It begins + // appending these new entries asynchronously. + // 8. The asynchronous log appends from the first Ready complete and stableTo + // is called. + // 9. However, the log entries from the second Ready are still in the + // asynchronous append pipeline and will overwrite (in stable storage) the + // entries from the first Ready at some future point. We can't truncate the + // unstable log yet or a future read from Storage might see the entries from + // step 5 before they have been replaced by the entries from step 7. + // Instead, we must wait until we are sure that the entries are stable and + // that no in-progress appends might overwrite them before removing entries + // from the unstable log. + // + // To prevent these kinds of problems, we also attach the current term to the + // MsgStorageAppendResp (above). If the term has changed by the time the + // MsgStorageAppendResp if returned, the response is ignored and the unstable + // log is not truncated. The unstable log is only truncated when the term has + // remained unchanged from the time that the MsgStorageAppend was sent to the + // time that the MsgStorageAppendResp is received, indicating that no-one else + // is in the process of truncating the stable log. + // + // However, this replaces a correctness problem with a liveness problem. If we + // only attempted to truncate the unstable log when appending new entries but + // also occasionally dropped these responses, then quiescence of new log entries + // could lead to the unstable log never being truncated. + // + // To combat this, we attempt to truncate the log on all MsgStorageAppendResp + // messages where the unstable log is not empty, not just those associated with + // entry appends. This includes MsgStorageAppendResp messages associated with an + // updated HardState, which occur after a term change. + // + // In other words, we set Index and LogTerm in a block that looks like: + // + // if r.raftLog.hasNextOrInProgressUnstableEnts() { ... } + // + // not like: + // + // if len(rd.Entries) > 0 { ... } + // + // To do so, we attach r.raftLog.lastIndex() and r.raftLog.lastTerm(), not the + // (index, term) of the last entry in rd.Entries. If rd.Entries is not empty, + // these will be the same. However, if rd.Entries is empty, we still want to + // attest that this (index, term) is correct at the current term, in case the + // MsgStorageAppend that contained the last entry in the unstable slice carried + // an earlier term and was dropped. + // TODO(nvanbenschoten): test this behavior in a data-driven test. + m.Index = r.raftLog.lastIndex() + m.LogTerm = r.raftLog.lastTerm() + } + if !IsEmptySnap(rd.Snapshot) { + snap := rd.Snapshot + m.Snapshot = &snap + } + return m +} + +func needStorageApply(rd Ready) bool { + return len(rd.CommittedEntries) > 0 +} + +// newStorageApplyMsg creates the message that should be sent to the local +// apply thread to instruct it to apply committed log entries. The message +// also carries a response that should be delivered after the rest of the +// message is processed. Used with AsyncStorageWrites. +func newStorageApplyMsg(r *raft, rd Ready) pb.Message { + ents := rd.CommittedEntries + last := ents[len(ents)-1].Index + return pb.Message{ + Type: pb.MsgStorageApply, + To: LocalApplyThread, + From: r.id, + Term: 0, // committed entries don't apply under a specific term + Entries: ents, + Index: last, + Responses: []pb.Message{ + newStorageApplyRespMsg(r, ents), + }, + } +} + +// newStorageApplyRespMsg creates the message that should be returned to node +// after the committed entries in the current Ready (along with those in all +// prior Ready structs) have been applied to the local state machine. +func newStorageApplyRespMsg(r *raft, committedEnts []pb.Entry) pb.Message { + last := committedEnts[len(committedEnts)-1].Index + size := r.getUncommittedSize(committedEnts) + return pb.Message{ + Type: pb.MsgStorageApplyResp, + To: r.id, + From: LocalApplyThread, + Term: 0, // committed entries don't apply under a specific term + Index: last, + // NOTE: we abuse the LogTerm field to store the aggregate entry size so + // that we don't need to introduce a new field on Message. + LogTerm: size, + } +} diff --git a/raft.go b/raft.go index e279baeb..33b058bc 100644 --- a/raft.go +++ b/raft.go @@ -31,9 +31,18 @@ import ( "go.etcd.io/raft/v3/tracker" ) -// None is a placeholder node ID used when there is no leader. -const None uint64 = 0 -const noLimit = math.MaxUint64 +const ( + // None is a placeholder node ID used when there is no leader. + None uint64 = 0 + // LocalAppendThread is a reference to a local thread that saves unstable + // log entries and snapshots to stable storage. The identifier is used as a + // target for MsgStorageAppend messages when AsyncStorageWrites is enabled. + LocalAppendThread uint64 = math.MaxUint64 + // LocalApplyThread is a reference to a local thread that applies committed + // log entries to the local state machine. The identifier is used as a + // target for MsgStorageApply messages when AsyncStorageWrites is enabled. + LocalApplyThread uint64 = math.MaxUint64 - 1 +) // Possible values for StateType. const ( @@ -70,6 +79,8 @@ const ( campaignTransfer CampaignType = "CampaignTransfer" ) +const noLimit = math.MaxUint64 + // ErrProposalDropped is returned when the proposal is ignored by some cases, // so that the proposer can be notified and fail fast. var ErrProposalDropped = errors.New("raft proposal dropped") @@ -140,6 +151,42 @@ type Config struct { // applied entries. This is a very application dependent configuration. Applied uint64 + // AsyncStorageWrites configures the raft node to write to its local storage + // (raft log and state machine) using a request/response message passing + // interface instead of the default Ready/Advance function call interface. + // Local storage messages can be pipelined and processed asynchronously + // (with respect to Ready iteration), facilitating reduced interference + // between Raft proposals and increased batching of log appends and state + // machine application. As a result, use of asynchronous storage writes can + // reduce end-to-end commit latency and increase maximum throughput. + // + // When true, the Ready.Message slice will include MsgStorageAppend and + // MsgStorageApply messages. The messages will target a LocalAppendThread + // and a LocalApplyThread, respectively. Messages to the same target must be + // reliably processed in order. In other words, they can't be dropped (like + // messages over the network) and those targeted at the same thread can't be + // reordered. Messages to different targets can be processed in any order. + // + // MsgStorageAppend carries Raft log entries to append, election votes to + // persist, and snapshots to apply. All writes performed in service of a + // MsgStorageAppend must be durable before response messages are delivered. + // However, if the MsgStorageAppend carries no response messages, durability + // is not required. The message assumes the role of the Entries, HardState, + // and Snapshot fields in Ready. + // + // MsgStorageApply carries committed entries to apply. Writes performed in + // service of a MsgStorageApply need not be durable before response messages + // are delivered. The message assumes the role of the CommittedEntries field + // in Ready. + // + // Local messages each carry one or more response messages which should be + // delivered after the corresponding storage write has been completed. These + // responses may target the same node or may target other nodes. The storage + // threads are not responsible for understanding the response messages, only + // for delivering them to the correct target after performing the storage + // write. + AsyncStorageWrites bool + // MaxSizePerMsg limits the max byte size of each append message. Smaller // value lowers the raft recovery cost(initial probing and message lost // during normal operation). On the other side, it might affect the @@ -212,6 +259,9 @@ func (c *Config) validate() error { if c.ID == None { return errors.New("cannot use none as id") } + if IsLocalMsgTarget(c.ID) { + return errors.New("cannot use local target as id") + } if c.HeartbeatTick <= 0 { return errors.New("heartbeat tick must be greater than 0") @@ -276,15 +326,22 @@ type raft struct { // isLearner is true if the local raft node is a learner. isLearner bool + // msgs contains the list of messages that should be sent out immediately to + // other nodes. + // + // Messages in this list must target other nodes. msgs []pb.Message - - // voteSelfOnAdvance is a marker that the local raft node should vote for - // itself upon its next call to advance. This is not meant to be the final - // approach towards handling self-votes, but it's a useful intermediate - // point to get all tests working and to write some additional tests that - // demonstrate possible race conditions when self-voting is asynchronous. - // This is replaced in a later commit. - voteSelfOnAdvance pb.Message + // msgsAfterAppend contains the list of messages that should be sent after + // the accumulated unstable state (e.g. term, vote, []entry, and snapshot) + // has been persisted to durable storage. This includes waiting for any + // unstable state that is already in the process of being persisted (i.e. + // has already been handed out in a prior Ready struct) to complete. + // + // Messages in this list may target other nodes or may target this node. + // + // Messages in this list have the type MsgAppResp, MsgVoteResp, or + // MsgPreVoteResp. See the comment in raft.send for details. + msgsAfterAppend []pb.Message // the leader id lead uint64 @@ -315,8 +372,9 @@ type raft struct { // only leader keeps heartbeatElapsed. heartbeatElapsed int - checkQuorum bool - preVote bool + asyncStorageWrites bool + checkQuorum bool + preVote bool heartbeatTimeout int electionTimeout int @@ -359,6 +417,7 @@ func newRaft(c *Config) *raft { electionTimeout: c.ElectionTick, heartbeatTimeout: c.HeartbeatTick, logger: c.Logger, + asyncStorageWrites: c.AsyncStorageWrites, checkQuorum: c.CheckQuorum, preVote: c.PreVote, readOnly: newReadOnly(c.ReadOnlyOption), @@ -424,11 +483,11 @@ func (r *raft) send(m pb.Message) { // - MsgPreVoteResp: m.Term is the term received in the original // MsgPreVote if the pre-vote was granted, non-zero for the // same reasons MsgPreVote is - panic(fmt.Sprintf("term should be set when sending %s", m.Type)) + r.logger.Panicf("term should be set when sending %s", m.Type) } } else { if m.Term != 0 { - panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term)) + r.logger.Panicf("term should not be set when sending %s (was %d)", m.Type, m.Term) } // do not attach term to MsgProp, MsgReadIndex // proposals are a way to forward to the leader and @@ -438,10 +497,59 @@ func (r *raft) send(m pb.Message) { m.Term = r.Term } } - if m.To == r.id { - r.logger.Panicf("message should not be self-addressed when sending %s", m.Type) + if m.Type == pb.MsgAppResp || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVoteResp { + // If async storage writes are enabled, messages added to the msgs slice + // are allowed to be sent out before unstable state (e.g. log entry + // writes and election votes) have been durably synced to the local + // disk. + // + // For most message types, this is not an issue. However, response + // messages that relate to "voting" on either leader election or log + // appends require durability before they can be sent. It would be + // incorrect to publish a vote in an election before that vote has been + // synced to stable storage locally. Similarly, it would be incorrect to + // acknowledge a log append to the leader before that entry has been + // synced to stable storage locally. + // + // Per the Raft thesis, section 3.8 Persisted state and server restarts: + // + // > Raft servers must persist enough information to stable storage to + // > survive server restarts safely. In particular, each server persists + // > its current term and vote; this is necessary to prevent the server + // > from voting twice in the same term or replacing log entries from a + // > newer leader with those from a deposed leader. Each server also + // > persists new log entries before they are counted towards the entries’ + // > commitment; this prevents committed entries from being lost or + // > “uncommitted” when servers restart + // + // To enforce this durability requirement, these response messages are + // queued to be sent out as soon as the current collection of unstable + // state (the state that the response message was predicated upon) has + // been durably persisted. This unstable state may have already been + // passed to a Ready struct whose persistence is in progress or may be + // waiting for the next Ready struct to begin being written to Storage. + // These messages must wait for all of this state to be durable before + // being published. + // + // Rejected responses (m.Reject == true) present an interesting case + // where the durability requirement is less unambiguous. A rejection may + // be predicated upon unstable state. For instance, a node may reject a + // vote for one peer because it has already begun syncing its vote for + // another peer. Or it may reject a vote from one peer because it has + // unstable log entries that indicate that the peer is behind on its + // log. In these cases, it is likely safe to send out the rejection + // response immediately without compromising safety in the presence of a + // server restart. However, because these rejections are rare and + // because the safety of such behavior has not been formally verified, + // we err on the side of safety and omit a `&& !m.Reject` condition + // above. + r.msgsAfterAppend = append(r.msgsAfterAppend, m) + } else { + if m.To == r.id { + r.logger.Panicf("message should not be self-addressed when sending %s", m.Type) + } + r.msgs = append(r.msgs, m) } - r.msgs = append(r.msgs, m) } // sendAppend sends an append RPC with new entries (if any) and the @@ -570,61 +678,39 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { }) } -func (r *raft) advance(rd Ready) { - r.reduceUncommittedSize(rd.CommittedEntries) - - // If entries were applied (or a snapshot), update our cursor for - // the next Ready. Note that if the current HardState contains a - // new Commit index, this does not mean that we're also applying - // all of the new entries due to commit pagination by size. - if newApplied := rd.appliedCursor(); newApplied > 0 { - r.raftLog.appliedTo(newApplied) - - if r.prs.Config.AutoLeave && newApplied >= r.pendingConfIndex && r.state == StateLeader { - // If the current (and most recent, at least for this leader's term) - // configuration should be auto-left, initiate that now. We use a - // nil Data which unmarshals into an empty ConfChangeV2 and has the - // benefit that appendEntry can never refuse it based on its size - // (which registers as zero). - m, err := confChangeToMsg(nil) - if err != nil { - panic(err) - } - // NB: this proposal can't be dropped due to size, but can be - // dropped if a leadership transfer is in progress. We'll keep - // checking this condition on each applied entry, so either the - // leadership transfer will succeed and the new leader will leave - // the joint configuration, or the leadership transfer will fail, - // and we will propose the config change on the next advance. - if err := r.Step(m); err != nil { - r.logger.Debugf("not initiating automatic transition out of joint configuration %s: %v", r.prs.Config, err) - } else { - r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config) - } +func (r *raft) appliedTo(index uint64) { + oldApplied := r.raftLog.applied + newApplied := max(index, oldApplied) + r.raftLog.appliedTo(newApplied) + + if r.prs.Config.AutoLeave && newApplied >= r.pendingConfIndex && r.state == StateLeader { + // If the current (and most recent, at least for this leader's term) + // configuration should be auto-left, initiate that now. We use a + // nil Data which unmarshals into an empty ConfChangeV2 and has the + // benefit that appendEntry can never refuse it based on its size + // (which registers as zero). + m, err := confChangeToMsg(nil) + if err != nil { + panic(err) } - } - - if len(rd.Entries) > 0 { - e := rd.Entries[len(rd.Entries)-1] - if r.id == r.lead { - // The leader needs to self-ack the entries just appended (since it doesn't - // send an MsgApp to itself). This is roughly equivalent to: - // - // r.prs.Progress[r.id].MaybeUpdate(e.Index) - // if r.maybeCommit() { - // r.bcastAppend() - // } - _ = r.Step(pb.Message{From: r.id, Type: pb.MsgAppResp, Index: e.Index}) + // NB: this proposal can't be dropped due to size, but can be + // dropped if a leadership transfer is in progress. We'll keep + // checking this condition on each applied entry, so either the + // leadership transfer will succeed and the new leader will leave + // the joint configuration, or the leadership transfer will fail, + // and we will propose the config change on the next advance. + if err := r.Step(m); err != nil { + r.logger.Debugf("not initiating automatic transition out of joint configuration %s: %v", r.prs.Config, err) + } else { + r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config) } - // NB: it's important for performance that this call happens after - // r.Step above on the leader. This is because r.Step can then use - // a fast-path for `r.raftLog.term()`. - r.raftLog.stableTo(e.Index, e.Term) - } - if !IsEmptySnap(rd.Snapshot) { - r.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index) } - r.maybeVoteForSelf() +} + +func (r *raft) appliedSnap(snap *pb.Snapshot) { + index := snap.Metadata.Index + r.raftLog.stableSnapTo(index) + r.appliedTo(index) } // maybeCommit attempts to advance the commit index. Returns true if @@ -635,22 +721,6 @@ func (r *raft) maybeCommit() bool { return r.raftLog.maybeCommit(mci, r.Term) } -// maybeVoteForSelf attempts to inform a (pre-)candidate node that its -// vote for itself has been made durable and can now be counted towards -// the active election, if one is still ongoing. Returns true if the -// node was informed of a self-vote. -func (r *raft) maybeVoteForSelf() bool { - if r.voteSelfOnAdvance.Type == 0 { - return false - } - voteMsg := r.voteSelfOnAdvance - // NB: Clear the voteSelfOnAdvance marker before calling Step. - // Step may re-set the marker and cause us to loop. - r.voteSelfOnAdvance = pb.Message{} - _ = r.Step(voteMsg) - return true -} - func (r *raft) reset(term uint64) { if r.Term != term { r.Term = term @@ -698,7 +768,18 @@ func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) { return false } // use latest "last" index after truncate/append - r.raftLog.append(es...) + li = r.raftLog.append(es...) + // The leader needs to self-ack the entries just appended once they have + // been durably persisted (since it doesn't send an MsgApp to itself). This + // response message will be added to msgsAfterAppend and delivered back to + // this node after these entries have been written to stable storage. When + // handled, this is roughly equivalent to: + // + // r.prs.Progress[r.id].MaybeUpdate(e.Index) + // if r.maybeCommit() { + // r.bcastAppend() + // } + r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li}) return true } @@ -818,7 +899,7 @@ func (r *raft) becomeLeader() { // uncommitted log quota. This is because we want to preserve the // behavior of allowing one entry larger than quota if the current // usage is zero. - r.reduceUncommittedSize([]pb.Entry{emptyEnt}) + r.uncommittedSize = 0 r.logger.Infof("%x became leader at term %d", r.id, r.Term) } @@ -876,7 +957,12 @@ func (r *raft) campaign(t CampaignType) { } for _, id := range ids { if id == r.id { - r.voteSelfOnAdvance = pb.Message{To: id, From: id, Term: term, Type: voteRespMsgType(voteMsg)} + // The candidate votes for itself and should account for this self + // vote once the vote has been durably persisted (since it doesn't + // send a MsgVote to itself). This response message will be added to + // msgsAfterAppend and delivered back to this node after the vote + // has been written to stable storage. + r.send(pb.Message{To: id, Term: term, Type: voteRespMsgType(voteMsg)}) continue } r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d", @@ -967,6 +1053,21 @@ func (r *raft) Step(m pb.Message) error { r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d", r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term) r.send(pb.Message{To: m.From, Term: r.Term, Type: pb.MsgPreVoteResp, Reject: true}) + } else if m.Type == pb.MsgStorageAppendResp { + if m.Index != 0 { + // Don't consider the appended log entries to be stable because + // they may have been overwritten in the unstable log during a + // later term. See the comment in newStorageAppendResp for more + // about this race. + r.logger.Infof("%x [term: %d] ignored entry appends from a %s message with lower term [term: %d]", + r.id, r.Term, m.Type, m.Term) + } + if m.Snapshot != nil { + // Even if the snapshot applied under a different term, its + // application is still valid. Snapshots carry committed + // (term-independent) state. + r.appliedSnap(m.Snapshot) + } } else { // ignore other cases r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]", @@ -983,6 +1084,20 @@ func (r *raft) Step(m pb.Message) error { r.hup(campaignElection) } + case pb.MsgStorageAppendResp: + if m.Index != 0 { + r.raftLog.stableTo(m.Index, m.LogTerm) + } + if m.Snapshot != nil { + r.appliedSnap(m.Snapshot) + } + + case pb.MsgStorageApplyResp: + r.appliedTo(m.Index) + // NOTE: we abuse the LogTerm field to store the aggregate entry size so + // that we don't need to introduce a new field on Message. + r.reduceUncommittedSize(m.LogTerm) + case pb.MsgVote, pb.MsgPreVote: // We can vote if this is a repeat of a vote we've already cast... canVote := r.Vote == m.From || @@ -1835,14 +1950,19 @@ func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool { return true } -// reduceUncommittedSize accounts for the newly committed entries by decreasing -// the uncommitted entry size limit. -func (r *raft) reduceUncommittedSize(ents []pb.Entry) { +// getUncommittedSize computes the aggregate size of the provided entries. +func (r *raft) getUncommittedSize(ents []pb.Entry) uint64 { if r.uncommittedSize == 0 { // Fast-path for followers, who do not track or enforce the limit. - return + return 0 } - if s := payloadsSize(ents); s > r.uncommittedSize { + return payloadsSize(ents) +} + +// reduceUncommittedSize accounts for the newly committed entries by decreasing +// the uncommitted entry size limit. +func (r *raft) reduceUncommittedSize(s uint64) { + if s > r.uncommittedSize { // uncommittedSize may underestimate the size of the uncommitted Raft // log tail but will never overestimate it. Saturate at 0 instead of // allowing overflow. diff --git a/raft_paper_test.go b/raft_paper_test.go index 72942c08..585168d0 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -162,7 +162,7 @@ func testNonleaderStartElection(t *testing.T, state StateType) { for i := 1; i < 2*et; i++ { r.tick() } - r.maybeVoteForSelf() + r.advanceMessagesAfterAppend() if r.Term != 2 { t.Errorf("term = %d, want 2", r.Term) @@ -219,7 +219,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) { r := newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(idsBySize(tt.size)...))) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) - r.maybeVoteForSelf() + r.advanceMessagesAfterAppend() for id, vote := range tt.votes { r.Step(pb.Message{From: id, To: 1, Term: r.Term, Type: pb.MsgVoteResp, Reject: !vote}) } @@ -255,7 +255,7 @@ func TestFollowerVote(t *testing.T) { r.Step(pb.Message{From: tt.nvote, To: 1, Term: 1, Type: pb.MsgVote}) - msgs := r.readMessages() + msgs := r.msgsAfterAppend wmsgs := []pb.Message{ {From: 1, To: tt.nvote, Term: 1, Type: pb.MsgVoteResp, Reject: tt.wreject}, } @@ -497,11 +497,8 @@ func TestLeaderAcknowledgeCommit(t *testing.T) { commitNoopEntry(r, s) li := r.raftLog.lastIndex() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) - - rd := newReady(r, &SoftState{}, pb.HardState{}) - s.Append(rd.Entries) - r.advance(rd) // simulate having appended entry on leader - for _, m := range rd.Messages { + r.advanceMessagesAfterAppend() + for _, m := range r.msgs { if tt.nonLeaderAcceptors[m.To] { r.Step(acceptAndReply(m)) } @@ -896,9 +893,7 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}}) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Term: r.Term, Index: tt.index}) - rd := newReady(r, &SoftState{}, pb.HardState{}) - storage.Append(rd.Entries) - r.advance(rd) + r.advanceMessagesAfterAppend() if r.raftLog.committed != tt.wcommit { t.Errorf("#%d: commit = %d, want %d", i, r.raftLog.committed, tt.wcommit) } diff --git a/raft_test.go b/raft_test.go index 030e59b5..f5979168 100644 --- a/raft_test.go +++ b/raft_test.go @@ -27,17 +27,19 @@ import ( "go.etcd.io/raft/v3/tracker" ) -// nextEnts returns the appliable entries and updates the applied index +// nextEnts returns the appliable entries and updates the applied index. func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) { - for { - rd := newReady(r, &SoftState{}, pb.HardState{}) - s.Append(rd.Entries) - r.advance(rd) - if len(rd.Entries)+len(rd.CommittedEntries) == 0 { - return ents - } - ents = append(ents, rd.CommittedEntries...) - } + // Append unstable entries. + s.Append(r.raftLog.nextUnstableEnts()) + r.raftLog.stableTo(r.raftLog.lastIndex(), r.raftLog.lastTerm()) + + // Run post-append steps. + r.advanceMessagesAfterAppend() + + // Return committed entries. + ents = r.raftLog.nextCommittedEnts(true) + r.raftLog.appliedTo(r.raftLog.committed) + return ents } func mustAppendEntry(r *raft, ents ...pb.Entry) { @@ -49,16 +51,45 @@ func mustAppendEntry(r *raft, ents ...pb.Entry) { type stateMachine interface { Step(m pb.Message) error readMessages() []pb.Message - maybeVoteForSelf() bool + advanceMessagesAfterAppend() } func (r *raft) readMessages() []pb.Message { + r.advanceMessagesAfterAppend() msgs := r.msgs - r.msgs = make([]pb.Message, 0) + r.msgs = nil + return msgs +} + +func (r *raft) advanceMessagesAfterAppend() { + for { + msgs := r.takeMessagesAfterAppend() + if len(msgs) == 0 { + break + } + r.stepOrSend(msgs) + } +} +func (r *raft) takeMessagesAfterAppend() []pb.Message { + msgs := r.msgsAfterAppend + r.msgsAfterAppend = nil return msgs } +func (r *raft) stepOrSend(msgs []pb.Message) error { + for _, m := range msgs { + if m.To == r.id { + if err := r.Step(m); err != nil { + return err + } + } else { + r.msgs = append(r.msgs, m) + } + } + return nil +} + func TestProgressLeader(t *testing.T) { s := newTestMemoryStorage(withPeers(1, 2)) r := newTestRaft(1, 5, 1, s) @@ -76,11 +107,11 @@ func TestProgressLeader(t *testing.T) { if m := r.prs.Progress[1].Match; m != 0 { t.Fatalf("expected zero match, got %d", m) } - rd := newReady(r, &SoftState{}, pb.HardState{}) - if len(rd.Entries) != 6 || len(rd.Entries[0].Data) > 0 || string(rd.Entries[5].Data) != "foo" { - t.Fatalf("unexpected Entries: %s", DescribeReady(rd, nil)) + ents := r.raftLog.nextUnstableEnts() + if len(ents) != 6 || len(ents[0].Data) > 0 || string(ents[5].Data) != "foo" { + t.Fatalf("unexpected entries: %v", ents) } - r.advance(rd) + r.advanceMessagesAfterAppend() if m := r.prs.Progress[1].Match; m != 6 { t.Fatalf("unexpected Match %d", m) } @@ -252,7 +283,7 @@ func TestUncommittedEntryLimit(t *testing.T) { if e := maxEntries * numFollowers; len(ms) != e { t.Fatalf("expected %d messages, got %d", e, len(ms)) } - r.reduceUncommittedSize(propEnts) + r.reduceUncommittedSize(r.getUncommittedSize(propEnts)) if r.uncommittedSize != 0 { t.Fatalf("committed everything, but still tracking %d", r.uncommittedSize) } @@ -288,7 +319,7 @@ func TestUncommittedEntryLimit(t *testing.T) { if e := 2 * numFollowers; len(ms) != e { t.Fatalf("expected %d messages, got %d", e, len(ms)) } - r.reduceUncommittedSize(propEnts) + r.reduceUncommittedSize(r.getUncommittedSize(propEnts)) if n := r.uncommittedSize; n != 0 { t.Fatalf("expected zero uncommitted size, got %d", n) } @@ -383,8 +414,8 @@ func TestLearnerPromotion(t *testing.T) { setRandomizedElectionTimeout(n1, n1.electionTimeout) for i := 0; i < n1.electionTimeout; i++ { n1.tick() - n1.maybeVoteForSelf() } + n1.advanceMessagesAfterAppend() if n1.state != StateLeader { t.Errorf("peer 1 state: %s, want %s", n1.state, StateLeader) @@ -405,8 +436,8 @@ func TestLearnerPromotion(t *testing.T) { setRandomizedElectionTimeout(n2, n2.electionTimeout) for i := 0; i < n2.electionTimeout; i++ { n2.tick() - n2.maybeVoteForSelf() } + n2.advanceMessagesAfterAppend() nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat}) @@ -427,10 +458,11 @@ func TestLearnerCanVote(t *testing.T) { n2.Step(pb.Message{From: 1, To: 2, Term: 2, Type: pb.MsgVote, LogTerm: 11, Index: 11}) - if len(n2.msgs) != 1 { - t.Fatalf("expected exactly one message, not %+v", n2.msgs) + msgs := n2.readMessages() + if len(msgs) != 1 { + t.Fatalf("expected exactly one message, not %+v", msgs) } - msg := n2.msgs[0] + msg := msgs[0] if msg.Type != pb.MsgVoteResp && !msg.Reject { t.Fatal("expected learner to not reject vote") } @@ -588,10 +620,11 @@ func testVoteFromAnyState(t *testing.T, vt pb.MessageType) { if err := r.Step(msg); err != nil { t.Errorf("%s,%s: Step failed: %s", vt, st, err) } - if len(r.msgs) != 1 { - t.Errorf("%s,%s: %d response messages, want 1: %+v", vt, st, len(r.msgs), r.msgs) + msgs := r.readMessages() + if len(msgs) != 1 { + t.Errorf("%s,%s: %d response messages, want 1: %+v", vt, st, len(msgs), msgs) } else { - resp := r.msgs[0] + resp := msgs[0] if resp.Type != voteRespMsgType(vt) { t.Errorf("%s,%s: response message is %s, want %s", vt, st, resp.Type, voteRespMsgType(vt)) @@ -703,8 +736,8 @@ func TestLearnerLogReplication(t *testing.T) { setRandomizedElectionTimeout(n1, n1.electionTimeout) for i := 0; i < n1.electionTimeout; i++ { n1.tick() - n1.maybeVoteForSelf() } + n1.advanceMessagesAfterAppend() nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat}) @@ -719,20 +752,11 @@ func TestLearnerLogReplication(t *testing.T) { nextCommitted := uint64(2) { nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}}) - rd := newReady(n1, &SoftState{}, pb.HardState{}) - nt.send(rd.Messages...) - s1.Append(rd.Entries) - n1.advance(rd) } if n1.raftLog.committed != nextCommitted { t.Errorf("peer 1 wants committed to %d, but still %d", nextCommitted, n1.raftLog.committed) } - { - rd := newReady(n1, &SoftState{}, pb.HardState{}) - nt.send(rd.Messages...) - } - if n1.raftLog.committed != n2.raftLog.committed { t.Errorf("peer 2 wants committed to %d, but still %d", n1.raftLog.committed, n2.raftLog.committed) } @@ -752,10 +776,6 @@ func TestSingleNodeCommit(t *testing.T) { tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) - rd := newReady(r, &SoftState{}, pb.HardState{}) - s.Append(rd.Entries) - r.advance(rd) - sm := tt.peers[1].(*raft) if sm.raftLog.committed != 3 { t.Errorf("committed = %d, want %d", sm.raftLog.committed, 3) @@ -1755,8 +1775,8 @@ func testCandidateResetTerm(t *testing.T, mt pb.MessageType) { c.resetRandomizedElectionTimeout() for i := 0; i < c.randomizedElectionTimeout; i++ { c.tick() - c.maybeVoteForSelf() } + c.advanceMessagesAfterAppend() if c.state != StateCandidate { t.Errorf("state = %s, want %s", c.state, StateCandidate) @@ -1796,7 +1816,7 @@ func testCandidateSelfVoteAfterLostElection(t *testing.T, preVote bool) { // n1 calls an election. sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) - voteMsg := sm.voteSelfOnAdvance + steps := sm.takeMessagesAfterAppend() // n1 hears that n2 already won the election before it has had a // change to sync its vote to disk and account for its self-vote. @@ -1807,7 +1827,7 @@ func testCandidateSelfVoteAfterLostElection(t *testing.T, preVote bool) { } // n1 remains a follower even after its self-vote is delivered. - sm.Step(voteMsg) + sm.stepOrSend(steps) if sm.state != StateFollower { t.Errorf("state = %v, want %v", sm.state, StateFollower) } @@ -1825,10 +1845,10 @@ func TestCandidateDeliversPreCandidateSelfVoteAfterBecomingCandidate(t *testing. // n1 calls an election. sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) - preVoteMsg := sm.voteSelfOnAdvance if sm.state != StatePreCandidate { t.Errorf("state = %v, want %v", sm.state, StatePreCandidate) } + steps := sm.takeMessagesAfterAppend() // n1 receives pre-candidate votes from both other peers before // voting for itself. n1 becomes a candidate. @@ -1841,11 +1861,11 @@ func TestCandidateDeliversPreCandidateSelfVoteAfterBecomingCandidate(t *testing. // n1 remains a candidate even after its delayed pre-vote self-vote is // delivered. - sm.Step(preVoteMsg) - voteMsg := sm.voteSelfOnAdvance + sm.stepOrSend(steps) if sm.state != StateCandidate { t.Errorf("state = %v, want %v", sm.state, StateCandidate) } + steps = sm.takeMessagesAfterAppend() // Its pre-vote self-vote does not make its way to its ProgressTracker. granted, _, _ := sm.prs.TallyVotes() @@ -1861,12 +1881,35 @@ func TestCandidateDeliversPreCandidateSelfVoteAfterBecomingCandidate(t *testing. // n1 becomes the leader once its self-vote is received because now // quorum is reached. - sm.Step(voteMsg) + sm.stepOrSend(steps) if sm.state != StateLeader { t.Errorf("state = %v, want %v", sm.state, StateLeader) } } +func TestLeaderMsgAppSelfAckAfterTermChange(t *testing.T) { + sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) + sm.becomeCandidate() + sm.becomeLeader() + + // n1 proposes a write. + sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}}) + steps := sm.takeMessagesAfterAppend() + + // n1 hears that n2 is the new leader. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term + 1, Type: pb.MsgHeartbeat}) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } + + // n1 advances, ignoring its earlier self-ack of its MsgApp. The + // corresponding MsgAppResp is ignored because it carries an earlier term. + sm.stepOrSend(steps) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } +} + func TestLeaderStepdownWhenQuorumActive(t *testing.T) { sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) @@ -2625,6 +2668,8 @@ func TestBcastBeat(t *testing.T) { for i := 0; i < 10; i++ { mustAppendEntry(sm, pb.Entry{Index: uint64(i) + 1}) } + sm.advanceMessagesAfterAppend() + // slow follower sm.prs.Progress[2].Match, sm.prs.Progress[2].Next = 5, 6 // normal follower @@ -3052,12 +3097,9 @@ func TestLearnerReceiveSnapshot(t *testing.T) { n2 := newTestLearnerRaft(2, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2))) n1.restore(s) - ready := newReady(n1, &SoftState{}, pb.HardState{}) - store.ApplySnapshot(ready.Snapshot) - n1.advance(ready) - - // Force set n1 appplied index. - n1.raftLog.appliedTo(n1.raftLog.committed) + snap := n1.raftLog.nextUnstableSnapshot() + store.ApplySnapshot(*snap) + n1.appliedSnap(snap) nt := newNetwork(n1, n2) @@ -3476,15 +3518,13 @@ func testCampaignWhileLeader(t *testing.T, preVote bool) { // We don't call campaign() directly because it comes after the check // for our current state. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) - for r.maybeVoteForSelf() { - } + r.advanceMessagesAfterAppend() if r.state != StateLeader { t.Errorf("expected single-node election to become leader but got %s", r.state) } term := r.Term r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) - for r.maybeVoteForSelf() { - } + r.advanceMessagesAfterAppend() if r.state != StateLeader { t.Errorf("expected to remain leader but got %s", r.state) } @@ -3705,9 +3745,9 @@ func TestLeaderTransferAfterSnapshot(t *testing.T) { // Apply snapshot and resume progress follower := nt.peers[3].(*raft) - ready := newReady(follower, &SoftState{}, pb.HardState{}) - nt.storage[3].ApplySnapshot(ready.Snapshot) - follower.advance(ready) + snap := follower.raftLog.nextUnstableSnapshot() + nt.storage[3].ApplySnapshot(*snap) + follower.appliedSnap(snap) nt.msgHook = nil nt.send(filtered) @@ -4851,9 +4891,8 @@ func (nw *network) send(msgs ...pb.Message) { if nw.t != nil { nw.t.Log(DescribeMessage(m, nil)) } - p.Step(m) - for p.maybeVoteForSelf() { - } + _ = p.Step(m) + p.advanceMessagesAfterAppend() msgs = append(msgs[1:], nw.filter(p.readMessages())...) } } @@ -4918,9 +4957,9 @@ type connem struct { type blackHole struct{} -func (blackHole) Step(pb.Message) error { return nil } -func (blackHole) readMessages() []pb.Message { return nil } -func (blackHole) maybeVoteForSelf() bool { return false } +func (blackHole) Step(pb.Message) error { return nil } +func (blackHole) readMessages() []pb.Message { return nil } +func (blackHole) advanceMessagesAfterAppend() {} var nopStepper = &blackHole{} diff --git a/rafttest/interaction_env.go b/rafttest/interaction_env.go index 3c245f07..a7dfc0cf 100644 --- a/rafttest/interaction_env.go +++ b/rafttest/interaction_env.go @@ -34,8 +34,10 @@ type Node struct { *raft.RawNode Storage - Config *raft.Config - History []pb.Snapshot + Config *raft.Config + AppendWork []pb.Message // []MsgStorageAppend + ApplyWork []pb.Message // []MsgStorageApply + History []pb.Snapshot } // InteractionEnv facilitates testing of complex interactions between the diff --git a/rafttest/interaction_env_handler.go b/rafttest/interaction_env_handler.go index 73e706fe..9f95bc12 100644 --- a/rafttest/interaction_env_handler.go +++ b/rafttest/interaction_env_handler.go @@ -43,7 +43,7 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string { case "add-nodes": // Example: // - // add-nodes voters=(1 2 3) learners=(4 5) index=2 content=foo + // add-nodes voters=(1 2 3) learners=(4 5) index=2 content=foo async-storage-writes=true err = env.handleAddNodes(t, d) case "campaign": // Example: @@ -67,6 +67,16 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string { // // process-ready 3 err = env.handleProcessReady(t, d) + case "process-append-thread": + // Example: + // + // process-append-thread 3 + err = env.handleProcessAppendThread(t, d) + case "process-apply-thread": + // Example: + // + // process-apply-thread 3 + err = env.handleProcessApplyThread(t, d) case "log-level": // Set the log level. NONE disables all output, including from the test // harness (except errors). diff --git a/rafttest/interaction_env_handler_add_nodes.go b/rafttest/interaction_env_handler_add_nodes.go index f164a6d2..f086aed6 100644 --- a/rafttest/interaction_env_handler_add_nodes.go +++ b/rafttest/interaction_env_handler_add_nodes.go @@ -48,6 +48,8 @@ func (env *InteractionEnv) handleAddNodes(t *testing.T, d datadriven.TestData) e cfg.Applied = snap.Metadata.Index case "content": arg.Scan(t, i, &snap.Data) + case "async-storage-writes": + arg.Scan(t, i, &cfg.AsyncStorageWrites) } } } diff --git a/rafttest/interaction_env_handler_process_append_thread.go b/rafttest/interaction_env_handler_process_append_thread.go new file mode 100644 index 00000000..3f74988a --- /dev/null +++ b/rafttest/interaction_env_handler_process_append_thread.go @@ -0,0 +1,97 @@ +// Copyright 2022 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rafttest + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/datadriven" + + "go.etcd.io/raft/v3" + "go.etcd.io/raft/v3/raftpb" +) + +func (env *InteractionEnv) handleProcessAppendThread(t *testing.T, d datadriven.TestData) error { + idxs := nodeIdxs(t, d) + for _, idx := range idxs { + var err error + if len(idxs) > 1 { + fmt.Fprintf(env.Output, "> %d processing append thread\n", idx+1) + env.withIndent(func() { err = env.ProcessAppendThread(idx) }) + } else { + err = env.ProcessAppendThread(idx) + } + if err != nil { + return err + } + } + return nil +} + +// ProcessAppendThread runs processes a single message on the "append" thread of +// the node with the given index. +func (env *InteractionEnv) ProcessAppendThread(idx int) error { + n := &env.Nodes[idx] + if len(n.AppendWork) == 0 { + env.Output.WriteString("no append work to perform") + return nil + } + m := n.AppendWork[0] + n.AppendWork = n.AppendWork[1:] + + resps := m.Responses + m.Responses = nil + env.Output.WriteString("Processing:\n") + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + var st raftpb.HardState + if m.HardState != nil { + st = *m.HardState + } + var snap raftpb.Snapshot + if m.Snapshot != nil { + snap = *m.Snapshot + } + if err := processAppend(n, st, m.Entries, snap); err != nil { + return err + } + + env.Output.WriteString("Responses:\n") + for _, m := range resps { + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + } + env.Messages = append(env.Messages, resps...) + return nil +} + +func processAppend(n *Node, st raftpb.HardState, ents []raftpb.Entry, snap raftpb.Snapshot) error { + // TODO(tbg): the order of operations here is not necessarily safe. See: + // https://github.com/etcd-io/etcd/pull/10861 + s := n.Storage + if !raft.IsEmptyHardState(st) { + if err := s.SetHardState(st); err != nil { + return err + } + } + if err := s.Append(ents); err != nil { + return err + } + if !raft.IsEmptySnap(snap) { + if err := s.ApplySnapshot(snap); err != nil { + return err + } + } + return nil +} diff --git a/rafttest/interaction_env_handler_process_apply_thread.go b/rafttest/interaction_env_handler_process_apply_thread.go new file mode 100644 index 00000000..d21317e0 --- /dev/null +++ b/rafttest/interaction_env_handler_process_apply_thread.go @@ -0,0 +1,111 @@ +// Copyright 2022 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rafttest + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/datadriven" + + "go.etcd.io/raft/v3" + "go.etcd.io/raft/v3/raftpb" +) + +func (env *InteractionEnv) handleProcessApplyThread(t *testing.T, d datadriven.TestData) error { + idxs := nodeIdxs(t, d) + for _, idx := range idxs { + var err error + if len(idxs) > 1 { + fmt.Fprintf(env.Output, "> %d processing apply thread\n", idx+1) + env.withIndent(func() { err = env.ProcessApplyThread(idx) }) + } else { + err = env.ProcessApplyThread(idx) + } + if err != nil { + return err + } + } + return nil +} + +// ProcessApplyThread runs processes a single message on the "apply" thread of +// the node with the given index. +func (env *InteractionEnv) ProcessApplyThread(idx int) error { + n := &env.Nodes[idx] + if len(n.ApplyWork) == 0 { + env.Output.WriteString("no apply work to perform") + return nil + } + m := n.ApplyWork[0] + n.ApplyWork = n.ApplyWork[1:] + + resps := m.Responses + m.Responses = nil + env.Output.WriteString("Processing:\n") + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + if err := processApply(n, m.Entries); err != nil { + return err + } + + env.Output.WriteString("Responses:\n") + for _, m := range resps { + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + } + env.Messages = append(env.Messages, resps...) + return nil +} + +func processApply(n *Node, ents []raftpb.Entry) error { + for _, ent := range ents { + var update []byte + var cs *raftpb.ConfState + switch ent.Type { + case raftpb.EntryConfChange: + var cc raftpb.ConfChange + if err := cc.Unmarshal(ent.Data); err != nil { + return err + } + update = cc.Context + cs = n.RawNode.ApplyConfChange(cc) + case raftpb.EntryConfChangeV2: + var cc raftpb.ConfChangeV2 + if err := cc.Unmarshal(ent.Data); err != nil { + return err + } + cs = n.RawNode.ApplyConfChange(cc) + update = cc.Context + default: + update = ent.Data + } + + // Record the new state by starting with the current state and applying + // the command. + lastSnap := n.History[len(n.History)-1] + var snap raftpb.Snapshot + snap.Data = append(snap.Data, lastSnap.Data...) + // NB: this hard-codes an "appender" state machine. + snap.Data = append(snap.Data, update...) + snap.Metadata.Index = ent.Index + snap.Metadata.Term = ent.Term + if cs == nil { + sl := n.History + cs = &sl[len(sl)-1].Metadata.ConfState + } + snap.Metadata.ConfState = *cs + n.History = append(n.History, snap) + } + return nil +} diff --git a/rafttest/interaction_env_handler_process_ready.go b/rafttest/interaction_env_handler_process_ready.go index 6b2ef18e..e72d3d9d 100644 --- a/rafttest/interaction_env_handler_process_ready.go +++ b/rafttest/interaction_env_handler_process_ready.go @@ -44,65 +44,39 @@ func (env *InteractionEnv) handleProcessReady(t *testing.T, d datadriven.TestDat // ProcessReady runs Ready handling on the node with the given index. func (env *InteractionEnv) ProcessReady(idx int) error { // TODO(tbg): Allow simulating crashes here. - rn, s := env.Nodes[idx].RawNode, env.Nodes[idx].Storage - rd := rn.Ready() + n := &env.Nodes[idx] + rd := n.Ready() env.Output.WriteString(raft.DescribeReady(rd, defaultEntryFormatter)) - // TODO(tbg): the order of operations here is not necessarily safe. See: - // https://github.com/etcd-io/etcd/pull/10861 - if !raft.IsEmptyHardState(rd.HardState) { - if err := s.SetHardState(rd.HardState); err != nil { + + if !n.Config.AsyncStorageWrites { + if err := processAppend(n, rd.HardState, rd.Entries, rd.Snapshot); err != nil { return err } - } - if err := s.Append(rd.Entries); err != nil { - return err - } - if !raft.IsEmptySnap(rd.Snapshot) { - if err := s.ApplySnapshot(rd.Snapshot); err != nil { + if err := processApply(n, rd.CommittedEntries); err != nil { return err } } - for _, ent := range rd.CommittedEntries { - var update []byte - var cs *raftpb.ConfState - switch ent.Type { - case raftpb.EntryConfChange: - var cc raftpb.ConfChange - if err := cc.Unmarshal(ent.Data); err != nil { - return err + + for _, m := range rd.Messages { + if raft.IsLocalMsgTarget(m.To) { + if !n.Config.AsyncStorageWrites { + panic("unexpected local msg target") } - update = cc.Context - cs = rn.ApplyConfChange(cc) - case raftpb.EntryConfChangeV2: - var cc raftpb.ConfChangeV2 - if err := cc.Unmarshal(ent.Data); err != nil { - return err + switch m.Type { + case raftpb.MsgStorageAppend: + n.AppendWork = append(n.AppendWork, m) + case raftpb.MsgStorageApply: + n.ApplyWork = append(n.ApplyWork, m) + default: + panic(fmt.Sprintf("unexpected message type %s", m.Type)) } - cs = rn.ApplyConfChange(cc) - update = cc.Context - default: - update = ent.Data - } - - // Record the new state by starting with the current state and applying - // the command. - lastSnap := env.Nodes[idx].History[len(env.Nodes[idx].History)-1] - var snap raftpb.Snapshot - snap.Data = append(snap.Data, lastSnap.Data...) - // NB: this hard-codes an "appender" state machine. - snap.Data = append(snap.Data, update...) - snap.Metadata.Index = ent.Index - snap.Metadata.Term = ent.Term - if cs == nil { - sl := env.Nodes[idx].History - cs = &sl[len(sl)-1].Metadata.ConfState + } else { + env.Messages = append(env.Messages, m) } - snap.Metadata.ConfState = *cs - env.Nodes[idx].History = append(env.Nodes[idx].History, snap) } - env.Messages = append(env.Messages, rd.Messages...) - - rn.Advance(rd) + if !n.Config.AsyncStorageWrites { + n.Advance(rd) + } return nil } diff --git a/rafttest/interaction_env_handler_stabilize.go b/rafttest/interaction_env_handler_stabilize.go index c4579be3..8fa6a8e8 100644 --- a/rafttest/interaction_env_handler_stabilize.go +++ b/rafttest/interaction_env_handler_stabilize.go @@ -31,22 +31,29 @@ func (env *InteractionEnv) handleStabilize(t *testing.T, d datadriven.TestData) // Stabilize repeatedly runs Ready handling on and message delivery to the set // of nodes specified via the idxs slice until reaching a fixed point. func (env *InteractionEnv) Stabilize(idxs ...int) error { - var nodes []Node - for _, idx := range idxs { - nodes = append(nodes, env.Nodes[idx]) - } - if len(nodes) == 0 { - nodes = env.Nodes + var nodes []*Node + if len(idxs) != 0 { + for _, idx := range idxs { + nodes = append(nodes, &env.Nodes[idx]) + } + } else { + for i := range env.Nodes { + nodes = append(nodes, &env.Nodes[i]) + } } for { done := true for _, rn := range nodes { if rn.HasReady() { - done = false idx := int(rn.Status().ID - 1) fmt.Fprintf(env.Output, "> %d handling Ready\n", idx+1) - env.withIndent(func() { env.ProcessReady(idx) }) + var err error + env.withIndent(func() { err = env.ProcessReady(idx) }) + if err != nil { + return err + } + done = false } } for _, rn := range nodes { @@ -59,6 +66,30 @@ func (env *InteractionEnv) Stabilize(idxs ...int) error { done = false } } + for _, rn := range nodes { + idx := int(rn.Status().ID - 1) + if len(rn.AppendWork) > 0 { + fmt.Fprintf(env.Output, "> %d processing append thread\n", idx+1) + for len(rn.AppendWork) > 0 { + var err error + env.withIndent(func() { err = env.ProcessAppendThread(idx) }) + if err != nil { + return err + } + } + done = false + } + } + for _, rn := range nodes { + idx := int(rn.Status().ID - 1) + if len(rn.ApplyWork) > 0 { + fmt.Fprintf(env.Output, "> %d processing apply thread\n", idx+1) + for len(rn.ApplyWork) > 0 { + env.withIndent(func() { env.ProcessApplyThread(idx) }) + } + done = false + } + } if done { return nil } diff --git a/rawnode.go b/rawnode.go index 82f2b0bf..9f463156 100644 --- a/rawnode.go +++ b/rawnode.go @@ -32,9 +32,10 @@ var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found") // The methods of this struct correspond to the methods of Node and are described // more fully there. type RawNode struct { - raft *raft - prevSoftSt *SoftState - prevHardSt pb.HardState + raft *raft + prevSoftSt *SoftState + prevHardSt pb.HardState + stepsOnAdvance []pb.Message } // NewRawNode instantiates a RawNode from the given configuration. @@ -108,14 +109,14 @@ func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState { // Step advances the state machine using the given message. func (rn *RawNode) Step(m pb.Message) error { - // ignore unexpected local messages receiving over network - if IsLocalMsg(m.Type) { + // Ignore unexpected local messages receiving over network. + if IsLocalMsg(m.Type) && !IsLocalMsgTarget(m.From) { return ErrStepLocalMsg } - if pr := rn.raft.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) { - return rn.raft.Step(m) + if IsResponseMsg(m.Type) && !IsLocalMsgTarget(m.From) && rn.raft.prs.Progress[m.From] == nil { + return ErrStepPeerNotFound } - return ErrStepPeerNotFound + return rn.raft.Step(m) } // Ready returns the outstanding work that the application needs to handle. This @@ -147,10 +148,26 @@ func (rn *RawNode) acceptReady(rd Ready) { if len(rd.ReadStates) != 0 { rn.raft.readStates = nil } + if !rn.raft.asyncStorageWrites { + if len(rn.stepsOnAdvance) != 0 { + rn.raft.logger.Panicf("two accepted Ready structs without call to Advance") + } + for _, m := range rn.raft.msgsAfterAppend { + if m.To == rn.raft.id { + rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) + } + } + if needStorageAppend(rd, false /* haveMsgsAfterAppend */) { + m := newStorageAppendRespMsg(rn.raft, rd) + rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) + } + if needStorageApply(rd) { + m := newStorageApplyRespMsg(rn.raft, rd.CommittedEntries) + rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) + } + } rn.raft.msgs = nil - // NB: this does not do anything yet, as entries and snapshots are always - // stabilized on the next Advance and committed entries are always applied - // by the next Advance. + rn.raft.msgsAfterAppend = nil rn.raft.raftLog.acceptUnstable() if len(rd.CommittedEntries) > 0 { ents := rd.CommittedEntries @@ -170,7 +187,10 @@ func (rn *RawNode) HasReady() bool { if r.raftLog.hasNextUnstableSnapshot() { return true } - if len(r.msgs) > 0 || r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(true /* allowUnstable */) { + if len(r.msgs) > 0 || len(r.msgsAfterAppend) > 0 { + return true + } + if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(!rn.raft.asyncStorageWrites) { return true } if len(r.readStates) != 0 { @@ -181,8 +201,21 @@ func (rn *RawNode) HasReady() bool { // Advance notifies the RawNode that the application has applied and saved progress in the // last Ready results. -func (rn *RawNode) Advance(rd Ready) { - rn.raft.advance(rd) +// +// NOTE: Advance must not be called when using AsyncStorageWrites. Response messages from +// the local append and apply threads take its place. +func (rn *RawNode) Advance(_ Ready) { + // The actions performed by this function are encoded into stepsOnAdvance in + // acceptReady. In earlier versions of this library, they were computed from + // the provided Ready struct. Retain the unused parameter for compatability. + if rn.raft.asyncStorageWrites { + rn.raft.logger.Panicf("Advance must not be called when using AsyncStorageWrites") + } + for i, m := range rn.stepsOnAdvance { + _ = rn.raft.Step(m) + rn.stepsOnAdvance[i] = pb.Message{} + } + rn.stepsOnAdvance = rn.stepsOnAdvance[:0] } // Status returns the current status of the given group. This allocates, see diff --git a/rawnode_test.go b/rawnode_test.go index d9e8551b..ceaed813 100644 --- a/rawnode_test.go +++ b/rawnode_test.go @@ -342,6 +342,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { if len(rd.Entries) > 0 { t.Fatal("expected no more entries") } + rawNode.Advance(rd) if tc.exp2 == nil { return } @@ -370,6 +371,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { if exp := tc.exp2; !reflect.DeepEqual(exp, cs) { t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs) } + rawNode.Advance(rd) }) } } diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt new file mode 100644 index 00000000..3831edcb --- /dev/null +++ b/testdata/async_storage_writes.txt @@ -0,0 +1,785 @@ +# Build a pipeline of in-progress entries to append. Let the pipeline grow to a +# depth of 2 before we start allowing work on the append thread to complete. + +add-nodes 3 voters=(1,2,3) index=10 async-storage-writes=true +---- +INFO 1 switched to configuration voters=(1 2 3) +INFO 1 became follower at term 0 +INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] +INFO 2 switched to configuration voters=(1 2 3) +INFO 2 became follower at term 0 +INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] +INFO 3 switched to configuration voters=(1 2 3) +INFO 3 became follower at term 0 +INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] + +campaign 1 +---- +INFO 1 is starting a new election at term 0 +INFO 1 became candidate at term 1 +INFO 1 [logterm: 1, index: 10] sent MsgVote request to 2 at term 1 +INFO 1 [logterm: 1, index: 10] sent MsgVote request to 3 at term 1 + +stabilize +---- +> 1 handling Ready + Ready MustSync=true: + Lead:0 State:StateCandidate + HardState Term:1 Vote:1 Commit:10 + Messages: + 1->2 MsgVote Term:1 Log:1/10 + 1->3 MsgVote Term:1 Log:1/10 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[1->1 MsgVoteResp Term:1 Log:0/0, AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0] +> 2 receiving messages + 1->2 MsgVote Term:1 Log:1/10 + INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] + INFO 2 became follower at term 1 + INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 +> 3 receiving messages + 1->3 MsgVote Term:1 Log:1/10 + INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1] + INFO 3 became follower at term 1 + INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 + Responses: + 1->1 MsgVoteResp Term:1 Log:0/0 + AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:10 + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[2->1 MsgVoteResp Term:1 Log:0/0, AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0] +> 3 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:10 + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[3->1 MsgVoteResp Term:1 Log:0/0, AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0] +> 1 receiving messages + 1->1 MsgVoteResp Term:1 Log:0/0 + INFO 1 received MsgVoteResp from 1 at term 1 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 + Responses: + 2->1 MsgVoteResp Term:1 Log:0/0 + AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 + Responses: + 3->1 MsgVoteResp Term:1 Log:0/0 + AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 +> 1 receiving messages + 2->1 MsgVoteResp Term:1 Log:0/0 + INFO 1 received MsgVoteResp from 2 at term 1 + INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 1 became leader at term 1 + 3->1 MsgVoteResp Term:1 Log:0/0 +> 2 receiving messages + AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 +> 3 receiving messages + AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 +> 1 handling Ready + Ready MustSync=true: + Lead:1 State:StateLeader + Entries: + 1/11 EntryNormal "" + Messages: + 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:1 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[1->1 MsgAppResp Term:1 Log:0/11, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11] +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + 1->1 MsgAppResp Term:1 Log:0/11 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 +> 2 handling Ready + Ready MustSync=true: + Lead:1 State:StateFollower + Entries: + 1/11 EntryNormal "" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[2->1 MsgAppResp Term:1 Log:0/11, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11] +> 3 handling Ready + Ready MustSync=true: + Lead:1 State:StateFollower + Entries: + 1/11 EntryNormal "" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[3->1 MsgAppResp Term:1 Log:0/11, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11] +> 1 receiving messages + 1->1 MsgAppResp Term:1 Log:0/11 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + 2->1 MsgAppResp Term:1 Log:0/11 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + 3->1 MsgAppResp Term:1 Log:0/11 + 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 receiving messages + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 +> 3 receiving messages + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:11 + CommittedEntries: + 1/11 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/11 Commit:11 + 1->3 MsgApp Term:1 Log:1/11 Commit:11 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0] + 1->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11] +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/11 Commit:11 +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/11 Commit:11 +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + Responses: + AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11 +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:11 + CommittedEntries: + 1/11 EntryNormal "" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[2->1 MsgAppResp Term:1 Log:0/11, AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0] + 2->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11] +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:11 + CommittedEntries: + 1/11 EntryNormal "" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[3->1 MsgAppResp Term:1 Log:0/11, AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0] + 3->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11] +> 1 receiving messages + AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + Responses: + 2->1 MsgAppResp Term:1 Log:0/11 + AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + Responses: + 3->1 MsgAppResp Term:1 Log:0/11 + AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11 +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 +> 2 receiving messages + AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11 +> 3 receiving messages + AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11 + +propose 1 prop_1 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/12 EntryNormal "prop_1" + Messages: + 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:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[1->1 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 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"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/12 EntryNormal "prop_1" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[2->1 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] +> 3 handling Ready + Ready MustSync=true: + Entries: + 1/12 EntryNormal "prop_1" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[3->1 MsgAppResp Term:1 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12] + +propose 1 prop_2 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "prop_2" + Messages: + 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:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[1->1 MsgAppResp Term:1 Log:0/13, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 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"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "prop_2" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[2->1 MsgAppResp Term:1 Log:0/13, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13] +> 3 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "prop_2" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[3->1 MsgAppResp Term:1 Log:0/13, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + 1->1 MsgAppResp Term:1 Log:0/12 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + 2->1 MsgAppResp Term:1 Log:0/12 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + 3->1 MsgAppResp Term:1 Log:0/12 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 +2->1 MsgAppResp Term:1 Log:0/12 +3->1 MsgAppResp Term:1 Log:0/12 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 + +propose 1 prop_3 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:12 + Entries: + 1/14 EntryNormal "prop_3" + CommittedEntries: + 1/12 EntryNormal "prop_1" + Messages: + 1->2 MsgApp Term:1 Log:1/13 Commit:12 + 1->3 MsgApp Term:1 Log:1/13 Commit:12 + 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 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[1->1 MsgAppResp Term:1 Log:0/14, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14] + 1->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/12] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/13 Commit:12 +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 +1->3 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:12 + Entries: + 1/14 EntryNormal "prop_3" + CommittedEntries: + 1/12 EntryNormal "prop_1" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[2->1 MsgAppResp Term:1 Log:0/13, 2->1 MsgAppResp Term:1 Log:0/14, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14] + 2->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/12] +> 3 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:12 + Entries: + 1/14 EntryNormal "prop_3" + CommittedEntries: + 1/12 EntryNormal "prop_1" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[3->1 MsgAppResp Term:1 Log:0/13, 3->1 MsgAppResp Term:1 Log:0/14, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14] + 3->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/12] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + 1->1 MsgAppResp Term:1 Log:0/13 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + 2->1 MsgAppResp Term:1 Log:0/13 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + 3->1 MsgAppResp Term:1 Log:0/13 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/13 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 +2->1 MsgAppResp Term:1 Log:0/13 +3->1 MsgAppResp Term:1 Log:0/13 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 + +propose 1 prop_4 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:13 + Entries: + 1/15 EntryNormal "prop_4" + CommittedEntries: + 1/13 EntryNormal "prop_2" + Messages: + 1->2 MsgApp Term:1 Log:1/14 Commit:13 + 1->3 MsgApp Term:1 Log:1/14 Commit:13 + 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 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[1->1 MsgAppResp Term:1 Log:0/15, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] + 1->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/13] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/14 Commit:13 +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 +1->3 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:13 + Entries: + 1/15 EntryNormal "prop_4" + CommittedEntries: + 1/13 EntryNormal "prop_2" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[2->1 MsgAppResp Term:1 Log:0/14, 2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] + 2->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/13] +> 3 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:13 + Entries: + 1/15 EntryNormal "prop_4" + CommittedEntries: + 1/13 EntryNormal "prop_2" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[3->1 MsgAppResp Term:1 Log:0/14, 3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] + 3->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/13] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 + Responses: + 1->1 MsgAppResp Term:1 Log:0/14 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 + Responses: + 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/14 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 + Responses: + 3->1 MsgAppResp Term:1 Log:0/13 + 3->1 MsgAppResp Term:1 Log:0/14 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/12 +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/12 +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/12 + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/14 +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 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/12 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/12 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/12 + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:14 + CommittedEntries: + 1/14 EntryNormal "prop_3" + Messages: + 1->2 MsgApp Term:1 Log:1/15 Commit:14 + 1->3 MsgApp Term:1 Log:1/15 Commit:14 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] + 1->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/14] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/15 Commit:14 +1->3 MsgApp Term:1 Log:1/15 Commit:14 + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:14 + CommittedEntries: + 1/14 EntryNormal "prop_3" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] + 2->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14] +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:14 + CommittedEntries: + 1/14 EntryNormal "prop_3" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] + 3->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 + Responses: + 1->1 MsgAppResp Term:1 Log:0/15 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 + Responses: + 2->1 MsgAppResp Term:1 Log:0/14 + 2->1 MsgAppResp Term:1 Log:0/15 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 + Responses: + 3->1 MsgAppResp Term:1 Log:0/14 + 3->1 MsgAppResp Term:1 Log:0/15 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/13 +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/13 +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/13 + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/15 +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 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/13 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/13 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/13 + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:15 + CommittedEntries: + 1/15 EntryNormal "prop_4" + Messages: + 1->2 MsgApp Term:1 Log:1/15 Commit:15 + 1->3 MsgApp Term:1 Log:1/15 Commit:15 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0] + 1->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/15 Commit:15 +1->3 MsgApp Term:1 Log:1/15 Commit:15 + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:15 + CommittedEntries: + 1/15 EntryNormal "prop_4" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0] + 2->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15] +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:15 + CommittedEntries: + 1/15 EntryNormal "prop_4" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0] + 3->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/15] + +process-append-thread 2 3 +---- +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 + Responses: + 2->1 MsgAppResp Term:1 Log:0/15 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 + Responses: + 3->1 MsgAppResp Term:1 Log:0/15 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/14 +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14 +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14 + +deliver-msgs 1 2 3 +---- +2->1 MsgAppResp Term:1 Log:0/15 +3->1 MsgAppResp Term:1 Log:0/15 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/14 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14 + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + +> 3 handling Ready + + +process-append-thread 2 3 +---- +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + Responses: + 2->1 MsgAppResp Term:1 Log:0/15 + AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + Responses: + 3->1 MsgAppResp Term:1 Log:0/15 + AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15 +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15 +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/15 + +deliver-msgs 1 2 3 +---- +2->1 MsgAppResp Term:1 Log:0/15 +3->1 MsgAppResp Term:1 Log:0/15 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15 +AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15 +AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/15 + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + +> 3 handling Ready + + +stabilize +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 + Responses: + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + Responses: + AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 +> 1 receiving messages + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 + AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 diff --git a/util.go b/util.go index d0872182..d5ed30b3 100644 --- a/util.go +++ b/util.go @@ -41,20 +41,26 @@ func max(a, b uint64) uint64 { } var isLocalMsg = [...]bool{ - pb.MsgHup: true, - pb.MsgBeat: true, - pb.MsgUnreachable: true, - pb.MsgSnapStatus: true, - pb.MsgCheckQuorum: true, + pb.MsgHup: true, + pb.MsgBeat: true, + pb.MsgUnreachable: true, + pb.MsgSnapStatus: true, + pb.MsgCheckQuorum: true, + pb.MsgStorageAppend: true, + pb.MsgStorageAppendResp: true, + pb.MsgStorageApply: true, + pb.MsgStorageApplyResp: true, } var isResponseMsg = [...]bool{ - pb.MsgAppResp: true, - pb.MsgVoteResp: true, - pb.MsgHeartbeatResp: true, - pb.MsgUnreachable: true, - pb.MsgReadIndexResp: true, - pb.MsgPreVoteResp: true, + pb.MsgAppResp: true, + pb.MsgVoteResp: true, + pb.MsgHeartbeatResp: true, + pb.MsgUnreachable: true, + pb.MsgReadIndexResp: true, + pb.MsgPreVoteResp: true, + pb.MsgStorageAppendResp: true, + pb.MsgStorageApplyResp: true, } func isMsgInArray(msgt pb.MessageType, arr []bool) bool { @@ -70,6 +76,10 @@ func IsResponseMsg(msgt pb.MessageType) bool { return isMsgInArray(msgt, isResponseMsg[:]) } +func IsLocalMsgTarget(id uint64) bool { + return id == LocalAppendThread || id == LocalApplyThread +} + // voteResponseType maps vote and prevote message types to their corresponding responses. func voteRespMsgType(msgt pb.MessageType) pb.MessageType { switch msgt { @@ -153,7 +163,8 @@ type EntryFormatter func([]byte) string // Message for debugging. func DescribeMessage(m pb.Message, f EntryFormatter) string { var buf bytes.Buffer - fmt.Fprintf(&buf, "%x->%x %v Term:%d Log:%d/%d", m.From, m.To, m.Type, m.Term, m.LogTerm, m.Index) + fmt.Fprintf(&buf, "%s->%s %v Term:%d Log:%d/%d", + describeTarget(m.From), describeTarget(m.To), m.Type, m.Term, m.LogTerm, m.Index) if m.Reject { fmt.Fprintf(&buf, " Rejected (Hint: %d)", m.RejectHint) } @@ -170,12 +181,38 @@ func DescribeMessage(m pb.Message, f EntryFormatter) string { } fmt.Fprint(&buf, "]") } + if m.HardState != nil { + fmt.Fprintf(&buf, " HardState: %s", DescribeHardState(*m.HardState)) + } if s := m.Snapshot; s != nil && !IsEmptySnap(*s) { fmt.Fprintf(&buf, " Snapshot: %s", DescribeSnapshot(*s)) } + if len(m.Responses) > 0 { + fmt.Fprintf(&buf, " Responses:[") + for i, m := range m.Responses { + if i != 0 { + buf.WriteString(", ") + } + buf.WriteString(DescribeMessage(m, f)) + } + fmt.Fprintf(&buf, "]") + } return buf.String() } +func describeTarget(id uint64) string { + switch id { + case None: + return "None" + case LocalAppendThread: + return "AppendThread" + case LocalApplyThread: + return "ApplyThread" + default: + return fmt.Sprintf("%x", id) + } +} + // PayloadSize is the size of the payload of this Entry. Notably, it does not // depend on its Index or Term. func PayloadSize(e pb.Entry) int { diff --git a/util_test.go b/util_test.go index d8f69dbb..e711ec16 100644 --- a/util_test.go +++ b/util_test.go @@ -89,6 +89,10 @@ func TestIsLocalMsg(t *testing.T) { {pb.MsgReadIndexResp, false}, {pb.MsgPreVote, false}, {pb.MsgPreVoteResp, false}, + {pb.MsgStorageAppend, true}, + {pb.MsgStorageAppendResp, true}, + {pb.MsgStorageApply, true}, + {pb.MsgStorageApplyResp, true}, } for _, tt := range tests { @@ -122,6 +126,10 @@ func TestIsResponseMsg(t *testing.T) { {pb.MsgReadIndexResp, true}, {pb.MsgPreVote, false}, {pb.MsgPreVoteResp, true}, + {pb.MsgStorageAppend, false}, + {pb.MsgStorageAppendResp, true}, + {pb.MsgStorageApply, false}, + {pb.MsgStorageApplyResp, true}, } for i, tt := range tests { From 3063891ba2e059ecb89b5908ac31c46bf5d6dbaa Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 2 Nov 2022 16:13:28 -0400 Subject: [PATCH 07/13] raft: move asyncStorageWrites to RawNode struct This commit makes it more clear that the asyncStorageWrites handling is entirely local to RawNode and that the raft object always operates in "async storage" mode. Signed-off-by: Nathan VanBenschoten --- node.go | 10 ++++++---- raft.go | 6 ++---- rawnode.go | 15 ++++++++++----- 3 files changed, 18 insertions(+), 13 deletions(-) diff --git a/node.go b/node.go index 5afeb44d..731e1143 100644 --- a/node.go +++ b/node.go @@ -408,7 +408,7 @@ func (n *node) run() { n.rn.Tick() case readyc <- rd: n.rn.acceptReady(rd) - if !n.rn.raft.asyncStorageWrites { + if !n.rn.asyncStorageWrites { advancec = n.advancec } else { rd = Ready{} @@ -579,10 +579,12 @@ func (n *node) ReadIndex(ctx context.Context, rctx []byte) error { return n.step(ctx, pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}}) } -func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { +// TODO(nvanbenschoten): move this function and the functions below it to +// rawnode.go. +func newReady(r *raft, asyncStorageWrites bool, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { rd := Ready{ Entries: r.raftLog.nextUnstableEnts(), - CommittedEntries: r.raftLog.nextCommittedEnts(!r.asyncStorageWrites), + CommittedEntries: r.raftLog.nextCommittedEnts(!asyncStorageWrites), Messages: r.msgs, } if softSt := r.softState(); !softSt.equal(prevSoftSt) { @@ -599,7 +601,7 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { } rd.MustSync = MustSync(r.hardState(), prevHardSt, len(rd.Entries)) - if r.asyncStorageWrites { + if asyncStorageWrites { // If async storage writes are enabled, enqueue messages to // local storage threads, where applicable. if needStorageAppend(rd, len(r.msgsAfterAppend) > 0) { diff --git a/raft.go b/raft.go index 33b058bc..58eccb17 100644 --- a/raft.go +++ b/raft.go @@ -372,9 +372,8 @@ type raft struct { // only leader keeps heartbeatElapsed. heartbeatElapsed int - asyncStorageWrites bool - checkQuorum bool - preVote bool + checkQuorum bool + preVote bool heartbeatTimeout int electionTimeout int @@ -417,7 +416,6 @@ func newRaft(c *Config) *raft { electionTimeout: c.ElectionTick, heartbeatTimeout: c.HeartbeatTick, logger: c.Logger, - asyncStorageWrites: c.AsyncStorageWrites, checkQuorum: c.CheckQuorum, preVote: c.PreVote, readOnly: newReadOnly(c.ReadOnlyOption), diff --git a/rawnode.go b/rawnode.go index 9f463156..11cb877b 100644 --- a/rawnode.go +++ b/rawnode.go @@ -32,7 +32,10 @@ var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found") // The methods of this struct correspond to the methods of Node and are described // more fully there. type RawNode struct { - raft *raft + raft *raft + asyncStorageWrites bool + + // Mutable fields. prevSoftSt *SoftState prevHardSt pb.HardState stepsOnAdvance []pb.Message @@ -50,6 +53,7 @@ func NewRawNode(config *Config) (*RawNode, error) { rn := &RawNode{ raft: r, } + rn.asyncStorageWrites = config.AsyncStorageWrites rn.prevSoftSt = r.softState() rn.prevHardSt = r.hardState() return rn, nil @@ -132,7 +136,7 @@ func (rn *RawNode) Ready() Ready { // readyWithoutAccept returns a Ready. This is a read-only operation, i.e. there // is no obligation that the Ready must be handled. func (rn *RawNode) readyWithoutAccept() Ready { - return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt) + return newReady(rn.raft, rn.asyncStorageWrites, rn.prevSoftSt, rn.prevHardSt) } // acceptReady is called when the consumer of the RawNode has decided to go @@ -148,7 +152,7 @@ func (rn *RawNode) acceptReady(rd Ready) { if len(rd.ReadStates) != 0 { rn.raft.readStates = nil } - if !rn.raft.asyncStorageWrites { + if !rn.asyncStorageWrites { if len(rn.stepsOnAdvance) != 0 { rn.raft.logger.Panicf("two accepted Ready structs without call to Advance") } @@ -177,6 +181,7 @@ func (rn *RawNode) acceptReady(rd Ready) { // HasReady called when RawNode user need to check if any Ready pending. func (rn *RawNode) HasReady() bool { + // TODO(nvanbenschoten): order these cases in terms of cost and frequency. r := rn.raft if !r.softState().equal(rn.prevSoftSt) { return true @@ -190,7 +195,7 @@ func (rn *RawNode) HasReady() bool { if len(r.msgs) > 0 || len(r.msgsAfterAppend) > 0 { return true } - if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(!rn.raft.asyncStorageWrites) { + if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(!rn.asyncStorageWrites) { return true } if len(r.readStates) != 0 { @@ -208,7 +213,7 @@ func (rn *RawNode) Advance(_ Ready) { // The actions performed by this function are encoded into stepsOnAdvance in // acceptReady. In earlier versions of this library, they were computed from // the provided Ready struct. Retain the unused parameter for compatability. - if rn.raft.asyncStorageWrites { + if rn.asyncStorageWrites { rn.raft.logger.Panicf("Advance must not be called when using AsyncStorageWrites") } for i, m := range rn.stepsOnAdvance { From 897e1cfe96547b2f4f85080e36f6a90aa77a1858 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 25 Oct 2022 00:09:41 -0400 Subject: [PATCH 08/13] raft: add data-driven test for subtle async storage write aba problem This commit adds a new data-driven test the reproduces a scenario similar to the one described in newStorageAppendRespMsg, exercising a few interesting interactions between asynchronous storage writes, term changes, and log truncation. Signed-off-by: Nathan VanBenschoten --- log_unstable.go | 4 + node.go | 1 - .../interaction_env_handler_deliver_msgs.go | 2 +- rafttest/interaction_env_handler_stabilize.go | 12 +- testdata/async_storage_writes.txt | 3 + .../async_storage_writes_append_aba_race.txt | 493 ++++++++++++++++++ 6 files changed, 510 insertions(+), 5 deletions(-) create mode 100644 testdata/async_storage_writes_append_aba_race.txt diff --git a/log_unstable.go b/log_unstable.go index 6e4298de..740367ba 100644 --- a/log_unstable.go +++ b/log_unstable.go @@ -135,10 +135,12 @@ func (u *unstable) stableTo(i, t uint64) { gt, ok := u.maybeTerm(i) if !ok { // Unstable entry missing. Ignore. + u.logger.Infof("entry at index %d missing from unstable log; ignoring", i) return } if i < u.offset { // Index matched unstable snapshot, not unstable entry. Ignore. + u.logger.Infof("entry at index %d matched unstable snapshot; ignoring", i) return } if gt != t { @@ -146,6 +148,8 @@ func (u *unstable) stableTo(i, t uint64) { // This is possible if part or all of the unstable log was replaced // between that time that a set of entries started to be written to // stable storage and when they finished. + u.logger.Infof("entry at (index,term)=(%d,%d) mismatched with "+ + "entry at (%d,%d) in unstable log; ignoring", i, t, i, gt) return } num := int(i + 1 - u.offset) diff --git a/node.go b/node.go index 731e1143..33940013 100644 --- a/node.go +++ b/node.go @@ -766,7 +766,6 @@ func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { // attest that this (index, term) is correct at the current term, in case the // MsgStorageAppend that contained the last entry in the unstable slice carried // an earlier term and was dropped. - // TODO(nvanbenschoten): test this behavior in a data-driven test. m.Index = r.raftLog.lastIndex() m.LogTerm = r.raftLog.lastTerm() } diff --git a/rafttest/interaction_env_handler_deliver_msgs.go b/rafttest/interaction_env_handler_deliver_msgs.go index 8a4ff10d..671e0539 100644 --- a/rafttest/interaction_env_handler_deliver_msgs.go +++ b/rafttest/interaction_env_handler_deliver_msgs.go @@ -73,7 +73,7 @@ func (env *InteractionEnv) DeliverMsgs(rs ...Recipient) int { var n int for _, r := range rs { var msgs []raftpb.Message - msgs, env.Messages = splitMsgs(env.Messages, r.ID) + msgs, env.Messages = splitMsgs(env.Messages, r.ID, r.Drop) n += len(msgs) for _, msg := range msgs { if r.Drop { diff --git a/rafttest/interaction_env_handler_stabilize.go b/rafttest/interaction_env_handler_stabilize.go index 8fa6a8e8..d79a9aa4 100644 --- a/rafttest/interaction_env_handler_stabilize.go +++ b/rafttest/interaction_env_handler_stabilize.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/datadriven" + "go.etcd.io/raft/v3" "go.etcd.io/raft/v3/raftpb" ) @@ -60,7 +61,7 @@ func (env *InteractionEnv) Stabilize(idxs ...int) error { id := rn.Status().ID // NB: we grab the messages just to see whether to print the header. // DeliverMsgs will do it again. - if msgs, _ := splitMsgs(env.Messages, id); len(msgs) > 0 { + if msgs, _ := splitMsgs(env.Messages, id, false /* drop */); len(msgs) > 0 { fmt.Fprintf(env.Output, "> %d receiving messages\n", id) env.withIndent(func() { env.DeliverMsgs(Recipient{ID: id}) }) done = false @@ -96,10 +97,10 @@ func (env *InteractionEnv) Stabilize(idxs ...int) error { } } -func splitMsgs(msgs []raftpb.Message, to uint64) (toMsgs []raftpb.Message, rmdr []raftpb.Message) { +func splitMsgs(msgs []raftpb.Message, to uint64, drop bool) (toMsgs []raftpb.Message, rmdr []raftpb.Message) { // NB: this method does not reorder messages. for _, msg := range msgs { - if msg.To == to { + if msg.To == to && !(drop && isLocalMsg(msg)) { toMsgs = append(toMsgs, msg) } else { rmdr = append(rmdr, msg) @@ -107,3 +108,8 @@ func splitMsgs(msgs []raftpb.Message, to uint64) (toMsgs []raftpb.Message, rmdr } return toMsgs, rmdr } + +// Don't drop local messages, which require reliable delivery. +func isLocalMsg(msg raftpb.Message) bool { + return msg.From == msg.To || raft.IsLocalMsgTarget(msg.From) || raft.IsLocalMsgTarget(msg.To) +} diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt index 3831edcb..2e628bdf 100644 --- a/testdata/async_storage_writes.txt +++ b/testdata/async_storage_writes.txt @@ -704,8 +704,10 @@ deliver-msgs 1 2 3 3->1 MsgAppResp Term:1 Log:0/15 ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/14 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +INFO entry at index 15 missing from unstable log; ignoring ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 +INFO entry at index 15 missing from unstable log; ignoring ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14 process-ready 1 2 3 @@ -782,4 +784,5 @@ stabilize AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 > 1 receiving messages AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 + INFO entry at index 15 missing from unstable log; ignoring AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 diff --git a/testdata/async_storage_writes_append_aba_race.txt b/testdata/async_storage_writes_append_aba_race.txt new file mode 100644 index 00000000..17dedfd9 --- /dev/null +++ b/testdata/async_storage_writes_append_aba_race.txt @@ -0,0 +1,493 @@ +# This test reproduces a scenario similar to the one described in +# newStorageAppendRespMsg, exercising a few interesting interactions +# between asynchronous storage writes, term changes, and log truncation. + +log-level none +---- +ok + +add-nodes 7 voters=(1,2,3,4,5,6,7) index=10 async-storage-writes=true +---- +ok + +# Step 1: node 2 is the leader. + +campaign 2 +---- +ok + +stabilize +---- +ok (quiet) + +log-level info +---- +ok + +# Step 2: node 2 proposes some log entries but only node 1 receives these entries. + +propose 2 init_prop +---- +ok + +process-ready 2 +---- +Ready MustSync=true: +Entries: +1/12 EntryNormal "init_prop" +Messages: +2->1 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->4 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->5 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +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:1 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[2->2 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] + +deliver-msgs 1 drop=(3,4,5,6,7) +---- +2->1 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->4 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->5 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->6 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->7 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] + +# Step 3: node 1 gets the Ready and the entries are appended asynchronously. + +process-ready 1 +---- +Ready MustSync=true: +Entries: +1/12 EntryNormal "init_prop" +Messages: +1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[1->2 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] + +# Step 4: node 3 becomes the leader after getting a vote from nodes 4, 5, and 6. + +campaign 3 +---- +INFO 3 is starting a new election at term 1 +INFO 3 became candidate at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 1 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 2 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 4 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 5 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 6 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 7 at term 2 + +process-ready 3 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:2 Vote:3 Commit:11 +Messages: +3->1 MsgVote Term:2 Log:1/11 +3->2 MsgVote Term:2 Log:1/11 +3->4 MsgVote Term:2 Log:1/11 +3->5 MsgVote Term:2 Log:1/11 +3->6 MsgVote Term:2 Log:1/11 +3->7 MsgVote Term:2 Log:1/11 +3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[3->3 MsgVoteResp Term:2 Log:0/0, AppendThread->3 MsgStorageAppendResp Term:2 Log:0/0] + +deliver-msgs 4 5 6 +---- +3->4 MsgVote Term:2 Log:1/11 +INFO 4 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 4 became follower at term 2 +INFO 4 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 +3->5 MsgVote Term:2 Log:1/11 +INFO 5 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 5 became follower at term 2 +INFO 5 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 +3->6 MsgVote Term:2 Log:1/11 +INFO 6 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 6 became follower at term 2 +INFO 6 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 + +process-ready 4 5 6 +---- +> 4 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:2 Vote:3 Commit:11 + Messages: + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[4->3 MsgVoteResp Term:2 Log:0/0, AppendThread->4 MsgStorageAppendResp Term:2 Log:0/0] +> 5 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:2 Vote:3 Commit:11 + Messages: + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[5->3 MsgVoteResp Term:2 Log:0/0, AppendThread->5 MsgStorageAppendResp Term:2 Log:0/0] +> 6 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:2 Vote:3 Commit:11 + Messages: + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[6->3 MsgVoteResp Term:2 Log:0/0, AppendThread->6 MsgStorageAppendResp Term:2 Log:0/0] + +process-append-thread 3 4 5 6 +---- +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + Responses: + 3->3 MsgVoteResp Term:2 Log:0/0 + AppendThread->3 MsgStorageAppendResp Term:2 Log:0/0 +> 4 processing append thread + Processing: + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + Responses: + 4->3 MsgVoteResp Term:2 Log:0/0 + AppendThread->4 MsgStorageAppendResp Term:2 Log:0/0 +> 5 processing append thread + Processing: + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + Responses: + 5->3 MsgVoteResp Term:2 Log:0/0 + AppendThread->5 MsgStorageAppendResp Term:2 Log:0/0 +> 6 processing append thread + Processing: + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + Responses: + 6->3 MsgVoteResp Term:2 Log:0/0 + AppendThread->6 MsgStorageAppendResp Term:2 Log:0/0 + +deliver-msgs 3 +---- +3->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 3 at term 2 +INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections +AppendThread->3 MsgStorageAppendResp Term:2 Log:0/0 +4->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 4 at term 2 +INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections +5->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 5 at term 2 +INFO 3 has received 3 MsgVoteResp votes and 0 vote rejections +6->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 6 at term 2 +INFO 3 has received 4 MsgVoteResp votes and 0 vote rejections +INFO 3 became leader at term 2 + +# Step 5: node 3 proposes some log entries and node 1 receives these entries, +# overwriting the previous unstable log entries that are in the process of being +# appended. The entries have a larger term than the previous entries but the +# same indexes. It begins appending these new entries asynchronously. + +process-ready 3 +---- +Ready MustSync=true: +Lead:3 State:StateLeader +Entries: +2/12 EntryNormal "" +Messages: +3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->4 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +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:2 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[3->3 MsgAppResp Term:2 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12] + +deliver-msgs 1 drop=(2,4,5,6,7) +---- +3->1 MsgVote Term:2 Log:1/11 +INFO 1 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 1 became follower at term 2 +INFO 1 [logterm: 1, index: 12, vote: 0] rejected MsgVote from 3 [logterm: 1, index: 11] at term 2 +3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +INFO found conflict at index 12 [existing term: 1, conflicting term: 2] +INFO replace the unstable entries from index 12 +dropped: 3->2 MsgVote Term:2 Log:1/11 +dropped: 3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->4 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->6 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->7 MsgVote Term:2 Log:1/11 +dropped: 3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] + +process-ready 1 +---- +Ready MustSync=true: +Lead:3 State:StateFollower +HardState Term:2 Commit:11 +Entries: +2/12 EntryNormal "" +Messages: +1->AppendThread MsgStorageAppend Term:2 Log:0/0 Entries:[2/12 EntryNormal ""] HardState: Term:2 Commit:11 Responses:[1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0), 1->3 MsgAppResp Term:2 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12] + +# Step 6: node 3 crashes and node 4 becomes leader getting the vote from 5, 6, and 7. + +campaign 4 +---- +INFO 4 is starting a new election at term 2 +INFO 4 became candidate at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 1 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 2 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 3 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 5 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 6 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 7 at term 3 + +process-ready 4 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:3 Vote:4 Commit:11 +Messages: +4->1 MsgVote Term:3 Log:1/11 +4->2 MsgVote Term:3 Log:1/11 +4->3 MsgVote Term:3 Log:1/11 +4->5 MsgVote Term:3 Log:1/11 +4->6 MsgVote Term:3 Log:1/11 +4->7 MsgVote Term:3 Log:1/11 +4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[4->4 MsgVoteResp Term:3 Log:0/0, AppendThread->4 MsgStorageAppendResp Term:3 Log:0/0] + +deliver-msgs 5 6 7 +---- +AppendThread->5 MsgStorageAppendResp Term:2 Log:0/0 +4->5 MsgVote Term:3 Log:1/11 +INFO 5 [term: 2] received a MsgVote message with higher term from 4 [term: 3] +INFO 5 became follower at term 3 +INFO 5 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 +AppendThread->6 MsgStorageAppendResp Term:2 Log:0/0 +4->6 MsgVote Term:3 Log:1/11 +INFO 6 [term: 2] received a MsgVote message with higher term from 4 [term: 3] +INFO 6 became follower at term 3 +INFO 6 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 +4->7 MsgVote Term:3 Log:1/11 +INFO 7 [term: 1] received a MsgVote message with higher term from 4 [term: 3] +INFO 7 became follower at term 3 +INFO 7 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 + +process-ready 5 6 7 +---- +> 5 handling Ready + Ready MustSync=true: + HardState Term:3 Vote:4 Commit:11 + Messages: + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[5->4 MsgVoteResp Term:3 Log:0/0, AppendThread->5 MsgStorageAppendResp Term:3 Log:0/0] +> 6 handling Ready + Ready MustSync=true: + HardState Term:3 Vote:4 Commit:11 + Messages: + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[6->4 MsgVoteResp Term:3 Log:0/0, AppendThread->6 MsgStorageAppendResp Term:3 Log:0/0] +> 7 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:3 Vote:4 Commit:11 + Messages: + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[7->4 MsgVoteResp Term:3 Log:0/0, AppendThread->7 MsgStorageAppendResp Term:3 Log:0/0] + +process-append-thread 4 5 6 7 +---- +> 4 processing append thread + Processing: + 4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + Responses: + 4->4 MsgVoteResp Term:3 Log:0/0 + AppendThread->4 MsgStorageAppendResp Term:3 Log:0/0 +> 5 processing append thread + Processing: + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + Responses: + 5->4 MsgVoteResp Term:3 Log:0/0 + AppendThread->5 MsgStorageAppendResp Term:3 Log:0/0 +> 6 processing append thread + Processing: + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + Responses: + 6->4 MsgVoteResp Term:3 Log:0/0 + AppendThread->6 MsgStorageAppendResp Term:3 Log:0/0 +> 7 processing append thread + Processing: + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + Responses: + 7->4 MsgVoteResp Term:3 Log:0/0 + AppendThread->7 MsgStorageAppendResp Term:3 Log:0/0 + +deliver-msgs 4 +---- +AppendThread->4 MsgStorageAppendResp Term:2 Log:0/0 +4->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 4 at term 3 +INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections +AppendThread->4 MsgStorageAppendResp Term:3 Log:0/0 +5->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 5 at term 3 +INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections +6->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 6 at term 3 +INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections +7->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 7 at term 3 +INFO 4 has received 4 MsgVoteResp votes and 0 vote rejections +INFO 4 became leader at term 3 + +process-ready 4 +---- +Ready MustSync=true: +Lead:4 State:StateLeader +Entries: +3/12 EntryNormal "" +Messages: +4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->2 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->3 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->5 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +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:3 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[4->4 MsgAppResp Term:3 Log:0/12, AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12] + +# Step 7: before the new entries reach node 1, it hears of the term change +# through a heartbeat and persists the new term. Node 1 then receives these +# entries, overwriting the previous unstable log entries that are in the process +# of being appended. The entries have a larger term than the previous entries +# but the same indexes. It begins appending these new entries asynchronously. + +deliver-msgs drop=1 +---- +dropped: 4->1 MsgVote Term:3 Log:1/11 +dropped: 4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] + +tick-heartbeat 4 +---- +ok + +process-ready 4 +---- +Ready MustSync=false: +Messages: +4->1 MsgHeartbeat Term:3 Log:0/0 +4->2 MsgHeartbeat Term:3 Log:0/0 +4->3 MsgHeartbeat Term:3 Log:0/0 +4->5 MsgHeartbeat Term:3 Log:0/0 +4->6 MsgHeartbeat Term:3 Log:0/0 +4->7 MsgHeartbeat Term:3 Log:0/0 + +deliver-msgs 1 +---- +4->1 MsgHeartbeat Term:3 Log:0/0 +INFO 1 [term: 2] received a MsgHeartbeat message with higher term from 4 [term: 3] +INFO 1 became follower at term 3 + +process-ready 1 +---- +Ready MustSync=true: +Lead:4 State:StateFollower +HardState Term:3 Commit:11 +Messages: +1->4 MsgHeartbeatResp Term:3 Log:0/0 +1->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Commit:11 Responses:[AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12] + +deliver-msgs 4 +---- +1->4 MsgHeartbeatResp Term:3 Log:0/0 + +process-ready 4 +---- +Ready MustSync=false: +Messages: +4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] + +deliver-msgs 1 +---- +4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +INFO found conflict at index 12 [existing term: 2, conflicting term: 3] +INFO replace the unstable entries from index 12 + +process-ready 1 +---- +Ready MustSync=true: +Entries: +3/12 EntryNormal "" +Messages: +1->AppendThread MsgStorageAppend Term:3 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[1->4 MsgAppResp Term:3 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12] + +# Step 8: The asynchronous log appends from the first Ready complete and the +# MsgStorageAppendResp is returned to the raft node state machine. A decision +# is made about whether to truncate the unstable log. + +raft-log 1 +---- +1/11 EntryNormal "" + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] +Responses: +1->2 MsgAppResp Term:1 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 + +raft-log 1 +---- +1/11 EntryNormal "" +1/12 EntryNormal "init_prop" + +# Step 9: However, the log entries from the second Ready are still in the +# asynchronous append pipeline and will overwrite (in stable storage) the +# entries from the first Ready at some future point. We can't truncate the +# unstable log yet or a future read from Storage might see the entries from step +# 5 before they have been replaced by the entries from step 7. Instead, we must +# wait until we are sure that the entries are stable and that no in-progress +# appends might overwrite them before removing entries from the unstable log. + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 +INFO 1 [term: 3] ignored entry appends from a MsgStorageAppendResp message with lower term [term: 1] + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:2 Log:0/0 Entries:[2/12 EntryNormal ""] HardState: Term:2 Commit:11 +Responses: +1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0) +1->3 MsgAppResp Term:2 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12 + +raft-log 1 +---- +1/11 EntryNormal "" +2/12 EntryNormal "" + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12 +INFO 1 [term: 3] ignored entry appends from a MsgStorageAppendResp message with lower term [term: 2] + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Commit:11 +Responses: +AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12 + +raft-log 1 +---- +1/11 EntryNormal "" +2/12 EntryNormal "" + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12 +INFO entry at (index,term)=(12,2) mismatched with entry at (12,3) in unstable log; ignoring + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:3 Log:0/0 Entries:[3/12 EntryNormal ""] +Responses: +1->4 MsgAppResp Term:3 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 + +raft-log 1 +---- +1/11 EntryNormal "" +3/12 EntryNormal "" + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 From 3cbcd6e7088c773cab6331459162561682e0d1df Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 2 Nov 2022 17:12:59 -0400 Subject: [PATCH 09/13] raft: move Ready constructor to rawnode.go Pure code movement. Eliminates asyncStorageWrites handling in node.go. Signed-off-by: Nathan VanBenschoten --- node.go | 239 ------------------------------------------------- rawnode.go | 256 ++++++++++++++++++++++++++++++++++++++++++++++++++++- 2 files changed, 254 insertions(+), 241 deletions(-) diff --git a/node.go b/node.go index 33940013..9c53aed2 100644 --- a/node.go +++ b/node.go @@ -578,242 +578,3 @@ func (n *node) TransferLeadership(ctx context.Context, lead, transferee uint64) func (n *node) ReadIndex(ctx context.Context, rctx []byte) error { return n.step(ctx, pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}}) } - -// TODO(nvanbenschoten): move this function and the functions below it to -// rawnode.go. -func newReady(r *raft, asyncStorageWrites bool, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { - rd := Ready{ - Entries: r.raftLog.nextUnstableEnts(), - CommittedEntries: r.raftLog.nextCommittedEnts(!asyncStorageWrites), - Messages: r.msgs, - } - if softSt := r.softState(); !softSt.equal(prevSoftSt) { - rd.SoftState = softSt - } - if hardSt := r.hardState(); !isHardStateEqual(hardSt, prevHardSt) { - rd.HardState = hardSt - } - if r.raftLog.hasNextUnstableSnapshot() { - rd.Snapshot = *r.raftLog.nextUnstableSnapshot() - } - if len(r.readStates) != 0 { - rd.ReadStates = r.readStates - } - rd.MustSync = MustSync(r.hardState(), prevHardSt, len(rd.Entries)) - - if asyncStorageWrites { - // If async storage writes are enabled, enqueue messages to - // local storage threads, where applicable. - if needStorageAppend(rd, len(r.msgsAfterAppend) > 0) { - m := newStorageAppendMsg(r, rd) - rd.Messages = append(rd.Messages, m) - } - if needStorageApply(rd) { - m := newStorageApplyMsg(r, rd) - rd.Messages = append(rd.Messages, m) - } - } else { - // If async storage writes are disabled, immediately enqueue - // msgsAfterAppend to be sent out. The Ready struct contract - // mandates that Messages cannot be sent until after Entries - // are written to stable storage. - for _, m := range r.msgsAfterAppend { - if m.To != r.id { - rd.Messages = append(rd.Messages, m) - } - } - } - - return rd -} - -// MustSync returns true if the hard state and count of Raft entries indicate -// that a synchronous write to persistent storage is required. -func MustSync(st, prevst pb.HardState, entsnum int) bool { - // Persistent state on all servers: - // (Updated on stable storage before responding to RPCs) - // currentTerm - // votedFor - // log entries[] - return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term -} - -func needStorageAppend(rd Ready, haveMsgsAfterAppend bool) bool { - // Return true if log entries, hard state, or a snapshot need to be written - // to stable storage. Also return true if any messages are contingent on all - // prior MsgStorageAppend being processed. - return len(rd.Entries) > 0 || - !IsEmptyHardState(rd.HardState) || - !IsEmptySnap(rd.Snapshot) || - haveMsgsAfterAppend -} - -// newStorageAppendMsg creates the message that should be sent to the local -// append thread to instruct it to append log entries, write an updated hard -// state, and apply a snapshot. The message also carries a set of responses -// that should be delivered after the rest of the message is processed. Used -// with AsyncStorageWrites. -func newStorageAppendMsg(r *raft, rd Ready) pb.Message { - m := pb.Message{ - Type: pb.MsgStorageAppend, - To: LocalAppendThread, - From: r.id, - Term: r.Term, - Entries: rd.Entries, - } - if !IsEmptyHardState(rd.HardState) { - hs := rd.HardState - m.HardState = &hs - } - if !IsEmptySnap(rd.Snapshot) { - snap := rd.Snapshot - m.Snapshot = &snap - } - // Attach all messages in msgsAfterAppend as responses to be delivered after - // the message is processed, along with a self-directed MsgStorageAppendResp - // to acknowledge the entry stability. - // - // NB: it is important for performance that MsgStorageAppendResp message be - // handled after self-directed MsgAppResp messages on the leader (which will - // be contained in msgsAfterAppend). This ordering allows the MsgAppResp - // handling to use a fast-path in r.raftLog.term() before the newly appended - // entries are removed from the unstable log. - m.Responses = r.msgsAfterAppend - m.Responses = append(m.Responses, newStorageAppendRespMsg(r, rd)) - return m -} - -// newStorageAppendRespMsg creates the message that should be returned to node -// after the unstable log entries, hard state, and snapshot in the current Ready -// (along with those in all prior Ready structs) have been saved to stable -// storage. -func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { - m := pb.Message{ - Type: pb.MsgStorageAppendResp, - To: r.id, - From: LocalAppendThread, - // Dropped after term change, see below. - Term: r.Term, - } - if r.raftLog.hasNextOrInProgressUnstableEnts() { - // If the raft log has unstable entries, attach the last index and term to the - // response message. This (index, term) tuple will be handed back and consulted - // when the stability of those log entries is signaled to the unstable. If the - // (index, term) match the unstable log by the time the response is received, - // the unstable log can be truncated. - // - // However, with just this logic, there would be an ABA problem that could lead - // to the unstable log and the stable log getting out of sync temporarily and - // leading to an inconsistent view. Consider the following example with 5 nodes, - // A B C D E: - // - // 1. A is the leader. - // 2. A proposes some log entries but only B receives these entries. - // 3. B gets the Ready and the entries are appended asynchronously. - // 4. A crashes and C becomes leader after getting a vote from D and E. - // 5. C proposes some log entries and B receives these entries, overwriting the - // previous unstable log entries that are in the process of being appended. - // The entries have a larger term than the previous entries but the same - // indexes. It begins appending these new entries asynchronously. - // 6. C crashes and A restarts and becomes leader again after getting the vote - // from D and E. - // 7. B receives the entries from A which are the same as the ones from step 2, - // overwriting the previous unstable log entries that are in the process of - // being appended from step 5. The entries have the original terms and - // indexes from step 2. Recall that log entries retain their original term - // numbers when a leader replicates entries from previous terms. It begins - // appending these new entries asynchronously. - // 8. The asynchronous log appends from the first Ready complete and stableTo - // is called. - // 9. However, the log entries from the second Ready are still in the - // asynchronous append pipeline and will overwrite (in stable storage) the - // entries from the first Ready at some future point. We can't truncate the - // unstable log yet or a future read from Storage might see the entries from - // step 5 before they have been replaced by the entries from step 7. - // Instead, we must wait until we are sure that the entries are stable and - // that no in-progress appends might overwrite them before removing entries - // from the unstable log. - // - // To prevent these kinds of problems, we also attach the current term to the - // MsgStorageAppendResp (above). If the term has changed by the time the - // MsgStorageAppendResp if returned, the response is ignored and the unstable - // log is not truncated. The unstable log is only truncated when the term has - // remained unchanged from the time that the MsgStorageAppend was sent to the - // time that the MsgStorageAppendResp is received, indicating that no-one else - // is in the process of truncating the stable log. - // - // However, this replaces a correctness problem with a liveness problem. If we - // only attempted to truncate the unstable log when appending new entries but - // also occasionally dropped these responses, then quiescence of new log entries - // could lead to the unstable log never being truncated. - // - // To combat this, we attempt to truncate the log on all MsgStorageAppendResp - // messages where the unstable log is not empty, not just those associated with - // entry appends. This includes MsgStorageAppendResp messages associated with an - // updated HardState, which occur after a term change. - // - // In other words, we set Index and LogTerm in a block that looks like: - // - // if r.raftLog.hasNextOrInProgressUnstableEnts() { ... } - // - // not like: - // - // if len(rd.Entries) > 0 { ... } - // - // To do so, we attach r.raftLog.lastIndex() and r.raftLog.lastTerm(), not the - // (index, term) of the last entry in rd.Entries. If rd.Entries is not empty, - // these will be the same. However, if rd.Entries is empty, we still want to - // attest that this (index, term) is correct at the current term, in case the - // MsgStorageAppend that contained the last entry in the unstable slice carried - // an earlier term and was dropped. - m.Index = r.raftLog.lastIndex() - m.LogTerm = r.raftLog.lastTerm() - } - if !IsEmptySnap(rd.Snapshot) { - snap := rd.Snapshot - m.Snapshot = &snap - } - return m -} - -func needStorageApply(rd Ready) bool { - return len(rd.CommittedEntries) > 0 -} - -// newStorageApplyMsg creates the message that should be sent to the local -// apply thread to instruct it to apply committed log entries. The message -// also carries a response that should be delivered after the rest of the -// message is processed. Used with AsyncStorageWrites. -func newStorageApplyMsg(r *raft, rd Ready) pb.Message { - ents := rd.CommittedEntries - last := ents[len(ents)-1].Index - return pb.Message{ - Type: pb.MsgStorageApply, - To: LocalApplyThread, - From: r.id, - Term: 0, // committed entries don't apply under a specific term - Entries: ents, - Index: last, - Responses: []pb.Message{ - newStorageApplyRespMsg(r, ents), - }, - } -} - -// newStorageApplyRespMsg creates the message that should be returned to node -// after the committed entries in the current Ready (along with those in all -// prior Ready structs) have been applied to the local state machine. -func newStorageApplyRespMsg(r *raft, committedEnts []pb.Entry) pb.Message { - last := committedEnts[len(committedEnts)-1].Index - size := r.getUncommittedSize(committedEnts) - return pb.Message{ - Type: pb.MsgStorageApplyResp, - To: r.id, - From: LocalApplyThread, - Term: 0, // committed entries don't apply under a specific term - Index: last, - // NOTE: we abuse the LogTerm field to store the aggregate entry size so - // that we don't need to introduce a new field on Message. - LogTerm: size, - } -} diff --git a/rawnode.go b/rawnode.go index 11cb877b..9f15adda 100644 --- a/rawnode.go +++ b/rawnode.go @@ -136,7 +136,259 @@ func (rn *RawNode) Ready() Ready { // readyWithoutAccept returns a Ready. This is a read-only operation, i.e. there // is no obligation that the Ready must be handled. func (rn *RawNode) readyWithoutAccept() Ready { - return newReady(rn.raft, rn.asyncStorageWrites, rn.prevSoftSt, rn.prevHardSt) + r := rn.raft + + rd := Ready{ + Entries: r.raftLog.nextUnstableEnts(), + CommittedEntries: r.raftLog.nextCommittedEnts(!rn.asyncStorageWrites), + Messages: r.msgs, + } + if softSt := r.softState(); !softSt.equal(rn.prevSoftSt) { + rd.SoftState = softSt + } + if hardSt := r.hardState(); !isHardStateEqual(hardSt, rn.prevHardSt) { + rd.HardState = hardSt + } + if r.raftLog.hasNextUnstableSnapshot() { + rd.Snapshot = *r.raftLog.nextUnstableSnapshot() + } + if len(r.readStates) != 0 { + rd.ReadStates = r.readStates + } + rd.MustSync = MustSync(r.hardState(), rn.prevHardSt, len(rd.Entries)) + + if rn.asyncStorageWrites { + // If async storage writes are enabled, enqueue messages to + // local storage threads, where applicable. + if needStorageAppend(rd, len(r.msgsAfterAppend) > 0) { + m := newStorageAppendMsg(r, rd) + rd.Messages = append(rd.Messages, m) + } + if needStorageApply(rd) { + m := newStorageApplyMsg(r, rd) + rd.Messages = append(rd.Messages, m) + } + } else { + // If async storage writes are disabled, immediately enqueue + // msgsAfterAppend to be sent out. The Ready struct contract + // mandates that Messages cannot be sent until after Entries + // are written to stable storage. + for _, m := range r.msgsAfterAppend { + if m.To != r.id { + rd.Messages = append(rd.Messages, m) + } + } + } + + return rd +} + +// MustSync returns true if the hard state and count of Raft entries indicate +// that a synchronous write to persistent storage is required. +func MustSync(st, prevst pb.HardState, entsnum int) bool { + // Persistent state on all servers: + // (Updated on stable storage before responding to RPCs) + // currentTerm + // votedFor + // log entries[] + return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term +} + +func needStorageAppend(rd Ready, haveMsgsAfterAppend bool) bool { + // Return true if log entries, hard state, or a snapshot need to be written + // to stable storage. Also return true if any messages are contingent on all + // prior MsgStorageAppend being processed. + return len(rd.Entries) > 0 || + !IsEmptyHardState(rd.HardState) || + !IsEmptySnap(rd.Snapshot) || + haveMsgsAfterAppend +} + +// newStorageAppendMsg creates the message that should be sent to the local +// append thread to instruct it to append log entries, write an updated hard +// state, and apply a snapshot. The message also carries a set of responses +// that should be delivered after the rest of the message is processed. Used +// with AsyncStorageWrites. +func newStorageAppendMsg(r *raft, rd Ready) pb.Message { + m := pb.Message{ + Type: pb.MsgStorageAppend, + To: LocalAppendThread, + From: r.id, + Term: r.Term, + Entries: rd.Entries, + } + if !IsEmptyHardState(rd.HardState) { + // TODO(nvanbenschoten): we could avoid this heap allocation by + // replacing the pb.Message.HardState field with a Vote uint64 field. We + // would then need to teach apps to construct a HardState from these + // three fields, or supply a function/method that does so. + // m.Term = rd.Term + // m.Vote = rd.Vote + // m.Commit = rd.Commit + hs := rd.HardState + m.HardState = &hs + } + if !IsEmptySnap(rd.Snapshot) { + snap := rd.Snapshot + m.Snapshot = &snap + } + // Attach all messages in msgsAfterAppend as responses to be delivered after + // the message is processed, along with a self-directed MsgStorageAppendResp + // to acknowledge the entry stability. + // + // NB: it is important for performance that MsgStorageAppendResp message be + // handled after self-directed MsgAppResp messages on the leader (which will + // be contained in msgsAfterAppend). This ordering allows the MsgAppResp + // handling to use a fast-path in r.raftLog.term() before the newly appended + // entries are removed from the unstable log. + m.Responses = r.msgsAfterAppend + m.Responses = append(m.Responses, newStorageAppendRespMsg(r, rd)) + return m +} + +// newStorageAppendRespMsg creates the message that should be returned to node +// after the unstable log entries, hard state, and snapshot in the current Ready +// (along with those in all prior Ready structs) have been saved to stable +// storage. +func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { + m := pb.Message{ + Type: pb.MsgStorageAppendResp, + To: r.id, + From: LocalAppendThread, + // Dropped after term change, see below. + Term: r.Term, + } + if r.raftLog.hasNextOrInProgressUnstableEnts() { + // If the raft log has unstable entries, attach the last index and term of the + // append to the response message. This (index, term) tuple will be handed back + // and consulted when the stability of those log entries is signaled to the + // unstable. If the (index, term) match the unstable log by the time the + // response is received (unstable.stableTo), the unstable log can be truncated. + // + // However, with just this logic, there would be an ABA problem[^1] that could + // lead to the unstable log and the stable log getting out of sync temporarily + // and leading to an inconsistent view. Consider the following example with 5 + // nodes, A B C D E: + // + // 1. A is the leader. + // 2. A proposes some log entries but only B receives these entries. + // 3. B gets the Ready and the entries are appended asynchronously. + // 4. A crashes and C becomes leader after getting a vote from D and E. + // 5. C proposes some log entries and B receives these entries, overwriting the + // previous unstable log entries that are in the process of being appended. + // The entries have a larger term than the previous entries but the same + // indexes. It begins appending these new entries asynchronously. + // 6. C crashes and A restarts and becomes leader again after getting the vote + // from D and E. + // 7. B receives the entries from A which are the same as the ones from step 2, + // overwriting the previous unstable log entries that are in the process of + // being appended from step 5. The entries have the original terms and + // indexes from step 2. Recall that log entries retain their original term + // numbers when a leader replicates entries from previous terms. It begins + // appending these new entries asynchronously. + // 8. The asynchronous log appends from the first Ready complete and stableTo + // is called. + // 9. However, the log entries from the second Ready are still in the + // asynchronous append pipeline and will overwrite (in stable storage) the + // entries from the first Ready at some future point. We can't truncate the + // unstable log yet or a future read from Storage might see the entries from + // step 5 before they have been replaced by the entries from step 7. + // Instead, we must wait until we are sure that the entries are stable and + // that no in-progress appends might overwrite them before removing entries + // from the unstable log. + // + // To prevent these kinds of problems, we also attach the current term to the + // MsgStorageAppendResp (above). If the term has changed by the time the + // MsgStorageAppendResp if returned, the response is ignored and the unstable + // log is not truncated. The unstable log is only truncated when the term has + // remained unchanged from the time that the MsgStorageAppend was sent to the + // time that the MsgStorageAppendResp is received, indicating that no-one else + // is in the process of truncating the stable log. + // + // However, this replaces a correctness problem with a liveness problem. If we + // only attempted to truncate the unstable log when appending new entries but + // also occasionally dropped these responses, then quiescence of new log entries + // could lead to the unstable log never being truncated. + // + // To combat this, we attempt to truncate the log on all MsgStorageAppendResp + // messages where the unstable log is not empty, not just those associated with + // entry appends. This includes MsgStorageAppendResp messages associated with an + // updated HardState, which occur after a term change. + // + // In other words, we set Index and LogTerm in a block that looks like: + // + // if r.raftLog.hasNextOrInProgressUnstableEnts() { ... } + // + // not like: + // + // if len(rd.Entries) > 0 { ... } + // + // To do so, we attach r.raftLog.lastIndex() and r.raftLog.lastTerm(), not the + // (index, term) of the last entry in rd.Entries. If rd.Entries is not empty, + // these will be the same. However, if rd.Entries is empty, we still want to + // attest that this (index, term) is correct at the current term, in case the + // MsgStorageAppend that contained the last entry in the unstable slice carried + // an earlier term and was dropped. + // + // A MsgStorageAppend with a new HardState is emitted on each term change. This + // is the same condition that causes MsgStorageAppendResp messages with earlier + // terms to be ignored. As a result, we are guaranteed that, assuming a bounded + // number of term changes, there will eventually be a MsgStorageAppendResp + // message that is not ignored. This means that entries in the unstable log + // which have been appended to stable storage will eventually be truncated and + // dropped from memory. + // + // [^1]: https://en.wikipedia.org/wiki/ABA_problem + m.Index = r.raftLog.lastIndex() + m.LogTerm = r.raftLog.lastTerm() + } + if !IsEmptySnap(rd.Snapshot) { + snap := rd.Snapshot + m.Snapshot = &snap + } + return m +} + +func needStorageApply(rd Ready) bool { + return len(rd.CommittedEntries) > 0 +} + +// newStorageApplyMsg creates the message that should be sent to the local +// apply thread to instruct it to apply committed log entries. The message +// also carries a response that should be delivered after the rest of the +// message is processed. Used with AsyncStorageWrites. +func newStorageApplyMsg(r *raft, rd Ready) pb.Message { + ents := rd.CommittedEntries + last := ents[len(ents)-1].Index + return pb.Message{ + Type: pb.MsgStorageApply, + To: LocalApplyThread, + From: r.id, + Term: 0, // committed entries don't apply under a specific term + Entries: ents, + Index: last, + Responses: []pb.Message{ + newStorageApplyRespMsg(r, ents), + }, + } +} + +// newStorageApplyRespMsg creates the message that should be returned to node +// after the committed entries in the current Ready (along with those in all +// prior Ready structs) have been applied to the local state machine. +func newStorageApplyRespMsg(r *raft, committedEnts []pb.Entry) pb.Message { + last := committedEnts[len(committedEnts)-1].Index + size := r.getUncommittedSize(committedEnts) + return pb.Message{ + Type: pb.MsgStorageApplyResp, + To: r.id, + From: LocalApplyThread, + Term: 0, // committed entries don't apply under a specific term + Index: last, + // NOTE: we abuse the LogTerm field to store the aggregate entry size so + // that we don't need to introduce a new field on Message. + LogTerm: size, + } } // acceptReady is called when the consumer of the RawNode has decided to go @@ -212,7 +464,7 @@ func (rn *RawNode) HasReady() bool { func (rn *RawNode) Advance(_ Ready) { // The actions performed by this function are encoded into stepsOnAdvance in // acceptReady. In earlier versions of this library, they were computed from - // the provided Ready struct. Retain the unused parameter for compatability. + // the provided Ready struct. Retain the unused parameter for compatibility. if rn.asyncStorageWrites { rn.raft.logger.Panicf("Advance must not be called when using AsyncStorageWrites") } From e96a974d5eea92ec32fb6d52663f350c946338ca Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 17 Nov 2022 17:28:29 -0500 Subject: [PATCH 10/13] raft: don't create unnecessary MsgStorageAppendResp messages This commit removes certain cases where `MsgStorageAppendResp` messages were attached as responses to a `MsgStorageAppend` message, even when the response contained no useful information. The most common case where this comes up is when the HardState changes but no new entries are appended to the log. Avoiding the response in these cases eliminates useless work. Additionally, if the HardState does not include a new vote and only includes a new Commit then there will be no response messages on the on `MsgStorageAppend`. Users of this library can use this condition to determine when an fsync is not necessary, similar to how it used to use the `Ready.MustSync` flag. Signed-off-by: Nathan VanBenschoten --- rawnode.go | 29 +++++++++----- testdata/async_storage_writes.txt | 38 +++++-------------- .../async_storage_writes_append_aba_race.txt | 29 ++++---------- testdata/replicate_pause.txt | 2 +- 4 files changed, 37 insertions(+), 61 deletions(-) diff --git a/rawnode.go b/rawnode.go index 9f15adda..675fa5bc 100644 --- a/rawnode.go +++ b/rawnode.go @@ -160,11 +160,11 @@ func (rn *RawNode) readyWithoutAccept() Ready { if rn.asyncStorageWrites { // If async storage writes are enabled, enqueue messages to // local storage threads, where applicable. - if needStorageAppend(rd, len(r.msgsAfterAppend) > 0) { + if needStorageAppendMsg(r, rd) { m := newStorageAppendMsg(r, rd) rd.Messages = append(rd.Messages, m) } - if needStorageApply(rd) { + if needStorageApplyMsg(rd) { m := newStorageApplyMsg(r, rd) rd.Messages = append(rd.Messages, m) } @@ -194,14 +194,22 @@ func MustSync(st, prevst pb.HardState, entsnum int) bool { return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term } -func needStorageAppend(rd Ready, haveMsgsAfterAppend bool) bool { +func needStorageAppendMsg(r *raft, rd Ready) bool { // Return true if log entries, hard state, or a snapshot need to be written // to stable storage. Also return true if any messages are contingent on all // prior MsgStorageAppend being processed. return len(rd.Entries) > 0 || !IsEmptyHardState(rd.HardState) || !IsEmptySnap(rd.Snapshot) || - haveMsgsAfterAppend + len(r.msgsAfterAppend) > 0 +} + +func needStorageAppendRespMsg(r *raft, rd Ready) bool { + // Return true if raft needs to hear about stabilized entries or an applied + // snapshot. See the comment in newStorageAppendRespMsg, which explains why + // we check hasNextOrInProgressUnstableEnts instead of len(rd.Entries) > 0. + return r.raftLog.hasNextOrInProgressUnstableEnts() || + !IsEmptySnap(rd.Snapshot) } // newStorageAppendMsg creates the message that should be sent to the local @@ -242,7 +250,9 @@ func newStorageAppendMsg(r *raft, rd Ready) pb.Message { // handling to use a fast-path in r.raftLog.term() before the newly appended // entries are removed from the unstable log. m.Responses = r.msgsAfterAppend - m.Responses = append(m.Responses, newStorageAppendRespMsg(r, rd)) + if needStorageAppendRespMsg(r, rd) { + m.Responses = append(m.Responses, newStorageAppendRespMsg(r, rd)) + } return m } @@ -349,9 +359,8 @@ func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { return m } -func needStorageApply(rd Ready) bool { - return len(rd.CommittedEntries) > 0 -} +func needStorageApplyMsg(rd Ready) bool { return len(rd.CommittedEntries) > 0 } +func needStorageApplyRespMsg(rd Ready) bool { return needStorageApplyMsg(rd) } // newStorageApplyMsg creates the message that should be sent to the local // apply thread to instruct it to apply committed log entries. The message @@ -413,11 +422,11 @@ func (rn *RawNode) acceptReady(rd Ready) { rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) } } - if needStorageAppend(rd, false /* haveMsgsAfterAppend */) { + if needStorageAppendRespMsg(rn.raft, rd) { m := newStorageAppendRespMsg(rn.raft, rd) rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) } - if needStorageApply(rd) { + if needStorageApplyRespMsg(rd) { m := newStorageApplyRespMsg(rn.raft, rd.CommittedEntries) rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) } diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt index 2e628bdf..bd48ec6e 100644 --- a/testdata/async_storage_writes.txt +++ b/testdata/async_storage_writes.txt @@ -29,7 +29,7 @@ stabilize Messages: 1->2 MsgVote Term:1 Log:1/10 1->3 MsgVote Term:1 Log:1/10 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[1->1 MsgVoteResp Term:1 Log:0/0, AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[1->1 MsgVoteResp Term:1 Log:0/0] > 2 receiving messages 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] @@ -45,44 +45,36 @@ stabilize 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses: 1->1 MsgVoteResp Term:1 Log:0/0 - AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 > 2 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:10 Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[2->1 MsgVoteResp Term:1 Log:0/0, AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[2->1 MsgVoteResp Term:1 Log:0/0] > 3 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:10 Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[3->1 MsgVoteResp Term:1 Log:0/0, AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[3->1 MsgVoteResp Term:1 Log:0/0] > 1 receiving messages 1->1 MsgVoteResp Term:1 Log:0/0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections - AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses: 2->1 MsgVoteResp Term:1 Log:0/0 - AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses: 3->1 MsgVoteResp Term:1 Log:0/0 - AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 > 1 receiving messages 2->1 MsgVoteResp Term:1 Log:0/0 INFO 1 received MsgVoteResp from 2 at term 1 INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 3->1 MsgVoteResp Term:1 Log:0/0 -> 2 receiving messages - AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 -> 3 receiving messages - AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader @@ -146,7 +138,7 @@ stabilize Messages: 1->2 MsgApp Term:1 Log:1/11 Commit:11 1->3 MsgApp Term:1 Log:1/11 Commit:11 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 1->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11] > 2 receiving messages 1->2 MsgApp Term:1 Log:1/11 Commit:11 @@ -156,7 +148,6 @@ stabilize Processing: 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses: - AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 > 1 processing apply thread Processing: 1->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] @@ -168,7 +159,7 @@ stabilize CommittedEntries: 1/11 EntryNormal "" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[2->1 MsgAppResp Term:1 Log:0/11, AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[2->1 MsgAppResp Term:1 Log:0/11] 2->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11] > 3 handling Ready Ready MustSync=false: @@ -176,23 +167,20 @@ stabilize CommittedEntries: 1/11 EntryNormal "" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[3->1 MsgAppResp Term:1 Log:0/11, AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[3->1 MsgAppResp Term:1 Log:0/11] 3->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11] > 1 receiving messages - AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11 > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses: 2->1 MsgAppResp Term:1 Log:0/11 - AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses: 3->1 MsgAppResp Term:1 Log:0/11 - AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 > 2 processing apply thread Processing: 2->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] @@ -207,10 +195,8 @@ stabilize 2->1 MsgAppResp Term:1 Log:0/11 3->1 MsgAppResp Term:1 Log:0/11 > 2 receiving messages - AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11 > 3 receiving messages - AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11 propose 1 prop_1 @@ -632,7 +618,7 @@ process-ready 1 2 3 Messages: 1->2 MsgApp Term:1 Log:1/15 Commit:15 1->3 MsgApp Term:1 Log:1/15 Commit:15 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 1->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15] > 2 handling Ready @@ -654,7 +640,7 @@ process-ready 1 2 3 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[2->1 MsgAppResp Term:1 Log:0/15] 2->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15] > 3 handling Ready Ready MustSync=false: @@ -662,7 +648,7 @@ process-ready 1 2 3 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[3->1 MsgAppResp Term:1 Log:0/15] 3->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/15] process-append-thread 2 3 @@ -726,13 +712,11 @@ process-append-thread 2 3 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses: 2->1 MsgAppResp Term:1 Log:0/15 - AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 > 3 processing append thread Processing: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses: 3->1 MsgAppResp Term:1 Log:0/15 - AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 process-apply-thread 1 2 3 ---- @@ -757,9 +741,7 @@ deliver-msgs 1 2 3 2->1 MsgAppResp Term:1 Log:0/15 3->1 MsgAppResp Term:1 Log:0/15 ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15 -AppendThread->2 MsgStorageAppendResp Term:1 Log:0/0 ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15 -AppendThread->3 MsgStorageAppendResp Term:1 Log:0/0 ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/15 process-ready 1 2 3 @@ -781,8 +763,6 @@ stabilize Processing: 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses: - AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 > 1 receiving messages AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 INFO entry at index 15 missing from unstable log; ignoring - AppendThread->1 MsgStorageAppendResp Term:1 Log:0/0 diff --git a/testdata/async_storage_writes_append_aba_race.txt b/testdata/async_storage_writes_append_aba_race.txt index 17dedfd9..6593a46e 100644 --- a/testdata/async_storage_writes_append_aba_race.txt +++ b/testdata/async_storage_writes_append_aba_race.txt @@ -88,7 +88,7 @@ Messages: 3->5 MsgVote Term:2 Log:1/11 3->6 MsgVote Term:2 Log:1/11 3->7 MsgVote Term:2 Log:1/11 -3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[3->3 MsgVoteResp Term:2 Log:0/0, AppendThread->3 MsgStorageAppendResp Term:2 Log:0/0] +3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[3->3 MsgVoteResp Term:2 Log:0/0] deliver-msgs 4 5 6 ---- @@ -112,19 +112,19 @@ process-ready 4 5 6 Lead:0 State:StateFollower HardState Term:2 Vote:3 Commit:11 Messages: - 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[4->3 MsgVoteResp Term:2 Log:0/0, AppendThread->4 MsgStorageAppendResp Term:2 Log:0/0] + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[4->3 MsgVoteResp Term:2 Log:0/0] > 5 handling Ready Ready MustSync=true: Lead:0 State:StateFollower HardState Term:2 Vote:3 Commit:11 Messages: - 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[5->3 MsgVoteResp Term:2 Log:0/0, AppendThread->5 MsgStorageAppendResp Term:2 Log:0/0] + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[5->3 MsgVoteResp Term:2 Log:0/0] > 6 handling Ready Ready MustSync=true: Lead:0 State:StateFollower HardState Term:2 Vote:3 Commit:11 Messages: - 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[6->3 MsgVoteResp Term:2 Log:0/0, AppendThread->6 MsgStorageAppendResp Term:2 Log:0/0] + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[6->3 MsgVoteResp Term:2 Log:0/0] process-append-thread 3 4 5 6 ---- @@ -133,32 +133,27 @@ process-append-thread 3 4 5 6 3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses: 3->3 MsgVoteResp Term:2 Log:0/0 - AppendThread->3 MsgStorageAppendResp Term:2 Log:0/0 > 4 processing append thread Processing: 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses: 4->3 MsgVoteResp Term:2 Log:0/0 - AppendThread->4 MsgStorageAppendResp Term:2 Log:0/0 > 5 processing append thread Processing: 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses: 5->3 MsgVoteResp Term:2 Log:0/0 - AppendThread->5 MsgStorageAppendResp Term:2 Log:0/0 > 6 processing append thread Processing: 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses: 6->3 MsgVoteResp Term:2 Log:0/0 - AppendThread->6 MsgStorageAppendResp Term:2 Log:0/0 deliver-msgs 3 ---- 3->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 3 at term 2 INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections -AppendThread->3 MsgStorageAppendResp Term:2 Log:0/0 4->3 MsgVoteResp Term:2 Log:0/0 INFO 3 received MsgVoteResp from 4 at term 2 INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections @@ -242,16 +237,14 @@ Messages: 4->5 MsgVote Term:3 Log:1/11 4->6 MsgVote Term:3 Log:1/11 4->7 MsgVote Term:3 Log:1/11 -4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[4->4 MsgVoteResp Term:3 Log:0/0, AppendThread->4 MsgStorageAppendResp Term:3 Log:0/0] +4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[4->4 MsgVoteResp Term:3 Log:0/0] deliver-msgs 5 6 7 ---- -AppendThread->5 MsgStorageAppendResp Term:2 Log:0/0 4->5 MsgVote Term:3 Log:1/11 INFO 5 [term: 2] received a MsgVote message with higher term from 4 [term: 3] INFO 5 became follower at term 3 INFO 5 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 -AppendThread->6 MsgStorageAppendResp Term:2 Log:0/0 4->6 MsgVote Term:3 Log:1/11 INFO 6 [term: 2] received a MsgVote message with higher term from 4 [term: 3] INFO 6 became follower at term 3 @@ -267,18 +260,18 @@ process-ready 5 6 7 Ready MustSync=true: HardState Term:3 Vote:4 Commit:11 Messages: - 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[5->4 MsgVoteResp Term:3 Log:0/0, AppendThread->5 MsgStorageAppendResp Term:3 Log:0/0] + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[5->4 MsgVoteResp Term:3 Log:0/0] > 6 handling Ready Ready MustSync=true: HardState Term:3 Vote:4 Commit:11 Messages: - 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[6->4 MsgVoteResp Term:3 Log:0/0, AppendThread->6 MsgStorageAppendResp Term:3 Log:0/0] + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[6->4 MsgVoteResp Term:3 Log:0/0] > 7 handling Ready Ready MustSync=true: Lead:0 State:StateFollower HardState Term:3 Vote:4 Commit:11 Messages: - 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[7->4 MsgVoteResp Term:3 Log:0/0, AppendThread->7 MsgStorageAppendResp Term:3 Log:0/0] + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[7->4 MsgVoteResp Term:3 Log:0/0] process-append-thread 4 5 6 7 ---- @@ -287,33 +280,27 @@ process-append-thread 4 5 6 7 4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses: 4->4 MsgVoteResp Term:3 Log:0/0 - AppendThread->4 MsgStorageAppendResp Term:3 Log:0/0 > 5 processing append thread Processing: 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses: 5->4 MsgVoteResp Term:3 Log:0/0 - AppendThread->5 MsgStorageAppendResp Term:3 Log:0/0 > 6 processing append thread Processing: 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses: 6->4 MsgVoteResp Term:3 Log:0/0 - AppendThread->6 MsgStorageAppendResp Term:3 Log:0/0 > 7 processing append thread Processing: 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses: 7->4 MsgVoteResp Term:3 Log:0/0 - AppendThread->7 MsgStorageAppendResp Term:3 Log:0/0 deliver-msgs 4 ---- -AppendThread->4 MsgStorageAppendResp Term:2 Log:0/0 4->4 MsgVoteResp Term:3 Log:0/0 INFO 4 received MsgVoteResp from 4 at term 3 INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections -AppendThread->4 MsgStorageAppendResp Term:3 Log:0/0 5->4 MsgVoteResp Term:3 Log:0/0 INFO 4 received MsgVoteResp from 5 at term 3 INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections diff --git a/testdata/replicate_pause.txt b/testdata/replicate_pause.txt index e7333ccc..67e64a95 100644 --- a/testdata/replicate_pause.txt +++ b/testdata/replicate_pause.txt @@ -187,4 +187,4 @@ status 1 ---- 1: StateReplicate match=17 next=18 2: StateReplicate match=17 next=18 -3: StateReplicate match=17 next=18 \ No newline at end of file +3: StateReplicate match=17 next=18 From 2c2960c5894f255ea1e9b2455d0c0c2902b598d2 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 17 Nov 2022 23:59:40 -0500 Subject: [PATCH 11/13] raft: check unstable log before firstIndex in raftLog.term MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This avoids a call to stable `Storage`. It turns a regression in firstIndex/op from 2 to 3 (or 5 to 7) into an improvement from 2 to 1 (or 5 to 3). ``` name old firstIndex/op new firstIndex/op delta RawNode/single-voter-10 3.00 ± 0% 1.00 ± 0% -66.67% (p=0.000 n=10+10) RawNode/two-voters-10 7.00 ± 0% 3.00 ± 0% -57.14% (p=0.000 n=10+10) ``` Signed-off-by: Nathan VanBenschoten --- log.go | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/log.go b/log.go index 82efda40..cd19e581 100644 --- a/log.go +++ b/log.go @@ -331,17 +331,20 @@ func (l *raftLog) lastTerm() uint64 { } func (l *raftLog) term(i uint64) (uint64, error) { - // the valid term range is [index of dummy entry, last index] + // Check the unstable log first, even before computing the valid term range, + // which may need to access stable Storage. If we find the entry's term in + // the unstable log, we know it was in the valid range. + if t, ok := l.unstable.maybeTerm(i); ok { + return t, nil + } + + // The valid term range is [index of dummy entry, last index]. dummyIndex := l.firstIndex() - 1 if i < dummyIndex || i > l.lastIndex() { // TODO: return an error instead? return 0, nil } - if t, ok := l.unstable.maybeTerm(i); ok { - return t, nil - } - t, err := l.storage.Term(i) if err == nil { return t, nil From 2e0653d20750c7cbbf8473fd1582c63a867996c1 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sat, 10 Dec 2022 14:44:26 -0500 Subject: [PATCH 12/13] raft: fix committed entry pagination with async storage writes This commit fixes the interactions between commit entry pagination and async storage writes. The pagination now properly applies across multiple Ready structs, acting as a limit on outstanding committed entries that have yet to be acked through a MsgStorageApplyResp message. The commit also resolves an abuse of the LogTerm field in MsgStorageApply{Resp}. Signed-off-by: Nathan VanBenschoten --- log.go | 96 ++++++++++++---- log_test.go | 177 ++++++++++++++++++++++++------ node_test.go | 149 +++++++++++++++++++++++++ raft.go | 58 ++++------ raft_paper_test.go | 2 +- raft_test.go | 10 +- rawnode.go | 33 +++--- rawnode_test.go | 6 +- storage.go | 2 +- testdata/async_storage_writes.txt | 120 ++++++++++---------- util.go | 42 +++++-- util_test.go | 2 +- 12 files changed, 513 insertions(+), 184 deletions(-) diff --git a/log.go b/log.go index cd19e581..daf7acef 100644 --- a/log.go +++ b/log.go @@ -36,18 +36,30 @@ type raftLog struct { // been instructed to apply to its state machine. Some of these // entries may be in the process of applying and have not yet // reached applied. + // Use: The field is incremented when accepting a Ready struct. // Invariant: applied <= applying && applying <= committed applying uint64 // applied is the highest log position that the application has // successfully applied to its state machine. + // Use: The field is incremented when advancing after the committed + // entries in a Ready struct have been applied (either synchronously + // or asynchronously). // Invariant: applied <= committed applied uint64 logger Logger - // maxNextCommittedEntsSize is the maximum number aggregate byte size of the - // messages returned from calls to nextCommittedEnts. - maxNextCommittedEntsSize uint64 + // maxApplyingEntsSize limits the outstanding byte size of the messages + // returned from calls to nextCommittedEnts that have not been acknowledged + // by a call to appliedTo. + maxApplyingEntsSize entryEncodingSize + // applyingEntsSize is the current outstanding byte size of the messages + // returned from calls to nextCommittedEnts that have not been acknowledged + // by a call to appliedTo. + applyingEntsSize entryEncodingSize + // applyingEntsPaused is true when entry application has been paused until + // enough progress is acknowledged. + applyingEntsPaused bool } // newLog returns log using the given storage and default options. It @@ -59,14 +71,14 @@ func newLog(storage Storage, logger Logger) *raftLog { // newLogWithSize returns a log using the given storage and max // message size. -func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uint64) *raftLog { +func newLogWithSize(storage Storage, logger Logger, maxApplyingEntsSize entryEncodingSize) *raftLog { if storage == nil { log.Panic("storage must not be nil") } log := &raftLog{ - storage: storage, - logger: logger, - maxNextCommittedEntsSize: maxNextCommittedEntsSize, + storage: storage, + logger: logger, + maxApplyingEntsSize: maxApplyingEntsSize, } firstIndex, err := storage.FirstIndex() if err != nil { @@ -81,8 +93,8 @@ func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uin log.unstable.logger = logger // Initialize our committed and applied pointers to the time of the last compaction. log.committed = firstIndex - 1 - log.applied = firstIndex - 1 log.applying = firstIndex - 1 + log.applied = firstIndex - 1 return log } @@ -204,20 +216,25 @@ func (l *raftLog) hasNextOrInProgressUnstableEnts() bool { // entries from the unstable log may be returned; otherwise, only entries known // to reside locally on stable storage will be returned. func (l *raftLog) nextCommittedEnts(allowUnstable bool) (ents []pb.Entry) { + if l.applyingEntsPaused { + // Entry application outstanding size limit reached. + return nil + } if l.hasNextOrInProgressSnapshot() { // See comment in hasNextCommittedEnts. return nil } - lo, hi := l.applying+1, l.committed+1 // [lo, hi) - if !allowUnstable { - hi = min(hi, l.unstable.offset) - } + lo, hi := l.applying+1, l.maxAppliableIndex(allowUnstable)+1 // [lo, hi) if lo >= hi { // Nothing to apply. return nil } - // TODO: handle pagination correctly. - ents, err := l.slice(lo, hi, l.maxNextCommittedEntsSize) + maxSize := l.maxApplyingEntsSize - l.applyingEntsSize + if maxSize <= 0 { + l.logger.Panicf("applying entry size (%d-%d)=%d not positive", + l.maxApplyingEntsSize, l.applyingEntsSize, maxSize) + } + ents, err := l.slice(lo, hi, maxSize) if err != nil { l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err) } @@ -227,17 +244,30 @@ func (l *raftLog) nextCommittedEnts(allowUnstable bool) (ents []pb.Entry) { // hasNextCommittedEnts returns if there is any available entries for execution. // This is a fast check without heavy raftLog.slice() in nextCommittedEnts(). func (l *raftLog) hasNextCommittedEnts(allowUnstable bool) bool { + if l.applyingEntsPaused { + // Entry application outstanding size limit reached. + return false + } if l.hasNextOrInProgressSnapshot() { // If we have a snapshot to apply, don't also return any committed // entries. Doing so raises questions about what should be applied // first. return false } - lo, hi := l.applying+1, l.committed+1 // [lo, hi) + lo, hi := l.applying+1, l.maxAppliableIndex(allowUnstable)+1 // [lo, hi) + return lo < hi +} + +// maxAppliableIndex returns the maximum committed index that can be applied. +// If allowUnstable is true, committed entries from the unstable log can be +// applied; otherwise, only entries known to reside locally on stable storage +// can be applied. +func (l *raftLog) maxAppliableIndex(allowUnstable bool) uint64 { + hi := l.committed if !allowUnstable { - hi = min(hi, l.unstable.offset) + hi = min(hi, l.unstable.offset-1) } - return lo < hi + return hi } // nextUnstableSnapshot returns the snapshot, if present, that is available to @@ -297,19 +327,39 @@ func (l *raftLog) commitTo(tocommit uint64) { } } -func (l *raftLog) appliedTo(i uint64) { +func (l *raftLog) appliedTo(i uint64, size entryEncodingSize) { if l.committed < i || i < l.applied { l.logger.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed) } l.applied = i l.applying = max(l.applying, i) + if l.applyingEntsSize > size { + l.applyingEntsSize -= size + } else { + // Defense against underflow. + l.applyingEntsSize = 0 + } + l.applyingEntsPaused = l.applyingEntsSize >= l.maxApplyingEntsSize } -func (l *raftLog) acceptApplying(i uint64) { +func (l *raftLog) acceptApplying(i uint64, size entryEncodingSize, allowUnstable bool) { if l.committed < i { l.logger.Panicf("applying(%d) is out of range [prevApplying(%d), committed(%d)]", i, l.applying, l.committed) } l.applying = i + l.applyingEntsSize += size + // Determine whether to pause entry application until some progress is + // acknowledged. We pause in two cases: + // 1. the outstanding entry size equals or exceeds the maximum size. + // 2. the outstanding entry size does not equal or exceed the maximum size, + // but we determine that the next entry in the log will push us over the + // limit. We determine this by comparing the last entry returned from + // raftLog.nextCommittedEnts to the maximum entry that the method was + // allowed to return had there been no size limit. If these indexes are + // not equal, then the returned entries slice must have been truncated to + // adhere to the memory limit. + l.applyingEntsPaused = l.applyingEntsSize >= l.maxApplyingEntsSize || + i < l.maxAppliableIndex(allowUnstable) } func (l *raftLog) stableTo(i, t uint64) { l.unstable.stableTo(i, t) } @@ -355,11 +405,11 @@ func (l *raftLog) term(i uint64) (uint64, error) { panic(err) // TODO(bdarnell) } -func (l *raftLog) entries(i, maxsize uint64) ([]pb.Entry, error) { +func (l *raftLog) entries(i uint64, maxSize entryEncodingSize) ([]pb.Entry, error) { if i > l.lastIndex() { return nil, nil } - return l.slice(i, l.lastIndex()+1, maxsize) + return l.slice(i, l.lastIndex()+1, maxSize) } // allEntries returns all entries in the log. @@ -408,7 +458,7 @@ func (l *raftLog) restore(s pb.Snapshot) { } // slice returns a slice of log entries from lo through hi-1, inclusive. -func (l *raftLog) slice(lo, hi, maxSize uint64) ([]pb.Entry, error) { +func (l *raftLog) slice(lo, hi uint64, maxSize entryEncodingSize) ([]pb.Entry, error) { err := l.mustCheckOutOfBounds(lo, hi) if err != nil { return nil, err @@ -418,7 +468,7 @@ func (l *raftLog) slice(lo, hi, maxSize uint64) ([]pb.Entry, error) { } var ents []pb.Entry if lo < l.unstable.offset { - storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable.offset), maxSize) + storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable.offset), uint64(maxSize)) if err == ErrCompacted { return nil, err } else if err == ErrUnavailable { diff --git a/log_test.go b/log_test.go index 3e03c7a2..89d5827e 100644 --- a/log_test.go +++ b/log_test.go @@ -270,7 +270,7 @@ func TestCompactionSideEffects(t *testing.T) { } require.True(t, raftLog.maybeCommit(lastIndex, lastTerm)) - raftLog.appliedTo(raftLog.committed) + raftLog.appliedTo(raftLog.committed, 0 /* size */) offset := uint64(500) storage.Compact(offset) @@ -310,22 +310,25 @@ func TestHasNextCommittedEnts(t *testing.T) { applied uint64 applying uint64 allowUnstable bool + paused bool snap bool whasNext bool }{ - {applied: 3, applying: 3, allowUnstable: true, snap: false, whasNext: true}, - {applied: 3, applying: 4, allowUnstable: true, snap: false, whasNext: true}, - {applied: 3, applying: 5, allowUnstable: true, snap: false, whasNext: false}, - {applied: 4, applying: 4, allowUnstable: true, snap: false, whasNext: true}, - {applied: 4, applying: 5, allowUnstable: true, snap: false, whasNext: false}, - {applied: 5, applying: 5, allowUnstable: true, snap: false, whasNext: false}, + {applied: 3, applying: 3, allowUnstable: true, whasNext: true}, + {applied: 3, applying: 4, allowUnstable: true, whasNext: true}, + {applied: 3, applying: 5, allowUnstable: true, whasNext: false}, + {applied: 4, applying: 4, allowUnstable: true, whasNext: true}, + {applied: 4, applying: 5, allowUnstable: true, whasNext: false}, + {applied: 5, applying: 5, allowUnstable: true, whasNext: false}, // Don't allow unstable entries. - {applied: 3, applying: 3, allowUnstable: false, snap: false, whasNext: true}, - {applied: 3, applying: 4, allowUnstable: false, snap: false, whasNext: false}, - {applied: 3, applying: 5, allowUnstable: false, snap: false, whasNext: false}, - {applied: 4, applying: 4, allowUnstable: false, snap: false, whasNext: false}, - {applied: 4, applying: 5, allowUnstable: false, snap: false, whasNext: false}, - {applied: 5, applying: 5, allowUnstable: false, snap: false, whasNext: false}, + {applied: 3, applying: 3, allowUnstable: false, whasNext: true}, + {applied: 3, applying: 4, allowUnstable: false, whasNext: false}, + {applied: 3, applying: 5, allowUnstable: false, whasNext: false}, + {applied: 4, applying: 4, allowUnstable: false, whasNext: false}, + {applied: 4, applying: 5, allowUnstable: false, whasNext: false}, + {applied: 5, applying: 5, allowUnstable: false, whasNext: false}, + // Paused. + {applied: 3, applying: 3, allowUnstable: true, paused: true, whasNext: false}, // With snapshot. {applied: 3, applying: 3, allowUnstable: true, snap: true, whasNext: false}, } @@ -339,8 +342,9 @@ func TestHasNextCommittedEnts(t *testing.T) { raftLog.append(ents...) raftLog.stableTo(4, 1) raftLog.maybeCommit(5, 1) - raftLog.appliedTo(tt.applied) - raftLog.acceptApplying(tt.applying) + raftLog.appliedTo(tt.applied, 0 /* size */) + raftLog.acceptApplying(tt.applying, 0 /* size */, tt.allowUnstable) + raftLog.applyingEntsPaused = tt.paused if tt.snap { newSnap := snap newSnap.Metadata.Index++ @@ -364,22 +368,25 @@ func TestNextCommittedEnts(t *testing.T) { applied uint64 applying uint64 allowUnstable bool + paused bool snap bool wents []pb.Entry }{ - {applied: 3, applying: 3, allowUnstable: true, snap: false, wents: ents[:2]}, - {applied: 3, applying: 4, allowUnstable: true, snap: false, wents: ents[1:2]}, - {applied: 3, applying: 5, allowUnstable: true, snap: false, wents: nil}, - {applied: 4, applying: 4, allowUnstable: true, snap: false, wents: ents[1:2]}, - {applied: 4, applying: 5, allowUnstable: true, snap: false, wents: nil}, - {applied: 5, applying: 5, allowUnstable: true, snap: false, wents: nil}, + {applied: 3, applying: 3, allowUnstable: true, wents: ents[:2]}, + {applied: 3, applying: 4, allowUnstable: true, wents: ents[1:2]}, + {applied: 3, applying: 5, allowUnstable: true, wents: nil}, + {applied: 4, applying: 4, allowUnstable: true, wents: ents[1:2]}, + {applied: 4, applying: 5, allowUnstable: true, wents: nil}, + {applied: 5, applying: 5, allowUnstable: true, wents: nil}, // Don't allow unstable entries. - {applied: 3, applying: 3, allowUnstable: false, snap: false, wents: ents[:1]}, - {applied: 3, applying: 4, allowUnstable: false, snap: false, wents: nil}, - {applied: 3, applying: 5, allowUnstable: false, snap: false, wents: nil}, - {applied: 4, applying: 4, allowUnstable: false, snap: false, wents: nil}, - {applied: 4, applying: 5, allowUnstable: false, snap: false, wents: nil}, - {applied: 5, applying: 5, allowUnstable: false, snap: false, wents: nil}, + {applied: 3, applying: 3, allowUnstable: false, wents: ents[:1]}, + {applied: 3, applying: 4, allowUnstable: false, wents: nil}, + {applied: 3, applying: 5, allowUnstable: false, wents: nil}, + {applied: 4, applying: 4, allowUnstable: false, wents: nil}, + {applied: 4, applying: 5, allowUnstable: false, wents: nil}, + {applied: 5, applying: 5, allowUnstable: false, wents: nil}, + // Paused. + {applied: 3, applying: 3, allowUnstable: true, paused: true, wents: nil}, // With snapshot. {applied: 3, applying: 3, allowUnstable: true, snap: true, wents: nil}, } @@ -393,8 +400,9 @@ func TestNextCommittedEnts(t *testing.T) { raftLog.append(ents...) raftLog.stableTo(4, 1) raftLog.maybeCommit(5, 1) - raftLog.appliedTo(tt.applied) - raftLog.acceptApplying(tt.applying) + raftLog.appliedTo(tt.applied, 0 /* size */) + raftLog.acceptApplying(tt.applying, 0 /* size */, tt.allowUnstable) + raftLog.applyingEntsPaused = tt.paused if tt.snap { newSnap := snap newSnap.Metadata.Index++ @@ -405,6 +413,113 @@ func TestNextCommittedEnts(t *testing.T) { } } +func TestAcceptApplying(t *testing.T) { + maxSize := entryEncodingSize(100) + snap := pb.Snapshot{ + Metadata: pb.SnapshotMetadata{Term: 1, Index: 3}, + } + ents := []pb.Entry{ + {Term: 1, Index: 4}, + {Term: 1, Index: 5}, + {Term: 1, Index: 6}, + } + tests := []struct { + index uint64 + allowUnstable bool + size entryEncodingSize + wpaused bool + }{ + {index: 3, allowUnstable: true, size: maxSize - 1, wpaused: true}, + {index: 3, allowUnstable: true, size: maxSize, wpaused: true}, + {index: 3, allowUnstable: true, size: maxSize + 1, wpaused: true}, + {index: 4, allowUnstable: true, size: maxSize - 1, wpaused: true}, + {index: 4, allowUnstable: true, size: maxSize, wpaused: true}, + {index: 4, allowUnstable: true, size: maxSize + 1, wpaused: true}, + {index: 5, allowUnstable: true, size: maxSize - 1, wpaused: false}, + {index: 5, allowUnstable: true, size: maxSize, wpaused: true}, + {index: 5, allowUnstable: true, size: maxSize + 1, wpaused: true}, + // Don't allow unstable entries. + {index: 3, allowUnstable: false, size: maxSize - 1, wpaused: true}, + {index: 3, allowUnstable: false, size: maxSize, wpaused: true}, + {index: 3, allowUnstable: false, size: maxSize + 1, wpaused: true}, + {index: 4, allowUnstable: false, size: maxSize - 1, wpaused: false}, + {index: 4, allowUnstable: false, size: maxSize, wpaused: true}, + {index: 4, allowUnstable: false, size: maxSize + 1, wpaused: true}, + {index: 5, allowUnstable: false, size: maxSize - 1, wpaused: false}, + {index: 5, allowUnstable: false, size: maxSize, wpaused: true}, + {index: 5, allowUnstable: false, size: maxSize + 1, wpaused: true}, + } + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + storage := NewMemoryStorage() + require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) + + raftLog := newLogWithSize(storage, raftLogger, maxSize) + raftLog.append(ents...) + raftLog.stableTo(4, 1) + raftLog.maybeCommit(5, 1) + raftLog.appliedTo(3, 0 /* size */) + + raftLog.acceptApplying(tt.index, tt.size, tt.allowUnstable) + require.Equal(t, tt.wpaused, raftLog.applyingEntsPaused) + }) + } +} + +func TestAppliedTo(t *testing.T) { + maxSize := entryEncodingSize(100) + overshoot := entryEncodingSize(5) + snap := pb.Snapshot{ + Metadata: pb.SnapshotMetadata{Term: 1, Index: 3}, + } + ents := []pb.Entry{ + {Term: 1, Index: 4}, + {Term: 1, Index: 5}, + {Term: 1, Index: 6}, + } + tests := []struct { + index uint64 + size entryEncodingSize + wapplyingSize entryEncodingSize + wpaused bool + }{ + // Apply some of in-progress entries (applying = 5 below). + {index: 4, size: overshoot - 1, wapplyingSize: maxSize + 1, wpaused: true}, + {index: 4, size: overshoot, wapplyingSize: maxSize, wpaused: true}, + {index: 4, size: overshoot + 1, wapplyingSize: maxSize - 1, wpaused: false}, + // Apply all of in-progress entries. + {index: 5, size: overshoot - 1, wapplyingSize: maxSize + 1, wpaused: true}, + {index: 5, size: overshoot, wapplyingSize: maxSize, wpaused: true}, + {index: 5, size: overshoot + 1, wapplyingSize: maxSize - 1, wpaused: false}, + // Apply all of outstanding bytes. + {index: 4, size: maxSize + overshoot, wapplyingSize: 0, wpaused: false}, + // Apply more than outstanding bytes. + // Incorrect accounting doesn't underflow applyingSize. + {index: 4, size: maxSize + overshoot + 1, wapplyingSize: 0, wpaused: false}, + } + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + storage := NewMemoryStorage() + require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) + + raftLog := newLogWithSize(storage, raftLogger, maxSize) + raftLog.append(ents...) + raftLog.stableTo(4, 1) + raftLog.maybeCommit(5, 1) + raftLog.appliedTo(3, 0 /* size */) + raftLog.acceptApplying(5, maxSize+overshoot, false /* allowUnstable */) + + raftLog.appliedTo(tt.index, tt.size) + require.Equal(t, tt.index, raftLog.applied) + require.Equal(t, uint64(5), raftLog.applying) + require.Equal(t, tt.wapplyingSize, raftLog.applyingEntsSize) + require.Equal(t, tt.wpaused, raftLog.applyingEntsPaused) + }) + } +} + // TestNextUnstableEnts ensures unstableEntries returns the unstable part of the // entries correctly. func TestNextUnstableEnts(t *testing.T) { @@ -553,7 +668,7 @@ func TestCompaction(t *testing.T) { raftLog := newLog(storage, raftLogger) raftLog.maybeCommit(tt.lastIndex, 0) - raftLog.appliedTo(raftLog.committed) + raftLog.appliedTo(raftLog.committed, 0 /* size */) for j := 0; j < len(tt.compact); j++ { err := storage.Compact(tt.compact[j]) if err != nil { @@ -765,7 +880,7 @@ func TestSlice(t *testing.T) { require.True(t, tt.wpanic) } }() - g, err := l.slice(tt.from, tt.to, tt.limit) + g, err := l.slice(tt.from, tt.to, entryEncodingSize(tt.limit)) require.False(t, tt.from <= offset && err != ErrCompacted) require.False(t, tt.from > offset && err != nil) require.Equal(t, tt.w, g) diff --git a/node_test.go b/node_test.go index 4343c234..068002b8 100644 --- a/node_test.go +++ b/node_test.go @@ -24,6 +24,8 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "go.etcd.io/raft/v3/raftpb" ) @@ -937,6 +939,153 @@ func TestCommitPagination(t *testing.T) { n.Advance() } +func TestCommitPaginationWithAsyncStorageWrites(t *testing.T) { + s := newTestMemoryStorage(withPeers(1)) + cfg := newTestConfig(1, 10, 1, s) + cfg.MaxCommittedSizePerReady = 2048 + cfg.AsyncStorageWrites = true + ctx, cancel, n := newNodeTestHarness(context.Background(), t, cfg) + defer cancel() + n.Campaign(ctx) + + // Persist vote. + rd := readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m := rd.Messages[0] + require.Equal(t, raftpb.MsgStorageAppend, m.Type) + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + // Append empty entry. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m = rd.Messages[0] + require.Equal(t, raftpb.MsgStorageAppend, m.Type) + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + // Apply empty entry. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 2) + for _, m := range rd.Messages { + switch m.Type { + case raftpb.MsgStorageAppend: + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + case raftpb.MsgStorageApply: + if len(m.Entries) != 1 { + t.Fatalf("expected 1 (empty) entry, got %d", len(m.Entries)) + } + require.Len(t, m.Responses, 1) + require.NoError(t, n.Step(ctx, m.Responses[0])) + default: + t.Fatalf("unexpected: %v", m) + } + } + + // Propose first entry. + blob := []byte(strings.Repeat("a", 1024)) + require.NoError(t, n.Propose(ctx, blob)) + + // Append first entry. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m = rd.Messages[0] + require.Equal(t, raftpb.MsgStorageAppend, m.Type) + require.Len(t, m.Entries, 1) + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + + // Propose second entry. + require.NoError(t, n.Propose(ctx, blob)) + + // Append second entry. Don't apply first entry yet. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 2) + var applyResps []raftpb.Message + for _, m := range rd.Messages { + switch m.Type { + case raftpb.MsgStorageAppend: + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + case raftpb.MsgStorageApply: + if len(m.Entries) != 1 { + t.Fatalf("expected 1 (empty) entry, got %d", len(m.Entries)) + } + require.Len(t, m.Responses, 1) + applyResps = append(applyResps, m.Responses[0]) + default: + t.Fatalf("unexpected: %v", m) + } + } + + // Propose third entry. + require.NoError(t, n.Propose(ctx, blob)) + + // Append third entry. Don't apply second entry yet. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 2) + for _, m := range rd.Messages { + switch m.Type { + case raftpb.MsgStorageAppend: + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + case raftpb.MsgStorageApply: + if len(m.Entries) != 1 { + t.Fatalf("expected 1 (empty) entry, got %d", len(m.Entries)) + } + require.Len(t, m.Responses, 1) + applyResps = append(applyResps, m.Responses[0]) + default: + t.Fatalf("unexpected: %v", m) + } + } + + // Third entry should not be returned to be applied until first entry's + // application is acknowledged. + drain := true + for drain { + select { + case rd := <-n.Ready(): + for _, m := range rd.Messages { + if m.Type == raftpb.MsgStorageApply { + t.Fatalf("expected MsgStorageApply, %v", m) + } + } + case <-time.After(10 * time.Millisecond): + drain = false + } + } + + // Acknowledged first entry application. + require.NoError(t, n.Step(ctx, applyResps[0])) + applyResps = applyResps[1:] + + // Third entry now returned for application. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m = rd.Messages[0] + require.Equal(t, raftpb.MsgStorageApply, m.Type) + require.Len(t, m.Entries, 1) + applyResps = append(applyResps, m.Responses[0]) + + // Acknowledged second and third entry application. + for _, resp := range applyResps { + require.NoError(t, n.Step(ctx, resp)) + } + applyResps = nil +} + type ignoreSizeHintMemStorage struct { *MemoryStorage } diff --git a/raft.go b/raft.go index 58eccb17..d43e917b 100644 --- a/raft.go +++ b/raft.go @@ -194,7 +194,11 @@ type Config struct { // 0 for at most one entry per message. MaxSizePerMsg uint64 // MaxCommittedSizePerReady limits the size of the committed entries which - // can be applied. + // can be applying at the same time. + // + // Despite its name (preserved for compatibility), this quota applies across + // Ready structs to encompass all outstanding entries in unacknowledged + // MsgStorageApply messages when AsyncStorageWrites is enabled. MaxCommittedSizePerReady uint64 // MaxUncommittedEntriesSize limits the aggregate byte size of the // uncommitted entries that may be appended to a leader's log. Once this @@ -316,8 +320,8 @@ type raft struct { // the log raftLog *raftLog - maxMsgSize uint64 - maxUncommittedSize uint64 + maxMsgSize entryEncodingSize + maxUncommittedSize entryPayloadSize // TODO(tbg): rename to trk. prs tracker.ProgressTracker @@ -358,7 +362,7 @@ type raft struct { // an estimate of the size of the uncommitted tail of the Raft log. Used to // prevent unbounded log growth. Only maintained by the leader. Reset on // term changes. - uncommittedSize uint64 + uncommittedSize entryPayloadSize readOnly *readOnly @@ -399,7 +403,7 @@ func newRaft(c *Config) *raft { if err := c.validate(); err != nil { panic(err.Error()) } - raftlog := newLogWithSize(c.Storage, c.Logger, c.MaxCommittedSizePerReady) + raftlog := newLogWithSize(c.Storage, c.Logger, entryEncodingSize(c.MaxCommittedSizePerReady)) hs, cs, err := c.Storage.InitialState() if err != nil { panic(err) // TODO(bdarnell) @@ -410,8 +414,8 @@ func newRaft(c *Config) *raft { lead: None, isLearner: false, raftLog: raftlog, - maxMsgSize: c.MaxSizePerMsg, - maxUncommittedSize: c.MaxUncommittedEntriesSize, + maxMsgSize: entryEncodingSize(c.MaxSizePerMsg), + maxUncommittedSize: entryPayloadSize(c.MaxUncommittedEntriesSize), prs: tracker.MakeProgressTracker(c.MaxInflightMsgs, c.MaxInflightBytes), electionTimeout: c.ElectionTick, heartbeatTimeout: c.HeartbeatTick, @@ -435,7 +439,7 @@ func newRaft(c *Config) *raft { r.loadState(hs) } if c.Applied > 0 { - raftlog.appliedTo(c.Applied) + raftlog.appliedTo(c.Applied, 0 /* size */) } r.becomeFollower(r.Term, None) @@ -613,7 +617,7 @@ func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool { // Send the actual MsgApp otherwise, and update the progress accordingly. next := pr.Next // save Next for later, as the progress update can change it - if err := pr.UpdateOnEntriesSend(len(ents), payloadsSize(ents), next); err != nil { + if err := pr.UpdateOnEntriesSend(len(ents), uint64(payloadsSize(ents)), next); err != nil { r.logger.Panicf("%x: %v", r.id, err) } r.send(pb.Message{ @@ -676,10 +680,10 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { }) } -func (r *raft) appliedTo(index uint64) { +func (r *raft) appliedTo(index uint64, size entryEncodingSize) { oldApplied := r.raftLog.applied newApplied := max(index, oldApplied) - r.raftLog.appliedTo(newApplied) + r.raftLog.appliedTo(newApplied, size) if r.prs.Config.AutoLeave && newApplied >= r.pendingConfIndex && r.state == StateLeader { // If the current (and most recent, at least for this leader's term) @@ -708,7 +712,7 @@ func (r *raft) appliedTo(index uint64) { func (r *raft) appliedSnap(snap *pb.Snapshot) { index := snap.Metadata.Index r.raftLog.stableSnapTo(index) - r.appliedTo(index) + r.appliedTo(index, 0 /* size */) } // maybeCommit attempts to advance the commit index. Returns true if @@ -897,7 +901,7 @@ func (r *raft) becomeLeader() { // uncommitted log quota. This is because we want to preserve the // behavior of allowing one entry larger than quota if the current // usage is zero. - r.uncommittedSize = 0 + r.reduceUncommittedSize(payloadSize(emptyEnt)) r.logger.Infof("%x became leader at term %d", r.id, r.Term) } @@ -1091,10 +1095,11 @@ func (r *raft) Step(m pb.Message) error { } case pb.MsgStorageApplyResp: - r.appliedTo(m.Index) - // NOTE: we abuse the LogTerm field to store the aggregate entry size so - // that we don't need to introduce a new field on Message. - r.reduceUncommittedSize(m.LogTerm) + if len(m.Entries) > 0 { + index := m.Entries[len(m.Entries)-1].Index + r.appliedTo(index, entsSize(m.Entries)) + r.reduceUncommittedSize(payloadsSize(m.Entries)) + } case pb.MsgVote, pb.MsgPreVote: // We can vote if this is a repeat of a vote we've already cast... @@ -1948,18 +1953,9 @@ func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool { return true } -// getUncommittedSize computes the aggregate size of the provided entries. -func (r *raft) getUncommittedSize(ents []pb.Entry) uint64 { - if r.uncommittedSize == 0 { - // Fast-path for followers, who do not track or enforce the limit. - return 0 - } - return payloadsSize(ents) -} - // reduceUncommittedSize accounts for the newly committed entries by decreasing // the uncommitted entry size limit. -func (r *raft) reduceUncommittedSize(s uint64) { +func (r *raft) reduceUncommittedSize(s entryPayloadSize) { if s > r.uncommittedSize { // uncommittedSize may underestimate the size of the uncommitted Raft // log tail but will never overestimate it. Saturate at 0 instead of @@ -1970,14 +1966,6 @@ func (r *raft) reduceUncommittedSize(s uint64) { } } -func payloadsSize(ents []pb.Entry) uint64 { - var s uint64 - for _, e := range ents { - s += uint64(PayloadSize(e)) - } - return s -} - func numOfPendingConf(ents []pb.Entry) int { n := 0 for i := range ents { diff --git a/raft_paper_test.go b/raft_paper_test.go index 585168d0..d7e9949c 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -922,7 +922,7 @@ func commitNoopEntry(r *raft, s *MemoryStorage) { // ignore further messages to refresh followers' commit index r.readMessages() s.Append(r.raftLog.nextUnstableEnts()) - r.raftLog.appliedTo(r.raftLog.committed) + r.raftLog.appliedTo(r.raftLog.committed, 0 /* size */) r.raftLog.stableTo(r.raftLog.lastIndex(), r.raftLog.lastTerm()) } diff --git a/raft_test.go b/raft_test.go index f5979168..6abda3cf 100644 --- a/raft_test.go +++ b/raft_test.go @@ -38,7 +38,7 @@ func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) { // Return committed entries. ents = r.raftLog.nextCommittedEnts(true) - r.raftLog.appliedTo(r.raftLog.committed) + r.raftLog.appliedTo(r.raftLog.committed, 0 /* size */) return ents } @@ -240,9 +240,9 @@ func TestUncommittedEntryLimit(t *testing.T) { // writing, the former). const maxEntries = 1024 testEntry := pb.Entry{Data: []byte("testdata")} - maxEntrySize := maxEntries * PayloadSize(testEntry) + maxEntrySize := maxEntries * payloadSize(testEntry) - if n := PayloadSize(pb.Entry{Data: nil}); n != 0 { + if n := payloadSize(pb.Entry{Data: nil}); n != 0 { t.Fatal("entry with no Data must have zero payload size") } @@ -283,7 +283,7 @@ func TestUncommittedEntryLimit(t *testing.T) { if e := maxEntries * numFollowers; len(ms) != e { t.Fatalf("expected %d messages, got %d", e, len(ms)) } - r.reduceUncommittedSize(r.getUncommittedSize(propEnts)) + r.reduceUncommittedSize(payloadsSize(propEnts)) if r.uncommittedSize != 0 { t.Fatalf("committed everything, but still tracking %d", r.uncommittedSize) } @@ -319,7 +319,7 @@ func TestUncommittedEntryLimit(t *testing.T) { if e := 2 * numFollowers; len(ms) != e { t.Fatalf("expected %d messages, got %d", e, len(ms)) } - r.reduceUncommittedSize(r.getUncommittedSize(propEnts)) + r.reduceUncommittedSize(payloadsSize(propEnts)) if n := r.uncommittedSize; n != 0 { t.Fatalf("expected zero uncommitted size, got %d", n) } diff --git a/rawnode.go b/rawnode.go index 675fa5bc..1e88a801 100644 --- a/rawnode.go +++ b/rawnode.go @@ -140,7 +140,7 @@ func (rn *RawNode) readyWithoutAccept() Ready { rd := Ready{ Entries: r.raftLog.nextUnstableEnts(), - CommittedEntries: r.raftLog.nextCommittedEnts(!rn.asyncStorageWrites), + CommittedEntries: r.raftLog.nextCommittedEnts(rn.applyUnstableEntries()), Messages: r.msgs, } if softSt := r.softState(); !softSt.equal(rn.prevSoftSt) { @@ -368,14 +368,12 @@ func needStorageApplyRespMsg(rd Ready) bool { return needStorageApplyMsg(rd) } // message is processed. Used with AsyncStorageWrites. func newStorageApplyMsg(r *raft, rd Ready) pb.Message { ents := rd.CommittedEntries - last := ents[len(ents)-1].Index return pb.Message{ Type: pb.MsgStorageApply, To: LocalApplyThread, From: r.id, Term: 0, // committed entries don't apply under a specific term Entries: ents, - Index: last, Responses: []pb.Message{ newStorageApplyRespMsg(r, ents), }, @@ -385,18 +383,13 @@ func newStorageApplyMsg(r *raft, rd Ready) pb.Message { // newStorageApplyRespMsg creates the message that should be returned to node // after the committed entries in the current Ready (along with those in all // prior Ready structs) have been applied to the local state machine. -func newStorageApplyRespMsg(r *raft, committedEnts []pb.Entry) pb.Message { - last := committedEnts[len(committedEnts)-1].Index - size := r.getUncommittedSize(committedEnts) +func newStorageApplyRespMsg(r *raft, ents []pb.Entry) pb.Message { return pb.Message{ - Type: pb.MsgStorageApplyResp, - To: r.id, - From: LocalApplyThread, - Term: 0, // committed entries don't apply under a specific term - Index: last, - // NOTE: we abuse the LogTerm field to store the aggregate entry size so - // that we don't need to introduce a new field on Message. - LogTerm: size, + Type: pb.MsgStorageApplyResp, + To: r.id, + From: LocalApplyThread, + Term: 0, // committed entries don't apply under a specific term + Entries: ents, } } @@ -436,10 +429,18 @@ func (rn *RawNode) acceptReady(rd Ready) { rn.raft.raftLog.acceptUnstable() if len(rd.CommittedEntries) > 0 { ents := rd.CommittedEntries - rn.raft.raftLog.acceptApplying(ents[len(ents)-1].Index) + index := ents[len(ents)-1].Index + rn.raft.raftLog.acceptApplying(index, entsSize(ents), rn.applyUnstableEntries()) } } +// applyUnstableEntries returns whether entries are allowed to be applied once +// they are known to be committed but before they have been written locally to +// stable storage. +func (rn *RawNode) applyUnstableEntries() bool { + return !rn.asyncStorageWrites +} + // HasReady called when RawNode user need to check if any Ready pending. func (rn *RawNode) HasReady() bool { // TODO(nvanbenschoten): order these cases in terms of cost and frequency. @@ -456,7 +457,7 @@ func (rn *RawNode) HasReady() bool { if len(r.msgs) > 0 || len(r.msgsAfterAppend) > 0 { return true } - if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(!rn.asyncStorageWrites) { + if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(rn.applyUnstableEntries()) { return true } if len(r.readStates) != 0 { diff --git a/rawnode_test.go b/rawnode_test.go index ceaed813..3e146f8b 100644 --- a/rawnode_test.go +++ b/rawnode_test.go @@ -979,12 +979,12 @@ func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) { const maxEntries = 16 data := []byte("testdata") testEntry := pb.Entry{Data: data} - maxEntrySize := uint64(maxEntries * PayloadSize(testEntry)) + maxEntrySize := maxEntries * payloadSize(testEntry) t.Log("maxEntrySize", maxEntrySize) s := newTestMemoryStorage(withPeers(1)) cfg := newTestConfig(1, 10, 1, s) - cfg.MaxUncommittedEntriesSize = maxEntrySize + cfg.MaxUncommittedEntriesSize = uint64(maxEntrySize) rawNode, err := NewRawNode(cfg) if err != nil { t.Fatal(err) @@ -1010,7 +1010,7 @@ func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) { // Check the size of leader's uncommitted log tail. It should not exceed the // MaxUncommittedEntriesSize limit. - checkUncommitted := func(exp uint64) { + checkUncommitted := func(exp entryPayloadSize) { t.Helper() if a := rawNode.raft.uncommittedSize; exp != a { t.Fatalf("expected %d uncommitted entry bytes, found %d", exp, a) diff --git a/storage.go b/storage.go index ecccb71b..b781dfa8 100644 --- a/storage.go +++ b/storage.go @@ -131,7 +131,7 @@ func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) { } ents := ms.ents[lo-offset : hi-offset] - return limitSize(ents, maxSize), nil + return limitSize(ents, entryEncodingSize(maxSize)), nil } // Term implements the Storage interface. diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt index bd48ec6e..f6a7ef8f 100644 --- a/testdata/async_storage_writes.txt +++ b/testdata/async_storage_writes.txt @@ -139,7 +139,7 @@ stabilize 1->2 MsgApp Term:1 Log:1/11 Commit:11 1->3 MsgApp Term:1 Log:1/11 Commit:11 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 - 1->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""]] > 2 receiving messages 1->2 MsgApp Term:1 Log:1/11 Commit:11 > 3 receiving messages @@ -150,9 +150,9 @@ stabilize Responses: > 1 processing apply thread Processing: - 1->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: - ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 2 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:11 @@ -160,7 +160,7 @@ stabilize 1/11 EntryNormal "" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[2->1 MsgAppResp Term:1 Log:0/11] - 2->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/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 ""]] > 3 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:11 @@ -168,9 +168,9 @@ stabilize 1/11 EntryNormal "" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[3->1 MsgAppResp Term:1 Log:0/11] - 3->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/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 ""]] > 1 receiving messages - ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/11 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 2 processing append thread Processing: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 @@ -183,21 +183,21 @@ stabilize 3->1 MsgAppResp Term:1 Log:0/11 > 2 processing apply thread Processing: - 2->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: - ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 3 processing apply thread Processing: - 3->ApplyThread MsgStorageApply Term:0 Log:0/11 Entries:[1/11 EntryNormal ""] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: - ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11 + 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 receiving messages - ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/11 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 3 receiving messages - ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/11 + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] propose 1 prop_1 ---- @@ -330,7 +330,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 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[1->1 MsgAppResp Term:1 Log:0/14, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14] - 1->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/12] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] > 2 handling Ready > 3 handling Ready @@ -356,7 +356,7 @@ process-ready 1 2 3 1/12 EntryNormal "prop_1" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[2->1 MsgAppResp Term:1 Log:0/13, 2->1 MsgAppResp Term:1 Log:0/14, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14] - 2->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/12] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] > 3 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:12 @@ -366,7 +366,7 @@ process-ready 1 2 3 1/12 EntryNormal "prop_1" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[3->1 MsgAppResp Term:1 Log:0/13, 3->1 MsgAppResp Term:1 Log:0/14, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14] - 3->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/12] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] process-append-thread 1 2 3 ---- @@ -417,7 +417,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 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[1->1 MsgAppResp Term:1 Log:0/15, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] - 1->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/13] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] > 2 handling Ready > 3 handling Ready @@ -443,7 +443,7 @@ process-ready 1 2 3 1/13 EntryNormal "prop_2" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[2->1 MsgAppResp Term:1 Log:0/14, 2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] - 2->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/13] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] > 3 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:13 @@ -453,7 +453,7 @@ process-ready 1 2 3 1/13 EntryNormal "prop_2" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[3->1 MsgAppResp Term:1 Log:0/14, 3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] - 3->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/13] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] process-append-thread 1 2 3 ---- @@ -482,19 +482,19 @@ process-apply-thread 1 2 3 ---- > 1 processing apply thread Processing: - 1->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: - ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/12 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] > 2 processing apply thread Processing: - 2->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: - ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/12 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] > 3 processing apply thread Processing: - 3->ApplyThread MsgStorageApply Term:0 Log:0/12 Entries:[1/12 EntryNormal "prop_1"] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: - ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/12 + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] deliver-msgs 1 2 3 ---- @@ -504,11 +504,11 @@ AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 2->1 MsgAppResp Term:1 Log:0/14 3->1 MsgAppResp Term:1 Log:0/13 3->1 MsgAppResp Term:1 Log:0/14 -ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/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/12 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 -ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/12 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] process-ready 1 2 3 ---- @@ -521,7 +521,7 @@ process-ready 1 2 3 1->2 MsgApp Term:1 Log:1/15 Commit:14 1->3 MsgApp Term:1 Log:1/15 Commit:14 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] - 1->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/14] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] > 2 handling Ready > 3 handling Ready @@ -543,7 +543,7 @@ process-ready 1 2 3 1/14 EntryNormal "prop_3" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] - 2->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] > 3 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:14 @@ -551,7 +551,7 @@ process-ready 1 2 3 1/14 EntryNormal "prop_3" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] - 3->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] process-append-thread 1 2 3 ---- @@ -580,19 +580,19 @@ process-apply-thread 1 2 3 ---- > 1 processing apply thread Processing: - 1->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: - ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/13 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] > 2 processing apply thread Processing: - 2->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: - ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/13 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] > 3 processing apply thread Processing: - 3->ApplyThread MsgStorageApply Term:0 Log:0/13 Entries:[1/13 EntryNormal "prop_2"] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: - ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/13 + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] deliver-msgs 1 2 3 ---- @@ -602,11 +602,11 @@ AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 2->1 MsgAppResp Term:1 Log:0/15 3->1 MsgAppResp Term:1 Log:0/14 3->1 MsgAppResp Term:1 Log:0/15 -ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/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/13 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 -ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/13 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] process-ready 1 2 3 ---- @@ -619,7 +619,7 @@ process-ready 1 2 3 1->2 MsgApp Term:1 Log:1/15 Commit:15 1->3 MsgApp Term:1 Log:1/15 Commit:15 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 - 1->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"]] > 2 handling Ready > 3 handling Ready @@ -641,7 +641,7 @@ process-ready 1 2 3 1/15 EntryNormal "prop_4" Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[2->1 MsgAppResp Term:1 Log:0/15] - 2->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/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"]] > 3 handling Ready Ready MustSync=false: HardState Term:1 Vote:1 Commit:15 @@ -649,7 +649,7 @@ process-ready 1 2 3 1/15 EntryNormal "prop_4" Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[3->1 MsgAppResp Term:1 Log:0/15] - 3->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/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"]] process-append-thread 2 3 ---- @@ -670,31 +670,31 @@ process-apply-thread 1 2 3 ---- > 1 processing apply thread Processing: - 1->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses: - ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/14 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] > 2 processing apply thread Processing: - 2->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses: - ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] > 3 processing apply thread Processing: - 3->ApplyThread MsgStorageApply Term:0 Log:0/14 Entries:[1/14 EntryNormal "prop_3"] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses: - ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14 + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] deliver-msgs 1 2 3 ---- 2->1 MsgAppResp Term:1 Log:0/15 3->1 MsgAppResp Term:1 Log:0/15 -ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/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 -ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/14 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 INFO entry at index 15 missing from unstable log; ignoring -ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/14 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] process-ready 1 2 3 ---- @@ -722,27 +722,27 @@ process-apply-thread 1 2 3 ---- > 1 processing apply thread Processing: - 1->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses: - ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15 + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] > 2 processing apply thread Processing: - 2->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses: - ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15 + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] > 3 processing apply thread Processing: - 3->ApplyThread MsgStorageApply Term:0 Log:0/15 Entries:[1/15 EntryNormal "prop_4"] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses: - ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/15 + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] deliver-msgs 1 2 3 ---- 2->1 MsgAppResp Term:1 Log:0/15 3->1 MsgAppResp Term:1 Log:0/15 -ApplyThread->1 MsgStorageApplyResp Term:0 Log:6/15 -ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/15 -ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/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"] process-ready 1 2 3 ---- diff --git a/util.go b/util.go index d5ed30b3..1bec5357 100644 --- a/util.go +++ b/util.go @@ -213,12 +213,6 @@ func describeTarget(id uint64) string { } } -// PayloadSize is the size of the payload of this Entry. Notably, it does not -// depend on its Index or Term. -func PayloadSize(e pb.Entry) int { - return len(e.Data) -} - // DescribeEntry returns a concise human-readable description of an // Entry for debugging. func DescribeEntry(e pb.Entry, f EntryFormatter) string { @@ -267,7 +261,19 @@ func DescribeEntries(ents []pb.Entry, f EntryFormatter) string { return buf.String() } -func limitSize(ents []pb.Entry, maxSize uint64) []pb.Entry { +// entryEncodingSize represents the protocol buffer encoding size of one or more +// entries. +type entryEncodingSize uint64 + +func entsSize(ents []pb.Entry) entryEncodingSize { + var size entryEncodingSize + for _, ent := range ents { + size += entryEncodingSize(ent.Size()) + } + return size +} + +func limitSize(ents []pb.Entry, maxSize entryEncodingSize) []pb.Entry { if len(ents) == 0 { return ents } @@ -275,13 +281,33 @@ func limitSize(ents []pb.Entry, maxSize uint64) []pb.Entry { var limit int for limit = 1; limit < len(ents); limit++ { size += ents[limit].Size() - if uint64(size) > maxSize { + if entryEncodingSize(size) > maxSize { break } } return ents[:limit] } +// entryPayloadSize represents the size of one or more entries' payloads. +// Notably, it does not depend on its Index or Term. Entries with empty +// payloads, like those proposed after a leadership change, are considered +// to be zero size. +type entryPayloadSize uint64 + +// payloadSize is the size of the payload of the provided entry. +func payloadSize(e pb.Entry) entryPayloadSize { + return entryPayloadSize(len(e.Data)) +} + +// payloadsSize is the size of the payloads of the provided entries. +func payloadsSize(ents []pb.Entry) entryPayloadSize { + var s entryPayloadSize + for _, e := range ents { + s += payloadSize(e) + } + return s +} + func assertConfStatesEquivalent(l Logger, cs1, cs2 pb.ConfState) { err := cs1.Equivalent(cs2) if err == nil { diff --git a/util_test.go b/util_test.go index e711ec16..fc7ddfbd 100644 --- a/util_test.go +++ b/util_test.go @@ -60,7 +60,7 @@ func TestLimitSize(t *testing.T) { for _, tt := range tests { t.Run("", func(t *testing.T) { - require.Equal(t, tt.wentries, limitSize(ents, tt.maxsize)) + require.Equal(t, tt.wentries, limitSize(ents, entryEncodingSize(tt.maxsize))) }) } } From 09c91d8dc0d2f550b69fbe183399e98b1fd6d606 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 21 Dec 2022 11:59:07 -0500 Subject: [PATCH 13/13] raft: store Vote in MsgStorageAppend, not *HardState This commit replaces the HardState field in Message with a Vote. For MsgStorageAppends, the term, vote, and commit fields will either all be set (to facilitate the construction of a HardState) if any of the fields have changed or will all be unset if none of the fields have changed. Signed-off-by: Nathan VanBenschoten --- raft.go | 12 +- raftpb/raft.pb.go | 198 ++++++++---------- raftpb/raft.proto | 16 +- raftpb/raft_test.go | 2 +- ...ction_env_handler_process_append_thread.go | 7 +- rawnode.go | 24 +-- testdata/async_storage_writes.txt | 108 +++++----- .../async_storage_writes_append_aba_race.txt | 54 ++--- util.go | 6 +- 9 files changed, 203 insertions(+), 224 deletions(-) diff --git a/raft.go b/raft.go index d43e917b..ee785945 100644 --- a/raft.go +++ b/raft.go @@ -167,12 +167,12 @@ type Config struct { // messages over the network) and those targeted at the same thread can't be // reordered. Messages to different targets can be processed in any order. // - // MsgStorageAppend carries Raft log entries to append, election votes to - // persist, and snapshots to apply. All writes performed in service of a - // MsgStorageAppend must be durable before response messages are delivered. - // However, if the MsgStorageAppend carries no response messages, durability - // is not required. The message assumes the role of the Entries, HardState, - // and Snapshot fields in Ready. + // MsgStorageAppend carries Raft log entries to append, election votes / + // term changes / updated commit indexes to persist, and snapshots to apply. + // All writes performed in service of a MsgStorageAppend must be durable + // before response messages are delivered. However, if the MsgStorageAppend + // carries no response messages, durability is not required. The message + // assumes the role of the Entries, HardState, and Snapshot fields in Ready. // // MsgStorageApply carries committed entries to apply. Writes performed in // service of a MsgStorageApply need not be durable before response messages diff --git a/raftpb/raft.pb.go b/raftpb/raft.pb.go index e1033908..14802639 100644 --- a/raftpb/raft.pb.go +++ b/raftpb/raft.pb.go @@ -404,14 +404,19 @@ type Message struct { // entries from its leader as it already has an entry with term 5 at index 100. // (type=MsgStorageAppendResp,index=100,logTerm=5) means the local node wrote // entries up to index=100 in stable storage, and the term of the entry at index - // 100 was 5. + // 100 was 5. This doesn't always mean that the corresponding MsgStorageAppend + // message was the one that carried these entries, just that those entries were + // stable at the time of processing the corresponding MsgStorageAppend. LogTerm uint64 `protobuf:"varint,5,opt,name=logTerm" json:"logTerm"` Index uint64 `protobuf:"varint,6,opt,name=index" json:"index"` Entries []Entry `protobuf:"bytes,7,rep,name=entries" json:"entries"` Commit uint64 `protobuf:"varint,8,opt,name=commit" json:"commit"` - // hardState can be non-nil for MsgStorageAppend messages and is nil for all - // other message types. - HardState *HardState `protobuf:"bytes,13,opt,name=hardState" json:"hardState,omitempty"` + // (type=MsgStorageAppend,vote=5,term=10) means the local node is voting for + // peer 5 in term 10. For MsgStorageAppends, the term, vote, and commit fields + // will either all be set (to facilitate the construction of a HardState) if + // any of the fields have changed or will all be unset if none of the fields + // have changed. + Vote uint64 `protobuf:"varint,13,opt,name=vote" json:"vote"` // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a // non-nil, empty value for the snapshot field of non-MsgSnap messages. Code @@ -721,76 +726,76 @@ func init() { func init() { proto.RegisterFile("raft.proto", fileDescriptor_b042552c306ae59b) } var fileDescriptor_b042552c306ae59b = []byte{ - // 1100 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcd, 0x6e, 0x23, 0x45, - 0x17, 0xed, 0x6e, 0x77, 0xfc, 0x73, 0xed, 0x38, 0x95, 0x8a, 0x27, 0x5f, 0x2b, 0x8a, 0x3c, 0xfe, - 0x3c, 0x83, 0xc6, 0x0a, 0x9a, 0x80, 0x8c, 0x40, 0x88, 0x5d, 0x7e, 0x46, 0x4a, 0x50, 0x1c, 0x06, - 0x27, 0x93, 0x05, 0x12, 0x8a, 0x2a, 0xee, 0x4a, 0xa7, 0xc1, 0xae, 0x6a, 0x55, 0x97, 0x43, 0xb2, - 0x41, 0x88, 0x17, 0x80, 0x25, 0x1b, 0xb6, 0xec, 0xe1, 0x29, 0xb2, 0xcc, 0x92, 0xd5, 0x88, 0x49, - 0x5e, 0x04, 0x55, 0x75, 0xf5, 0x8f, 0x9d, 0x68, 0x16, 0xec, 0xaa, 0xce, 0x39, 0x75, 0xeb, 0xdc, - 0x7b, 0xfb, 0x56, 0x03, 0x08, 0x72, 0x2e, 0x37, 0x23, 0xc1, 0x25, 0xc7, 0x65, 0xb5, 0x8e, 0xce, - 0xd6, 0x5a, 0x01, 0x0f, 0xb8, 0x86, 0x3e, 0x52, 0xab, 0x84, 0xed, 0xfe, 0x08, 0x0b, 0xaf, 0x98, - 0x14, 0xd7, 0xd8, 0x03, 0xf7, 0x98, 0x8a, 0x89, 0xe7, 0x74, 0xec, 0x9e, 0xbb, 0xed, 0xde, 0xbc, - 0x7d, 0x6a, 0x0d, 0x35, 0x82, 0xd7, 0x60, 0x61, 0x9f, 0xf9, 0xf4, 0xca, 0x2b, 0x15, 0xa8, 0x04, - 0xc2, 0x1f, 0x82, 0x7b, 0x7c, 0x1d, 0x51, 0xcf, 0xee, 0xd8, 0xbd, 0x66, 0x7f, 0x79, 0x33, 0xb9, - 0x6b, 0x53, 0x87, 0x54, 0x44, 0x16, 0xe8, 0x3a, 0xa2, 0x18, 0x83, 0xbb, 0x4b, 0x24, 0xf1, 0xdc, - 0x8e, 0xdd, 0x6b, 0x0c, 0xf5, 0xba, 0xfb, 0x93, 0x0d, 0xe8, 0x88, 0x91, 0x28, 0xbe, 0xe0, 0x72, - 0x40, 0x25, 0xf1, 0x89, 0x24, 0xf8, 0x33, 0x80, 0x11, 0x67, 0xe7, 0xa7, 0xb1, 0x24, 0x32, 0x89, - 0x5d, 0xcf, 0x63, 0xef, 0x70, 0x76, 0x7e, 0xa4, 0x08, 0x13, 0xbb, 0x36, 0x4a, 0x01, 0xe5, 0x34, - 0xd4, 0x4e, 0x8b, 0x49, 0x24, 0x90, 0xca, 0x4f, 0xaa, 0xfc, 0x8a, 0x49, 0x68, 0xa4, 0xfb, 0x0d, - 0x54, 0x53, 0x07, 0xca, 0xa2, 0x72, 0xa0, 0xef, 0x6c, 0x0c, 0xf5, 0x1a, 0x7f, 0x01, 0xd5, 0x89, - 0x71, 0xa6, 0x03, 0xd7, 0xfb, 0x5e, 0xea, 0x65, 0xde, 0xb9, 0x89, 0x9b, 0xe9, 0xbb, 0xbf, 0xb8, - 0x50, 0x19, 0xd0, 0x38, 0x26, 0x01, 0xc5, 0x2f, 0xc1, 0x95, 0x79, 0xad, 0x56, 0xd2, 0x18, 0x86, - 0x2e, 0x56, 0x4b, 0xc9, 0x70, 0x0b, 0x1c, 0xc9, 0x67, 0x32, 0x71, 0x24, 0x57, 0x69, 0x9c, 0x0b, - 0x3e, 0x97, 0x86, 0x42, 0xb2, 0x04, 0xdd, 0xf9, 0x04, 0x71, 0x1b, 0x2a, 0x63, 0x1e, 0xe8, 0xee, - 0x2e, 0x14, 0xc8, 0x14, 0xcc, 0xcb, 0x56, 0x7e, 0x58, 0xb6, 0x97, 0x50, 0xa1, 0x4c, 0x8a, 0x90, - 0xc6, 0x5e, 0xa5, 0x53, 0xea, 0xd5, 0xfb, 0x8b, 0x33, 0x3d, 0x4e, 0x43, 0x19, 0x0d, 0x5e, 0x87, - 0xf2, 0x88, 0x4f, 0x26, 0xa1, 0xf4, 0xaa, 0x85, 0x58, 0x06, 0xc3, 0x9f, 0x42, 0xed, 0x82, 0x08, - 0x5f, 0x37, 0xcb, 0x5b, 0x9c, 0x6d, 0xeb, 0x5e, 0x4a, 0xe8, 0x33, 0xf6, 0x30, 0x57, 0xe2, 0x3e, - 0x54, 0x63, 0x53, 0x68, 0xaf, 0xa6, 0x4f, 0xa1, 0xf9, 0x06, 0x98, 0x43, 0x99, 0x4e, 0x19, 0x11, - 0xf4, 0x3b, 0x3a, 0x92, 0x1e, 0x74, 0xec, 0x5e, 0x35, 0x35, 0x92, 0x60, 0xf8, 0x39, 0x40, 0xb2, - 0xda, 0x0b, 0x99, 0xf4, 0xea, 0x05, 0xab, 0x05, 0x5c, 0xd5, 0x6d, 0xc4, 0x99, 0xa4, 0x57, 0xd2, - 0x6b, 0xa8, 0xef, 0xc1, 0x5c, 0x92, 0x82, 0xf8, 0x13, 0xa8, 0x09, 0x1a, 0x47, 0x9c, 0xc5, 0x34, - 0xf6, 0x9a, 0xba, 0x3a, 0x4b, 0x73, 0x5d, 0x4d, 0xbf, 0xd1, 0x4c, 0xd7, 0xfd, 0x16, 0x6a, 0x59, - 0xaa, 0x59, 0xcf, 0xec, 0x07, 0x3d, 0xf3, 0xc0, 0xbd, 0xe4, 0x92, 0xce, 0x8e, 0xa3, 0x42, 0x0a, - 0x25, 0x2e, 0x3d, 0x2c, 0x71, 0xf7, 0x2f, 0x1b, 0x6a, 0xd9, 0x84, 0xe0, 0x55, 0x28, 0xab, 0x33, - 0x22, 0xf6, 0xec, 0x4e, 0xa9, 0xe7, 0x0e, 0xcd, 0x0e, 0xaf, 0x41, 0x75, 0x4c, 0x89, 0x60, 0x8a, - 0x71, 0x34, 0x93, 0xed, 0xf1, 0x0b, 0x58, 0x4a, 0x54, 0xa7, 0x7c, 0x2a, 0x03, 0x1e, 0xb2, 0xc0, - 0x2b, 0x69, 0x49, 0x33, 0x81, 0xbf, 0x32, 0x28, 0x7e, 0x06, 0x8b, 0xe9, 0xa1, 0x53, 0xa6, 0x8a, - 0xe4, 0x6a, 0x59, 0x23, 0x05, 0x0f, 0x55, 0x8d, 0x9e, 0x01, 0x90, 0xa9, 0xe4, 0xa7, 0x63, 0x4a, - 0x2e, 0xa9, 0xfe, 0xfc, 0xd2, 0x5e, 0xd4, 0x14, 0x7e, 0xa0, 0xe0, 0xee, 0xef, 0x36, 0x80, 0x32, - 0xbd, 0x73, 0x41, 0x58, 0x40, 0xf1, 0xc7, 0x66, 0x50, 0x1c, 0x3d, 0x28, 0xab, 0xc5, 0xc1, 0x4f, - 0x14, 0x0f, 0x66, 0xe5, 0x05, 0x54, 0x18, 0xf7, 0xe9, 0x69, 0xe8, 0x9b, 0xa2, 0x34, 0x15, 0x79, - 0xf7, 0xf6, 0x69, 0xf9, 0x90, 0xfb, 0x74, 0x7f, 0x77, 0x58, 0x56, 0xf4, 0xbe, 0x8f, 0xbd, 0xbc, - 0xa5, 0xc9, 0x2b, 0x94, 0x35, 0x73, 0x0d, 0x9c, 0xd0, 0x37, 0x8d, 0x00, 0x73, 0xda, 0xd9, 0xdf, - 0x1d, 0x3a, 0xa1, 0xdf, 0x9d, 0x00, 0xca, 0x2f, 0x3f, 0x0a, 0x59, 0x30, 0xce, 0x4d, 0xda, 0xff, - 0xc5, 0xa4, 0xf3, 0x3e, 0x93, 0xdd, 0x3f, 0x6c, 0x68, 0xe4, 0x71, 0x4e, 0xfa, 0x78, 0x1b, 0x40, - 0x0a, 0xc2, 0xe2, 0x50, 0x86, 0x9c, 0x99, 0x1b, 0xd7, 0x1f, 0xb9, 0x31, 0xd3, 0xa4, 0x1f, 0x73, - 0x7e, 0x0a, 0x7f, 0x0e, 0x95, 0x91, 0x56, 0x25, 0x1d, 0x2f, 0x3c, 0x62, 0xf3, 0xa9, 0xa5, 0x33, - 0x6d, 0xe4, 0xc5, 0x9a, 0x95, 0x66, 0x6a, 0xb6, 0xb1, 0x07, 0xb5, 0xec, 0xa5, 0xc7, 0x4b, 0x50, - 0xd7, 0x9b, 0x43, 0x2e, 0x26, 0x64, 0x8c, 0x2c, 0xbc, 0x02, 0x4b, 0x1a, 0xc8, 0xe3, 0x23, 0x1b, - 0x3f, 0x81, 0xe5, 0x39, 0xf0, 0xa4, 0x8f, 0x9c, 0x8d, 0x3f, 0x4b, 0x50, 0x2f, 0x3c, 0x84, 0x18, - 0xa0, 0x3c, 0x88, 0x83, 0xbd, 0x69, 0x84, 0x2c, 0x5c, 0x87, 0xca, 0x20, 0x0e, 0xb6, 0x29, 0x91, - 0xc8, 0x36, 0x9b, 0xd7, 0x82, 0x47, 0xc8, 0x31, 0xaa, 0xad, 0x28, 0x42, 0x25, 0xdc, 0x04, 0x48, - 0xd6, 0x43, 0x1a, 0x47, 0xc8, 0x35, 0xc2, 0x13, 0x2e, 0x29, 0x5a, 0x50, 0xde, 0xcc, 0x46, 0xb3, - 0x65, 0xc3, 0xaa, 0xd7, 0x03, 0x55, 0x30, 0x82, 0x86, 0xba, 0x8c, 0x12, 0x21, 0xcf, 0xd4, 0x2d, - 0x55, 0xdc, 0x02, 0x54, 0x44, 0xf4, 0xa1, 0x1a, 0xc6, 0xd0, 0x1c, 0xc4, 0xc1, 0x1b, 0x26, 0x28, - 0x19, 0x5d, 0x90, 0xb3, 0x31, 0x45, 0x80, 0x97, 0x61, 0xd1, 0x04, 0x52, 0x13, 0x37, 0x8d, 0x51, - 0xdd, 0xc8, 0x76, 0x2e, 0xe8, 0xe8, 0xfb, 0xaf, 0xa7, 0x5c, 0x4c, 0x27, 0xa8, 0xa1, 0xd2, 0x1e, - 0xc4, 0x81, 0x6e, 0xd0, 0x39, 0x15, 0x07, 0x94, 0xf8, 0x54, 0xa0, 0x45, 0x73, 0xfa, 0x38, 0x9c, - 0x50, 0x3e, 0x95, 0x87, 0xfc, 0x07, 0xd4, 0x34, 0x66, 0x86, 0x94, 0xf8, 0xfa, 0x0f, 0x8b, 0x96, - 0x8c, 0x99, 0x0c, 0xd1, 0x66, 0x90, 0xc9, 0xf7, 0xb5, 0xa0, 0x3a, 0xc5, 0x65, 0x73, 0xab, 0xd9, - 0x6b, 0x0d, 0x36, 0x27, 0x8f, 0x24, 0x17, 0x24, 0xa0, 0x5b, 0x51, 0x44, 0x99, 0x8f, 0x56, 0xb0, - 0x07, 0xad, 0x79, 0x54, 0xeb, 0x5b, 0xaa, 0x63, 0x33, 0xcc, 0xf8, 0x1a, 0x3d, 0xc1, 0xff, 0x83, - 0x95, 0x39, 0x50, 0xab, 0x57, 0x37, 0x7e, 0xb6, 0xa1, 0xf5, 0xd8, 0xc7, 0x87, 0xd7, 0xc1, 0x7b, - 0x0c, 0xdf, 0x9a, 0x4a, 0x8e, 0x2c, 0xfc, 0x01, 0xfc, 0xff, 0x31, 0xf6, 0x4b, 0x1e, 0x32, 0xb9, - 0x3f, 0x89, 0xc6, 0xe1, 0x28, 0x54, 0x8d, 0x7e, 0x9f, 0xec, 0xd5, 0x95, 0x91, 0x39, 0x1b, 0xd7, - 0xd0, 0x9c, 0x1d, 0x39, 0x55, 0xea, 0x1c, 0xd9, 0xf2, 0x7d, 0x35, 0x5c, 0xc8, 0x52, 0x59, 0xe7, - 0xf0, 0x90, 0x4e, 0xf8, 0x25, 0xd5, 0x8c, 0x3d, 0xcb, 0xbc, 0x89, 0x7c, 0x22, 0x13, 0xc6, 0x99, - 0x4d, 0x64, 0xcb, 0xf7, 0x0f, 0x92, 0x97, 0x4d, 0xb3, 0xa5, 0xed, 0xe7, 0x37, 0xef, 0xda, 0xd6, - 0xed, 0xbb, 0xb6, 0x75, 0x73, 0xd7, 0xb6, 0x6f, 0xef, 0xda, 0xf6, 0x3f, 0x77, 0x6d, 0xfb, 0xd7, - 0xfb, 0xb6, 0xf5, 0xdb, 0x7d, 0xdb, 0xba, 0xbd, 0x6f, 0x5b, 0x7f, 0xdf, 0xb7, 0xad, 0x7f, 0x03, - 0x00, 0x00, 0xff, 0xff, 0x17, 0x42, 0x8c, 0x08, 0x8b, 0x09, 0x00, 0x00, + // 1091 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcb, 0x6e, 0x23, 0x45, + 0x14, 0xed, 0x6e, 0x77, 0xfc, 0xb8, 0x76, 0x9c, 0x4a, 0xc5, 0x13, 0x5a, 0x51, 0xe4, 0x31, 0x9e, + 0x41, 0x63, 0x05, 0x4d, 0x40, 0x46, 0x42, 0x88, 0x5d, 0x1e, 0x23, 0x25, 0x28, 0x0e, 0x83, 0x93, + 0xc9, 0x02, 0x09, 0x45, 0x15, 0x77, 0xa5, 0xd3, 0x60, 0x57, 0xb5, 0xaa, 0xcb, 0x21, 0xd9, 0x20, + 0xc4, 0x17, 0xb0, 0x64, 0xc3, 0x96, 0x3d, 0x7c, 0x45, 0x96, 0x59, 0xb2, 0x1a, 0x31, 0xc9, 0x1f, + 0xf0, 0x05, 0xa8, 0xaa, 0xab, 0x1f, 0x76, 0xa2, 0x59, 0xb0, 0xab, 0x3a, 0xf7, 0xd4, 0xbd, 0xe7, + 0x9e, 0xdb, 0x55, 0x0d, 0x20, 0xc8, 0xb9, 0xdc, 0x8c, 0x04, 0x97, 0x1c, 0x97, 0xd5, 0x3a, 0x3a, + 0x5b, 0x6b, 0x05, 0x3c, 0xe0, 0x1a, 0xfa, 0x44, 0xad, 0x92, 0x68, 0xf7, 0x27, 0x58, 0x78, 0xc5, + 0xa4, 0xb8, 0xc6, 0x1e, 0xb8, 0xc7, 0x54, 0x4c, 0x3c, 0xa7, 0x63, 0xf7, 0xdc, 0x6d, 0xf7, 0xe6, + 0xed, 0x53, 0x6b, 0xa8, 0x11, 0xbc, 0x06, 0x0b, 0xfb, 0xcc, 0xa7, 0x57, 0x5e, 0xa9, 0x10, 0x4a, + 0x20, 0xfc, 0x31, 0xb8, 0xc7, 0xd7, 0x11, 0xf5, 0xec, 0x8e, 0xdd, 0x6b, 0xf6, 0x97, 0x37, 0x93, + 0x5a, 0x9b, 0x3a, 0xa5, 0x0a, 0x64, 0x89, 0xae, 0x23, 0x8a, 0x31, 0xb8, 0xbb, 0x44, 0x12, 0xcf, + 0xed, 0xd8, 0xbd, 0xc6, 0x50, 0xaf, 0xbb, 0x3f, 0xdb, 0x80, 0x8e, 0x18, 0x89, 0xe2, 0x0b, 0x2e, + 0x07, 0x54, 0x12, 0x9f, 0x48, 0x82, 0x3f, 0x07, 0x18, 0x71, 0x76, 0x7e, 0x1a, 0x4b, 0x22, 0x93, + 0xdc, 0xf5, 0x3c, 0xf7, 0x0e, 0x67, 0xe7, 0x47, 0x2a, 0x60, 0x72, 0xd7, 0x46, 0x29, 0xa0, 0x94, + 0x86, 0x5a, 0x69, 0xb1, 0x89, 0x04, 0x52, 0xfd, 0x49, 0xd5, 0x5f, 0xb1, 0x09, 0x8d, 0x74, 0xbf, + 0x85, 0x6a, 0xaa, 0x40, 0x49, 0x54, 0x0a, 0x74, 0xcd, 0xc6, 0x50, 0xaf, 0xf1, 0x97, 0x50, 0x9d, + 0x18, 0x65, 0x3a, 0x71, 0xbd, 0xef, 0xa5, 0x5a, 0xe6, 0x95, 0x9b, 0xbc, 0x19, 0xbf, 0xfb, 0x6f, + 0x09, 0x2a, 0x03, 0x1a, 0xc7, 0x24, 0xa0, 0xf8, 0x25, 0xb8, 0x32, 0xf7, 0x6a, 0x25, 0xcd, 0x61, + 0xc2, 0x45, 0xb7, 0x14, 0x0d, 0xb7, 0xc0, 0x91, 0x7c, 0xa6, 0x13, 0x47, 0x72, 0xd5, 0xc6, 0xb9, + 0xe0, 0x73, 0x6d, 0x28, 0x24, 0x6b, 0xd0, 0x9d, 0x6f, 0x10, 0xb7, 0xa1, 0x32, 0xe6, 0x81, 0x9e, + 0xee, 0x42, 0x21, 0x98, 0x82, 0xb9, 0x6d, 0xe5, 0x87, 0xb6, 0xbd, 0x84, 0x0a, 0x65, 0x52, 0x84, + 0x34, 0xf6, 0x2a, 0x9d, 0x52, 0xaf, 0xde, 0x5f, 0x9c, 0x99, 0x71, 0x9a, 0xca, 0x70, 0xf0, 0x3a, + 0x94, 0x47, 0x7c, 0x32, 0x09, 0xa5, 0x57, 0x2d, 0xe4, 0x32, 0x98, 0x92, 0x78, 0xc9, 0x25, 0xf5, + 0x16, 0x8b, 0x12, 0x15, 0x82, 0xfb, 0x50, 0x8d, 0x8d, 0x97, 0x5e, 0x4d, 0x7b, 0x8c, 0xe6, 0x3d, + 0xd6, 0x7c, 0x7b, 0x98, 0xf1, 0x54, 0x2d, 0x41, 0xbf, 0xa7, 0x23, 0xe9, 0x41, 0xc7, 0xee, 0x55, + 0xd3, 0x5a, 0x09, 0x86, 0x9f, 0x03, 0x24, 0xab, 0xbd, 0x90, 0x49, 0xaf, 0x5e, 0xa8, 0x58, 0xc0, + 0x95, 0x35, 0x23, 0xce, 0x24, 0xbd, 0x92, 0x5e, 0x43, 0x8d, 0xdc, 0x14, 0x49, 0x41, 0xfc, 0x19, + 0xd4, 0x04, 0x8d, 0x23, 0xce, 0x62, 0x1a, 0x7b, 0x4d, 0x6d, 0xc0, 0xd2, 0xdc, 0xe0, 0xd2, 0xcf, + 0x30, 0xe3, 0x75, 0xbf, 0x83, 0xda, 0x1e, 0x11, 0x7e, 0xf2, 0x4d, 0xa6, 0x63, 0xb1, 0x1f, 0x8c, + 0x25, 0x75, 0xc3, 0x79, 0xe0, 0x46, 0xee, 0x62, 0xe9, 0xa1, 0x8b, 0xdd, 0xbf, 0x6c, 0xa8, 0x65, + 0x97, 0x00, 0xaf, 0x42, 0x59, 0x9d, 0x11, 0xb1, 0x67, 0x77, 0x4a, 0x3d, 0x77, 0x68, 0x76, 0x78, + 0x0d, 0xaa, 0x63, 0x4a, 0x04, 0x53, 0x11, 0x47, 0x47, 0xb2, 0x3d, 0x7e, 0x01, 0x4b, 0x09, 0xeb, + 0x94, 0x4f, 0x65, 0xc0, 0x43, 0x16, 0x78, 0x25, 0x4d, 0x69, 0x26, 0xf0, 0xd7, 0x06, 0xc5, 0xcf, + 0x60, 0x31, 0x3d, 0x74, 0xca, 0x94, 0x49, 0xae, 0xa6, 0x35, 0x52, 0xf0, 0x50, 0x79, 0xf4, 0x0c, + 0x80, 0x4c, 0x25, 0x3f, 0x1d, 0x53, 0x72, 0x49, 0xf5, 0x17, 0x96, 0xce, 0xa2, 0xa6, 0xf0, 0x03, + 0x05, 0x77, 0x7f, 0xb7, 0x01, 0x94, 0xe8, 0x9d, 0x0b, 0xc2, 0x02, 0x8a, 0x3f, 0x35, 0x77, 0xc1, + 0xd1, 0x77, 0x61, 0xb5, 0x78, 0xb7, 0x13, 0xc6, 0x83, 0xeb, 0xf0, 0x02, 0x2a, 0x8c, 0xfb, 0xf4, + 0x34, 0xf4, 0x8d, 0x29, 0x4d, 0x15, 0xbc, 0x7b, 0xfb, 0xb4, 0x7c, 0xc8, 0x7d, 0xba, 0xbf, 0x3b, + 0x2c, 0xab, 0xf0, 0xbe, 0x8f, 0xbd, 0x7c, 0xa4, 0xc9, 0x43, 0x93, 0x0d, 0x73, 0x0d, 0x9c, 0xd0, + 0x37, 0x83, 0x00, 0x73, 0xda, 0xd9, 0xdf, 0x1d, 0x3a, 0xa1, 0xdf, 0x9d, 0x00, 0xca, 0x8b, 0x1f, + 0x85, 0x2c, 0x18, 0xe7, 0x22, 0xed, 0xff, 0x23, 0xd2, 0x79, 0x9f, 0xc8, 0xee, 0x1f, 0x36, 0x34, + 0xf2, 0x3c, 0x27, 0x7d, 0xbc, 0x0d, 0x20, 0x05, 0x61, 0x71, 0x28, 0x43, 0xce, 0x4c, 0xc5, 0xf5, + 0x47, 0x2a, 0x66, 0x9c, 0xf4, 0x63, 0xce, 0x4f, 0xe1, 0x2f, 0xa0, 0x32, 0xd2, 0xac, 0x64, 0xe2, + 0x85, 0x77, 0x6a, 0xbe, 0xb5, 0xf4, 0xda, 0x1a, 0x7a, 0xd1, 0xb3, 0xd2, 0x8c, 0x67, 0x1b, 0x7b, + 0x50, 0xcb, 0x1e, 0x73, 0xbc, 0x04, 0x75, 0xbd, 0x39, 0xe4, 0x62, 0x42, 0xc6, 0xc8, 0xc2, 0x2b, + 0xb0, 0xa4, 0x81, 0x3c, 0x3f, 0xb2, 0xf1, 0x13, 0x58, 0x9e, 0x03, 0x4f, 0xfa, 0xc8, 0xd9, 0xf8, + 0xb3, 0x04, 0xf5, 0xc2, 0x5b, 0x87, 0x01, 0xca, 0x83, 0x38, 0xd8, 0x9b, 0x46, 0xc8, 0xc2, 0x75, + 0xa8, 0x0c, 0xe2, 0x60, 0x9b, 0x12, 0x89, 0x6c, 0xb3, 0x79, 0x2d, 0x78, 0x84, 0x1c, 0xc3, 0xda, + 0x8a, 0x22, 0x54, 0xc2, 0x4d, 0x80, 0x64, 0x3d, 0xa4, 0x71, 0x84, 0x5c, 0x43, 0x3c, 0xe1, 0x92, + 0xa2, 0x05, 0xa5, 0xcd, 0x6c, 0x74, 0xb4, 0x6c, 0xa2, 0xea, 0xf5, 0x40, 0x15, 0x8c, 0xa0, 0xa1, + 0x8a, 0x51, 0x22, 0xe4, 0x99, 0xaa, 0x52, 0xc5, 0x2d, 0x40, 0x45, 0x44, 0x1f, 0xaa, 0x61, 0x0c, + 0xcd, 0x41, 0x1c, 0xbc, 0x61, 0x82, 0x92, 0xd1, 0x05, 0x39, 0x1b, 0x53, 0x04, 0x78, 0x19, 0x16, + 0x4d, 0x22, 0x75, 0xe3, 0xa6, 0x31, 0xaa, 0x1b, 0xda, 0xce, 0x05, 0x1d, 0xfd, 0xf0, 0xcd, 0x94, + 0x8b, 0xe9, 0x04, 0x35, 0x54, 0xdb, 0x83, 0x38, 0xd0, 0x03, 0x3a, 0xa7, 0xe2, 0x80, 0x12, 0x9f, + 0x0a, 0xb4, 0x68, 0x4e, 0x1f, 0x87, 0x13, 0xca, 0xa7, 0xf2, 0x90, 0xff, 0x88, 0x9a, 0x46, 0xcc, + 0x90, 0x12, 0x5f, 0xff, 0x44, 0xd1, 0x92, 0x11, 0x93, 0x21, 0x5a, 0x0c, 0x32, 0xfd, 0xbe, 0x16, + 0x54, 0xb7, 0xb8, 0x6c, 0xaa, 0x9a, 0xbd, 0xe6, 0x60, 0x73, 0xf2, 0x48, 0x72, 0x41, 0x02, 0xba, + 0x15, 0x45, 0x94, 0xf9, 0x68, 0x05, 0x7b, 0xd0, 0x9a, 0x47, 0x35, 0xbf, 0xa5, 0x26, 0x36, 0x13, + 0x19, 0x5f, 0xa3, 0x27, 0xf8, 0x03, 0x58, 0x99, 0x03, 0x35, 0x7b, 0x75, 0xe3, 0x17, 0x1b, 0x5a, + 0x8f, 0x7d, 0x7c, 0x78, 0x1d, 0xbc, 0xc7, 0xf0, 0xad, 0xa9, 0xe4, 0xc8, 0xc2, 0x1f, 0xc1, 0x87, + 0x8f, 0x45, 0xbf, 0xe2, 0x21, 0x93, 0xfb, 0x93, 0x68, 0x1c, 0x8e, 0x42, 0x35, 0xe8, 0xf7, 0xd1, + 0x5e, 0x5d, 0x19, 0x9a, 0xb3, 0x71, 0x0d, 0xcd, 0xd9, 0x2b, 0xa7, 0xac, 0xce, 0x91, 0x2d, 0xdf, + 0x57, 0x97, 0x0b, 0x59, 0xaa, 0xeb, 0x1c, 0x1e, 0xd2, 0x09, 0xbf, 0xa4, 0x3a, 0x62, 0xcf, 0x46, + 0xde, 0x44, 0x3e, 0x91, 0x49, 0xc4, 0x99, 0x6d, 0x64, 0xcb, 0xf7, 0x0f, 0x92, 0x97, 0x4d, 0x47, + 0x4b, 0xdb, 0xcf, 0x6f, 0xde, 0xb5, 0xad, 0xdb, 0x77, 0x6d, 0xeb, 0xe6, 0xae, 0x6d, 0xdf, 0xde, + 0xb5, 0xed, 0x7f, 0xee, 0xda, 0xf6, 0xaf, 0xf7, 0x6d, 0xeb, 0xb7, 0xfb, 0xb6, 0x75, 0x7b, 0xdf, + 0xb6, 0xfe, 0xbe, 0x6f, 0x5b, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x8e, 0x4c, 0xc0, 0x6e, + 0x09, 0x00, 0x00, } func (m *Entry) Marshal() (dAtA []byte, err error) { @@ -945,18 +950,9 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x72 } } - if m.HardState != nil { - { - size, err := m.HardState.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRaft(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x6a - } + i = encodeVarintRaft(dAtA, i, uint64(m.Vote)) + i-- + dAtA[i] = 0x68 if m.Context != nil { i -= len(m.Context) copy(dAtA[i:], m.Context) @@ -1315,10 +1311,7 @@ func (m *Message) Size() (n int) { l = len(m.Context) n += 1 + l + sovRaft(uint64(l)) } - if m.HardState != nil { - l = m.HardState.Size() - n += 1 + l + sovRaft(uint64(l)) - } + n += 1 + sovRaft(uint64(m.Vote)) if len(m.Responses) > 0 { for _, e := range m.Responses { l = e.Size() @@ -2108,10 +2101,10 @@ func (m *Message) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 13: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field HardState", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Vote", wireType) } - var msglen int + m.Vote = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRaft @@ -2121,28 +2114,11 @@ func (m *Message) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Vote |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthRaft - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthRaft - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.HardState == nil { - m.HardState = &HardState{} - } - if err := m.HardState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 14: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType) diff --git a/raftpb/raft.proto b/raftpb/raft.proto index 35c763f1..a70f736a 100644 --- a/raftpb/raft.proto +++ b/raftpb/raft.proto @@ -13,14 +13,12 @@ option (gogoproto.goproto_unrecognized_all) = false; option (gogoproto.goproto_sizecache_all) = false; enum EntryType { - EntryNormal = 0; EntryConfChange = 1; // corresponds to pb.ConfChange EntryConfChangeV2 = 2; // corresponds to pb.ConfChangeV2 } message Entry { - optional uint64 Term = 2 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations optional uint64 Index = 3 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations optional EntryType Type = 1 [(gogoproto.nullable) = false]; @@ -81,14 +79,19 @@ message Message { // entries from its leader as it already has an entry with term 5 at index 100. // (type=MsgStorageAppendResp,index=100,logTerm=5) means the local node wrote // entries up to index=100 in stable storage, and the term of the entry at index - // 100 was 5. + // 100 was 5. This doesn't always mean that the corresponding MsgStorageAppend + // message was the one that carried these entries, just that those entries were + // stable at the time of processing the corresponding MsgStorageAppend. optional uint64 logTerm = 5 [(gogoproto.nullable) = false]; optional uint64 index = 6 [(gogoproto.nullable) = false]; repeated Entry entries = 7 [(gogoproto.nullable) = false]; optional uint64 commit = 8 [(gogoproto.nullable) = false]; - // hardState can be non-nil for MsgStorageAppend messages and is nil for all - // other message types. - optional HardState hardState = 13 [(gogoproto.nullable) = true]; + // (type=MsgStorageAppend,vote=5,term=10) means the local node is voting for + // peer 5 in term 10. For MsgStorageAppends, the term, vote, and commit fields + // will either all be set (to facilitate the construction of a HardState) if + // any of the fields have changed or will all be unset if none of the fields + // have changed. + optional uint64 vote = 13 [(gogoproto.nullable) = false]; // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a // non-nil, empty value for the snapshot field of non-MsgSnap messages. Code @@ -204,7 +207,6 @@ message ConfChangeSingle { // // [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf message ConfChangeV2 { - optional ConfChangeTransition transition = 1 [(gogoproto.nullable) = false]; repeated ConfChangeSingle changes = 2 [(gogoproto.nullable) = false]; optional bytes context = 3; diff --git a/raftpb/raft_test.go b/raftpb/raft_test.go index 7f5be88b..704ff94f 100644 --- a/raftpb/raft_test.go +++ b/raftpb/raft_test.go @@ -45,7 +45,7 @@ func TestProtoMemorySizes(t *testing.T) { assert(unsafe.Sizeof(s), if64Bit(144, 80), "Snapshot") var m Message - assert(unsafe.Sizeof(m), if64Bit(160, 108), "Message") + assert(unsafe.Sizeof(m), if64Bit(160, 112), "Message") var hs HardState assert(unsafe.Sizeof(hs), 24, "HardState") diff --git a/rafttest/interaction_env_handler_process_append_thread.go b/rafttest/interaction_env_handler_process_append_thread.go index 3f74988a..38c16f25 100644 --- a/rafttest/interaction_env_handler_process_append_thread.go +++ b/rafttest/interaction_env_handler_process_append_thread.go @@ -56,9 +56,10 @@ func (env *InteractionEnv) ProcessAppendThread(idx int) error { m.Responses = nil env.Output.WriteString("Processing:\n") env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") - var st raftpb.HardState - if m.HardState != nil { - st = *m.HardState + st := raftpb.HardState{ + Term: m.Term, + Vote: m.Vote, + Commit: m.Commit, } var snap raftpb.Snapshot if m.Snapshot != nil { diff --git a/rawnode.go b/rawnode.go index 1e88a801..b4618286 100644 --- a/rawnode.go +++ b/rawnode.go @@ -222,19 +222,19 @@ func newStorageAppendMsg(r *raft, rd Ready) pb.Message { Type: pb.MsgStorageAppend, To: LocalAppendThread, From: r.id, - Term: r.Term, Entries: rd.Entries, } if !IsEmptyHardState(rd.HardState) { - // TODO(nvanbenschoten): we could avoid this heap allocation by - // replacing the pb.Message.HardState field with a Vote uint64 field. We - // would then need to teach apps to construct a HardState from these - // three fields, or supply a function/method that does so. - // m.Term = rd.Term - // m.Vote = rd.Vote - // m.Commit = rd.Commit - hs := rd.HardState - m.HardState = &hs + // If the Ready includes a HardState update, assign each of its fields + // to the corresponding fields in the Message. This allows clients to + // reconstruct the HardState and save it to stable storage. + // + // If the Ready does not include a HardState update, make sure to not + // assign a value to any of the fields so that a HardState reconstructed + // from them will be empty (return true from raft.IsEmptyHardState). + m.Term = rd.Term + m.Vote = rd.Vote + m.Commit = rd.Commit } if !IsEmptySnap(rd.Snapshot) { snap := rd.Snapshot @@ -340,8 +340,8 @@ func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { // MsgStorageAppend that contained the last entry in the unstable slice carried // an earlier term and was dropped. // - // A MsgStorageAppend with a new HardState is emitted on each term change. This - // is the same condition that causes MsgStorageAppendResp messages with earlier + // A MsgStorageAppend with a new term is emitted on each term change. This is + // the same condition that causes MsgStorageAppendResp messages with earlier // terms to be ignored. As a result, we are guaranteed that, assuming a bounded // number of term changes, there will eventually be a MsgStorageAppendResp // message that is not ignored. This means that entries in the unstable log diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt index f6a7ef8f..52f37092 100644 --- a/testdata/async_storage_writes.txt +++ b/testdata/async_storage_writes.txt @@ -29,7 +29,7 @@ stabilize Messages: 1->2 MsgVote Term:1 Log:1/10 1->3 MsgVote Term:1 Log:1/10 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[1->1 MsgVoteResp Term:1 Log:0/0] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[1->1 MsgVoteResp Term:1 Log:0/0] > 2 receiving messages 1->2 MsgVote Term:1 Log:1/10 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] @@ -42,31 +42,31 @@ stabilize INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses: 1->1 MsgVoteResp Term:1 Log:0/0 > 2 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:10 Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[2->1 MsgVoteResp Term:1 Log:0/0] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[2->1 MsgVoteResp Term:1 Log:0/0] > 3 handling Ready Ready MustSync=true: HardState Term:1 Vote:1 Commit:10 Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 Responses:[3->1 MsgVoteResp Term:1 Log:0/0] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[3->1 MsgVoteResp Term:1 Log:0/0] > 1 receiving messages 1->1 MsgVoteResp Term:1 Log:0/0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses: 2->1 MsgVoteResp Term:1 Log:0/0 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:10 + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses: 3->1 MsgVoteResp Term:1 Log:0/0 > 1 receiving messages @@ -83,14 +83,14 @@ stabilize Messages: 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:1 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[1->1 MsgAppResp Term:1 Log:0/11, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11] + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[1->1 MsgAppResp Term:1 Log:0/11, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11] > 2 receiving messages 1->2 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] > 3 receiving messages 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 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 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 @@ -100,26 +100,26 @@ stabilize Entries: 1/11 EntryNormal "" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[2->1 MsgAppResp Term:1 Log:0/11, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11] + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[2->1 MsgAppResp Term:1 Log:0/11, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11] > 3 handling Ready Ready MustSync=true: Lead:1 State:StateFollower Entries: 1/11 EntryNormal "" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[3->1 MsgAppResp Term:1 Log:0/11, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11] + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[3->1 MsgAppResp Term:1 Log:0/11, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11] > 1 receiving messages 1->1 MsgAppResp Term:1 Log:0/11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: 2->1 MsgAppResp Term:1 Log:0/11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/11 EntryNormal ""] + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses: 3->1 MsgAppResp Term:1 Log:0/11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 @@ -138,7 +138,7 @@ stabilize Messages: 1->2 MsgApp Term:1 Log:1/11 Commit:11 1->3 MsgApp Term:1 Log:1/11 Commit:11 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""]] > 2 receiving messages 1->2 MsgApp Term:1 Log:1/11 Commit:11 @@ -146,7 +146,7 @@ stabilize 1->3 MsgApp Term:1 Log:1/11 Commit:11 > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses: > 1 processing apply thread Processing: @@ -159,7 +159,7 @@ stabilize CommittedEntries: 1/11 EntryNormal "" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[2->1 MsgAppResp Term:1 Log:0/11] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses:[2->1 MsgAppResp Term:1 Log:0/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 ""]] > 3 handling Ready Ready MustSync=false: @@ -167,18 +167,18 @@ stabilize CommittedEntries: 1/11 EntryNormal "" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 Responses:[3->1 MsgAppResp Term:1 Log:0/11] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses:[3->1 MsgAppResp Term:1 Log:0/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 ""]] > 1 receiving messages ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses: 2->1 MsgAppResp Term:1 Log:0/11 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:11 + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses: 3->1 MsgAppResp Term:1 Log:0/11 > 2 processing apply thread @@ -212,7 +212,7 @@ process-ready 1 2 3 Messages: 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:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[1->1 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[1->1 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] > 2 handling Ready > 3 handling Ready @@ -232,13 +232,13 @@ process-ready 1 2 3 Entries: 1/12 EntryNormal "prop_1" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[2->1 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[2->1 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] > 3 handling Ready Ready MustSync=true: Entries: 1/12 EntryNormal "prop_1" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[3->1 MsgAppResp Term:1 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12] + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[3->1 MsgAppResp Term:1 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12] propose 1 prop_2 ---- @@ -253,7 +253,7 @@ process-ready 1 2 3 Messages: 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:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[1->1 MsgAppResp Term:1 Log:0/13, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13] + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[1->1 MsgAppResp Term:1 Log:0/13, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13] > 2 handling Ready > 3 handling Ready @@ -273,31 +273,31 @@ process-ready 1 2 3 Entries: 1/13 EntryNormal "prop_2" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[2->1 MsgAppResp Term:1 Log:0/13, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13] + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[2->1 MsgAppResp Term:1 Log:0/13, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13] > 3 handling Ready Ready MustSync=true: Entries: 1/13 EntryNormal "prop_2" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[3->1 MsgAppResp Term:1 Log:0/13, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13] + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[3->1 MsgAppResp Term:1 Log:0/13, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13] process-append-thread 1 2 3 ---- > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 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 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: 2->1 MsgAppResp Term:1 Log:0/12 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses: 3->1 MsgAppResp Term:1 Log:0/12 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 @@ -329,7 +329,7 @@ process-ready 1 2 3 1->3 MsgApp Term:1 Log:1/13 Commit:12 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 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[1->1 MsgAppResp Term:1 Log:0/14, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14] + 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, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14] 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] > 2 handling Ready @@ -355,7 +355,7 @@ process-ready 1 2 3 CommittedEntries: 1/12 EntryNormal "prop_1" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[2->1 MsgAppResp Term:1 Log:0/13, 2->1 MsgAppResp Term:1 Log:0/14, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14] + 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, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] > 3 handling Ready Ready MustSync=true: @@ -365,26 +365,26 @@ process-ready 1 2 3 CommittedEntries: 1/12 EntryNormal "prop_1" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 Responses:[3->1 MsgAppResp Term:1 Log:0/13, 3->1 MsgAppResp Term:1 Log:0/14, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14] + 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, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] process-append-thread 1 2 3 ---- > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 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 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: 2->1 MsgAppResp Term:1 Log:0/13 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses: 3->1 MsgAppResp Term:1 Log:0/13 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 @@ -416,7 +416,7 @@ process-ready 1 2 3 1->3 MsgApp Term:1 Log:1/14 Commit:13 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 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[1->1 MsgAppResp Term:1 Log:0/15, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] + 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, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] > 2 handling Ready @@ -442,7 +442,7 @@ process-ready 1 2 3 CommittedEntries: 1/13 EntryNormal "prop_2" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[2->1 MsgAppResp Term:1 Log:0/14, 2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] + 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, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] > 3 handling Ready Ready MustSync=true: @@ -452,27 +452,27 @@ process-ready 1 2 3 CommittedEntries: 1/13 EntryNormal "prop_2" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 Responses:[3->1 MsgAppResp Term:1 Log:0/14, 3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] + 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, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] process-append-thread 1 2 3 ---- > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 + 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 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 + 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 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] HardState: Term:1 Vote:1 Commit:12 + 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 @@ -520,7 +520,7 @@ process-ready 1 2 3 Messages: 1->2 MsgApp Term:1 Log:1/15 Commit:14 1->3 MsgApp Term:1 Log:1/15 Commit:14 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] > 2 handling Ready @@ -542,7 +542,7 @@ process-ready 1 2 3 CommittedEntries: 1/14 EntryNormal "prop_3" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] > 3 handling Ready Ready MustSync=false: @@ -550,27 +550,27 @@ process-ready 1 2 3 CommittedEntries: 1/14 EntryNormal "prop_3" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] process-append-thread 1 2 3 ---- > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 + 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 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 + 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 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] HardState: Term:1 Vote:1 Commit:13 + 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 @@ -618,7 +618,7 @@ process-ready 1 2 3 Messages: 1->2 MsgApp Term:1 Log:1/15 Commit:15 1->3 MsgApp Term:1 Log:1/15 Commit:15 - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"]] > 2 handling Ready @@ -640,7 +640,7 @@ process-ready 1 2 3 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[2->1 MsgAppResp Term:1 Log:0/15] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses:[2->1 MsgAppResp Term:1 Log:0/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"]] > 3 handling Ready Ready MustSync=false: @@ -648,20 +648,20 @@ process-ready 1 2 3 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 Responses:[3->1 MsgAppResp Term:1 Log:0/15] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses:[3->1 MsgAppResp Term:1 Log:0/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"]] process-append-thread 2 3 ---- > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses: 2->1 MsgAppResp Term:1 Log:0/15 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses: 3->1 MsgAppResp Term:1 Log:0/15 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 @@ -709,12 +709,12 @@ process-append-thread 2 3 ---- > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses: 2->1 MsgAppResp Term:1 Log:0/15 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses: 3->1 MsgAppResp Term:1 Log:0/15 @@ -757,11 +757,11 @@ stabilize ---- > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:14 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses: AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 Processing: - 1->AppendThread MsgStorageAppend Term:1 Log:0/0 HardState: Term:1 Vote:1 Commit:15 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses: > 1 receiving messages AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 diff --git a/testdata/async_storage_writes_append_aba_race.txt b/testdata/async_storage_writes_append_aba_race.txt index 6593a46e..b1e82f12 100644 --- a/testdata/async_storage_writes_append_aba_race.txt +++ b/testdata/async_storage_writes_append_aba_race.txt @@ -42,7 +42,7 @@ Messages: 2->5 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] 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:1 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[2->2 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] +2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[2->2 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] deliver-msgs 1 drop=(3,4,5,6,7) ---- @@ -61,7 +61,7 @@ Ready MustSync=true: Entries: 1/12 EntryNormal "init_prop" Messages: -1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[1->2 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[1->2 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] # Step 4: node 3 becomes the leader after getting a vote from nodes 4, 5, and 6. @@ -88,7 +88,7 @@ Messages: 3->5 MsgVote Term:2 Log:1/11 3->6 MsgVote Term:2 Log:1/11 3->7 MsgVote Term:2 Log:1/11 -3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[3->3 MsgVoteResp Term:2 Log:0/0] +3->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[3->3 MsgVoteResp Term:2 Log:0/0] deliver-msgs 4 5 6 ---- @@ -112,40 +112,40 @@ process-ready 4 5 6 Lead:0 State:StateFollower HardState Term:2 Vote:3 Commit:11 Messages: - 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[4->3 MsgVoteResp Term:2 Log:0/0] + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[4->3 MsgVoteResp Term:2 Log:0/0] > 5 handling Ready Ready MustSync=true: Lead:0 State:StateFollower HardState Term:2 Vote:3 Commit:11 Messages: - 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[5->3 MsgVoteResp Term:2 Log:0/0] + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[5->3 MsgVoteResp Term:2 Log:0/0] > 6 handling Ready Ready MustSync=true: Lead:0 State:StateFollower HardState Term:2 Vote:3 Commit:11 Messages: - 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 Responses:[6->3 MsgVoteResp Term:2 Log:0/0] + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[6->3 MsgVoteResp Term:2 Log:0/0] process-append-thread 3 4 5 6 ---- > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + 3->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses: 3->3 MsgVoteResp Term:2 Log:0/0 > 4 processing append thread Processing: - 4->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses: 4->3 MsgVoteResp Term:2 Log:0/0 > 5 processing append thread Processing: - 5->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses: 5->3 MsgVoteResp Term:2 Log:0/0 > 6 processing append thread Processing: - 6->AppendThread MsgStorageAppend Term:2 Log:0/0 HardState: Term:2 Vote:3 Commit:11 + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses: 6->3 MsgVoteResp Term:2 Log:0/0 @@ -183,7 +183,7 @@ Messages: 3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] 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:2 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[3->3 MsgAppResp Term:2 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12] +3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[3->3 MsgAppResp Term:2 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12] deliver-msgs 1 drop=(2,4,5,6,7) ---- @@ -210,7 +210,7 @@ HardState Term:2 Commit:11 Entries: 2/12 EntryNormal "" Messages: -1->AppendThread MsgStorageAppend Term:2 Log:0/0 Entries:[2/12 EntryNormal ""] HardState: Term:2 Commit:11 Responses:[1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0), 1->3 MsgAppResp Term:2 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12] +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, AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12] # Step 6: node 3 crashes and node 4 becomes leader getting the vote from 5, 6, and 7. @@ -237,7 +237,7 @@ Messages: 4->5 MsgVote Term:3 Log:1/11 4->6 MsgVote Term:3 Log:1/11 4->7 MsgVote Term:3 Log:1/11 -4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[4->4 MsgVoteResp Term:3 Log:0/0] +4->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[4->4 MsgVoteResp Term:3 Log:0/0] deliver-msgs 5 6 7 ---- @@ -260,39 +260,39 @@ process-ready 5 6 7 Ready MustSync=true: HardState Term:3 Vote:4 Commit:11 Messages: - 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[5->4 MsgVoteResp Term:3 Log:0/0] + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[5->4 MsgVoteResp Term:3 Log:0/0] > 6 handling Ready Ready MustSync=true: HardState Term:3 Vote:4 Commit:11 Messages: - 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[6->4 MsgVoteResp Term:3 Log:0/0] + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[6->4 MsgVoteResp Term:3 Log:0/0] > 7 handling Ready Ready MustSync=true: Lead:0 State:StateFollower HardState Term:3 Vote:4 Commit:11 Messages: - 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 Responses:[7->4 MsgVoteResp Term:3 Log:0/0] + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[7->4 MsgVoteResp Term:3 Log:0/0] process-append-thread 4 5 6 7 ---- > 4 processing append thread Processing: - 4->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + 4->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses: 4->4 MsgVoteResp Term:3 Log:0/0 > 5 processing append thread Processing: - 5->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses: 5->4 MsgVoteResp Term:3 Log:0/0 > 6 processing append thread Processing: - 6->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses: 6->4 MsgVoteResp Term:3 Log:0/0 > 7 processing append thread Processing: - 7->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Vote:4 Commit:11 + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses: 7->4 MsgVoteResp Term:3 Log:0/0 @@ -325,7 +325,7 @@ Messages: 4->5 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] 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:3 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[4->4 MsgAppResp Term:3 Log:0/12, AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12] +4->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[4->4 MsgAppResp Term:3 Log:0/12, AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12] # Step 7: before the new entries reach node 1, it hears of the term change # through a heartbeat and persists the new term. Node 1 then receives these @@ -366,7 +366,7 @@ Lead:4 State:StateFollower HardState Term:3 Commit:11 Messages: 1->4 MsgHeartbeatResp Term:3 Log:0/0 -1->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Commit:11 Responses:[AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12] +1->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Responses:[AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12] deliver-msgs 4 ---- @@ -390,7 +390,7 @@ Ready MustSync=true: Entries: 3/12 EntryNormal "" Messages: -1->AppendThread MsgStorageAppend Term:3 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[1->4 MsgAppResp Term:3 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12] +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[1->4 MsgAppResp Term:3 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12] # Step 8: The asynchronous log appends from the first Ready complete and the # MsgStorageAppendResp is returned to the raft node state machine. A decision @@ -403,7 +403,7 @@ raft-log 1 process-append-thread 1 ---- Processing: -1->AppendThread MsgStorageAppend Term:1 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses: 1->2 MsgAppResp Term:1 Log:0/12 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 @@ -429,7 +429,7 @@ INFO 1 [term: 3] ignored entry appends from a MsgStorageAppendResp message with process-append-thread 1 ---- Processing: -1->AppendThread MsgStorageAppend Term:2 Log:0/0 Entries:[2/12 EntryNormal ""] HardState: Term:2 Commit:11 +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 @@ -448,7 +448,7 @@ INFO 1 [term: 3] ignored entry appends from a MsgStorageAppendResp message with process-append-thread 1 ---- Processing: -1->AppendThread MsgStorageAppend Term:3 Log:0/0 HardState: Term:3 Commit:11 +1->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Responses: AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12 @@ -465,7 +465,7 @@ INFO entry at (index,term)=(12,2) mismatched with entry at (12,3) in unstable lo process-append-thread 1 ---- Processing: -1->AppendThread MsgStorageAppend Term:3 Log:0/0 Entries:[3/12 EntryNormal ""] +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses: 1->4 MsgAppResp Term:3 Log:0/12 AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 diff --git a/util.go b/util.go index 1bec5357..6b68fe2d 100644 --- a/util.go +++ b/util.go @@ -171,6 +171,9 @@ func DescribeMessage(m pb.Message, f EntryFormatter) string { if m.Commit != 0 { fmt.Fprintf(&buf, " Commit:%d", m.Commit) } + if m.Vote != 0 { + fmt.Fprintf(&buf, " Vote:%d", m.Vote) + } if len(m.Entries) > 0 { fmt.Fprint(&buf, " Entries:[") for i, e := range m.Entries { @@ -181,9 +184,6 @@ func DescribeMessage(m pb.Message, f EntryFormatter) string { } fmt.Fprint(&buf, "]") } - if m.HardState != nil { - fmt.Fprintf(&buf, " HardState: %s", DescribeHardState(*m.HardState)) - } if s := m.Snapshot; s != nil && !IsEmptySnap(*s) { fmt.Fprintf(&buf, " Snapshot: %s", DescribeSnapshot(*s)) }