|
|
@@ -24,6 +24,7 @@ import (
|
|
|
"testing"
|
|
|
|
|
|
pb "go.etcd.io/etcd/raft/raftpb"
|
|
|
+ "go.etcd.io/etcd/raft/tracker"
|
|
|
)
|
|
|
|
|
|
// nextEnts returns the appliable entries and updates the applied index
|
|
|
@@ -55,228 +56,16 @@ func (r *raft) readMessages() []pb.Message {
|
|
|
return msgs
|
|
|
}
|
|
|
|
|
|
-func TestProgressBecomeProbe(t *testing.T) {
|
|
|
- match := uint64(1)
|
|
|
- tests := []struct {
|
|
|
- p *Progress
|
|
|
- wnext uint64
|
|
|
- }{
|
|
|
- {
|
|
|
- &Progress{State: ProgressStateReplicate, Match: match, Next: 5, ins: newInflights(256)},
|
|
|
- 2,
|
|
|
- },
|
|
|
- {
|
|
|
- // snapshot finish
|
|
|
- &Progress{State: ProgressStateSnapshot, Match: match, Next: 5, PendingSnapshot: 10, ins: newInflights(256)},
|
|
|
- 11,
|
|
|
- },
|
|
|
- {
|
|
|
- // snapshot failure
|
|
|
- &Progress{State: ProgressStateSnapshot, Match: match, Next: 5, PendingSnapshot: 0, ins: newInflights(256)},
|
|
|
- 2,
|
|
|
- },
|
|
|
- }
|
|
|
- for i, tt := range tests {
|
|
|
- tt.p.becomeProbe()
|
|
|
- if tt.p.State != ProgressStateProbe {
|
|
|
- t.Errorf("#%d: state = %s, want %s", i, tt.p.State, ProgressStateProbe)
|
|
|
- }
|
|
|
- if tt.p.Match != match {
|
|
|
- t.Errorf("#%d: match = %d, want %d", i, tt.p.Match, match)
|
|
|
- }
|
|
|
- if tt.p.Next != tt.wnext {
|
|
|
- t.Errorf("#%d: next = %d, want %d", i, tt.p.Next, tt.wnext)
|
|
|
- }
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-func TestProgressBecomeReplicate(t *testing.T) {
|
|
|
- p := &Progress{State: ProgressStateProbe, Match: 1, Next: 5, ins: newInflights(256)}
|
|
|
- p.becomeReplicate()
|
|
|
-
|
|
|
- if p.State != ProgressStateReplicate {
|
|
|
- t.Errorf("state = %s, want %s", p.State, ProgressStateReplicate)
|
|
|
- }
|
|
|
- if p.Match != 1 {
|
|
|
- t.Errorf("match = %d, want 1", p.Match)
|
|
|
- }
|
|
|
- if w := p.Match + 1; p.Next != w {
|
|
|
- t.Errorf("next = %d, want %d", p.Next, w)
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-func TestProgressBecomeSnapshot(t *testing.T) {
|
|
|
- p := &Progress{State: ProgressStateProbe, Match: 1, Next: 5, ins: newInflights(256)}
|
|
|
- p.becomeSnapshot(10)
|
|
|
-
|
|
|
- if p.State != ProgressStateSnapshot {
|
|
|
- t.Errorf("state = %s, want %s", p.State, ProgressStateSnapshot)
|
|
|
- }
|
|
|
- if p.Match != 1 {
|
|
|
- t.Errorf("match = %d, want 1", p.Match)
|
|
|
- }
|
|
|
- if p.PendingSnapshot != 10 {
|
|
|
- t.Errorf("pendingSnapshot = %d, want 10", p.PendingSnapshot)
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-func TestProgressUpdate(t *testing.T) {
|
|
|
- prevM, prevN := uint64(3), uint64(5)
|
|
|
- tests := []struct {
|
|
|
- update uint64
|
|
|
-
|
|
|
- wm uint64
|
|
|
- wn uint64
|
|
|
- wok bool
|
|
|
- }{
|
|
|
- {prevM - 1, prevM, prevN, false}, // do not decrease match, next
|
|
|
- {prevM, prevM, prevN, false}, // do not decrease next
|
|
|
- {prevM + 1, prevM + 1, prevN, true}, // increase match, do not decrease next
|
|
|
- {prevM + 2, prevM + 2, prevN + 1, true}, // increase match, next
|
|
|
- }
|
|
|
- for i, tt := range tests {
|
|
|
- p := &Progress{
|
|
|
- Match: prevM,
|
|
|
- Next: prevN,
|
|
|
- }
|
|
|
- ok := p.maybeUpdate(tt.update)
|
|
|
- if ok != tt.wok {
|
|
|
- t.Errorf("#%d: ok= %v, want %v", i, ok, tt.wok)
|
|
|
- }
|
|
|
- if p.Match != tt.wm {
|
|
|
- t.Errorf("#%d: match= %d, want %d", i, p.Match, tt.wm)
|
|
|
- }
|
|
|
- if p.Next != tt.wn {
|
|
|
- t.Errorf("#%d: next= %d, want %d", i, p.Next, tt.wn)
|
|
|
- }
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-func TestProgressMaybeDecr(t *testing.T) {
|
|
|
- tests := []struct {
|
|
|
- state ProgressStateType
|
|
|
- m uint64
|
|
|
- n uint64
|
|
|
- rejected uint64
|
|
|
- last uint64
|
|
|
-
|
|
|
- w bool
|
|
|
- wn uint64
|
|
|
- }{
|
|
|
- {
|
|
|
- // state replicate and rejected is not greater than match
|
|
|
- ProgressStateReplicate, 5, 10, 5, 5, false, 10,
|
|
|
- },
|
|
|
- {
|
|
|
- // state replicate and rejected is not greater than match
|
|
|
- ProgressStateReplicate, 5, 10, 4, 4, false, 10,
|
|
|
- },
|
|
|
- {
|
|
|
- // state replicate and rejected is greater than match
|
|
|
- // directly decrease to match+1
|
|
|
- ProgressStateReplicate, 5, 10, 9, 9, true, 6,
|
|
|
- },
|
|
|
- {
|
|
|
- // next-1 != rejected is always false
|
|
|
- ProgressStateProbe, 0, 0, 0, 0, false, 0,
|
|
|
- },
|
|
|
- {
|
|
|
- // next-1 != rejected is always false
|
|
|
- ProgressStateProbe, 0, 10, 5, 5, false, 10,
|
|
|
- },
|
|
|
- {
|
|
|
- // next>1 = decremented by 1
|
|
|
- ProgressStateProbe, 0, 10, 9, 9, true, 9,
|
|
|
- },
|
|
|
- {
|
|
|
- // next>1 = decremented by 1
|
|
|
- ProgressStateProbe, 0, 2, 1, 1, true, 1,
|
|
|
- },
|
|
|
- {
|
|
|
- // next<=1 = reset to 1
|
|
|
- ProgressStateProbe, 0, 1, 0, 0, true, 1,
|
|
|
- },
|
|
|
- {
|
|
|
- // decrease to min(rejected, last+1)
|
|
|
- ProgressStateProbe, 0, 10, 9, 2, true, 3,
|
|
|
- },
|
|
|
- {
|
|
|
- // rejected < 1, reset to 1
|
|
|
- ProgressStateProbe, 0, 10, 9, 0, true, 1,
|
|
|
- },
|
|
|
- }
|
|
|
- for i, tt := range tests {
|
|
|
- p := &Progress{
|
|
|
- State: tt.state,
|
|
|
- Match: tt.m,
|
|
|
- Next: tt.n,
|
|
|
- }
|
|
|
- if g := p.maybeDecrTo(tt.rejected, tt.last); g != tt.w {
|
|
|
- t.Errorf("#%d: maybeDecrTo= %t, want %t", i, g, tt.w)
|
|
|
- }
|
|
|
- if gm := p.Match; gm != tt.m {
|
|
|
- t.Errorf("#%d: match= %d, want %d", i, gm, tt.m)
|
|
|
- }
|
|
|
- if gn := p.Next; gn != tt.wn {
|
|
|
- t.Errorf("#%d: next= %d, want %d", i, gn, tt.wn)
|
|
|
- }
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-func TestProgressIsPaused(t *testing.T) {
|
|
|
- tests := []struct {
|
|
|
- state ProgressStateType
|
|
|
- paused bool
|
|
|
-
|
|
|
- w bool
|
|
|
- }{
|
|
|
- {ProgressStateProbe, false, false},
|
|
|
- {ProgressStateProbe, true, true},
|
|
|
- {ProgressStateReplicate, false, false},
|
|
|
- {ProgressStateReplicate, true, false},
|
|
|
- {ProgressStateSnapshot, false, true},
|
|
|
- {ProgressStateSnapshot, true, true},
|
|
|
- }
|
|
|
- for i, tt := range tests {
|
|
|
- p := &Progress{
|
|
|
- State: tt.state,
|
|
|
- Paused: tt.paused,
|
|
|
- ins: newInflights(256),
|
|
|
- }
|
|
|
- if g := p.IsPaused(); g != tt.w {
|
|
|
- t.Errorf("#%d: paused= %t, want %t", i, g, tt.w)
|
|
|
- }
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
-// TestProgressResume ensures that progress.maybeUpdate and progress.maybeDecrTo
|
|
|
-// will reset progress.paused.
|
|
|
-func TestProgressResume(t *testing.T) {
|
|
|
- p := &Progress{
|
|
|
- Next: 2,
|
|
|
- Paused: true,
|
|
|
- }
|
|
|
- p.maybeDecrTo(1, 1)
|
|
|
- if p.Paused {
|
|
|
- t.Errorf("paused= %v, want false", p.Paused)
|
|
|
- }
|
|
|
- p.Paused = true
|
|
|
- p.maybeUpdate(2)
|
|
|
- if p.Paused {
|
|
|
- t.Errorf("paused= %v, want false", p.Paused)
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
func TestProgressLeader(t *testing.T) {
|
|
|
r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
|
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
|
- r.prs.prs[2].becomeReplicate()
|
|
|
+ r.prs.Progress[2].BecomeReplicate()
|
|
|
|
|
|
// Send proposals to r1. The first 5 entries should be appended to the log.
|
|
|
propMsg := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("foo")}}}
|
|
|
for i := 0; i < 5; i++ {
|
|
|
- if pr := r.prs.prs[r.id]; pr.State != ProgressStateReplicate || pr.Match != uint64(i+1) || pr.Next != pr.Match+1 {
|
|
|
+ if pr := r.prs.Progress[r.id]; pr.State != tracker.StateReplicate || pr.Match != uint64(i+1) || pr.Next != pr.Match+1 {
|
|
|
t.Errorf("unexpected progress %v", pr)
|
|
|
}
|
|
|
if err := r.Step(propMsg); err != nil {
|
|
|
@@ -291,17 +80,17 @@ func TestProgressResumeByHeartbeatResp(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
|
|
|
|
- r.prs.prs[2].Paused = true
|
|
|
+ r.prs.Progress[2].ProbeSent = true
|
|
|
|
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
|
- if !r.prs.prs[2].Paused {
|
|
|
- t.Errorf("paused = %v, want true", r.prs.prs[2].Paused)
|
|
|
+ if !r.prs.Progress[2].ProbeSent {
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.Progress[2].ProbeSent)
|
|
|
}
|
|
|
|
|
|
- r.prs.prs[2].becomeReplicate()
|
|
|
+ r.prs.Progress[2].BecomeReplicate()
|
|
|
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
|
|
|
- if r.prs.prs[2].Paused {
|
|
|
- t.Errorf("paused = %v, want false", r.prs.prs[2].Paused)
|
|
|
+ if r.prs.Progress[2].ProbeSent {
|
|
|
+ t.Errorf("paused = %v, want false", r.prs.Progress[2].ProbeSent)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -331,7 +120,7 @@ func TestProgressFlowControl(t *testing.T) {
|
|
|
r.readMessages()
|
|
|
|
|
|
// While node 2 is in probe state, propose a bunch of entries.
|
|
|
- r.prs.prs[2].becomeProbe()
|
|
|
+ r.prs.Progress[2].BecomeProbe()
|
|
|
blob := []byte(strings.Repeat("a", 1000))
|
|
|
for i := 0; i < 10; i++ {
|
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: blob}}})
|
|
|
@@ -409,8 +198,8 @@ func TestUncommittedEntryLimit(t *testing.T) {
|
|
|
|
|
|
// Set the two followers to the replicate state. Commit to tail of log.
|
|
|
const numFollowers = 2
|
|
|
- r.prs.prs[2].becomeReplicate()
|
|
|
- r.prs.prs[3].becomeReplicate()
|
|
|
+ r.prs.Progress[2].BecomeReplicate()
|
|
|
+ r.prs.Progress[3].BecomeReplicate()
|
|
|
r.uncommittedSize = 0
|
|
|
|
|
|
// Send proposals to r1. The first 5 entries should be appended to the log.
|
|
|
@@ -889,7 +678,7 @@ func TestLearnerLogReplication(t *testing.T) {
|
|
|
t.Errorf("peer 2 wants committed to %d, but still %d", n1.raftLog.committed, n2.raftLog.committed)
|
|
|
}
|
|
|
|
|
|
- match := n1.prs.getProgress(2).Match
|
|
|
+ match := n1.prs.Progress[2].Match
|
|
|
if match != n2.raftLog.committed {
|
|
|
t.Errorf("progress 2 of leader 1 wants match %d, but got %d", n2.raftLog.committed, match)
|
|
|
}
|
|
|
@@ -1351,9 +1140,9 @@ func TestCommit(t *testing.T) {
|
|
|
storage.hardState = pb.HardState{Term: tt.smTerm}
|
|
|
|
|
|
sm := newTestRaft(1, []uint64{1}, 10, 2, storage)
|
|
|
- sm.prs.removeAny(1)
|
|
|
+ sm.prs.RemoveAny(1)
|
|
|
for j := 0; j < len(tt.matches); j++ {
|
|
|
- sm.prs.initProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1, false)
|
|
|
+ sm.prs.InitProgress(uint64(j)+1, tt.matches[j], tt.matches[j]+1, false)
|
|
|
}
|
|
|
sm.maybeCommit()
|
|
|
if g := sm.raftLog.committed; g != tt.w {
|
|
|
@@ -2138,7 +1927,7 @@ func TestNonPromotableVoterWithCheckQuorum(t *testing.T) {
|
|
|
nt := newNetwork(a, b)
|
|
|
setRandomizedElectionTimeout(b, b.electionTimeout+1)
|
|
|
// Need to remove 2 again to make it a non-promotable node since newNetwork overwritten some internal states
|
|
|
- b.prs.removeAny(2)
|
|
|
+ b.prs.RemoveAny(2)
|
|
|
|
|
|
if b.promotable() {
|
|
|
t.Fatalf("promotable = %v, want false", b.promotable())
|
|
|
@@ -2632,7 +2421,7 @@ func TestLeaderAppResp(t *testing.T) {
|
|
|
sm.readMessages()
|
|
|
sm.Step(pb.Message{From: 2, Type: pb.MsgAppResp, Index: tt.index, Term: sm.Term, Reject: tt.reject, RejectHint: tt.index})
|
|
|
|
|
|
- p := sm.prs.prs[2]
|
|
|
+ p := sm.prs.Progress[2]
|
|
|
if p.Match != tt.wmatch {
|
|
|
t.Errorf("#%d match = %d, want %d", i, p.Match, tt.wmatch)
|
|
|
}
|
|
|
@@ -2679,9 +2468,9 @@ func TestBcastBeat(t *testing.T) {
|
|
|
mustAppendEntry(sm, pb.Entry{Index: uint64(i) + 1})
|
|
|
}
|
|
|
// slow follower
|
|
|
- sm.prs.prs[2].Match, sm.prs.prs[2].Next = 5, 6
|
|
|
+ sm.prs.Progress[2].Match, sm.prs.Progress[2].Next = 5, 6
|
|
|
// normal follower
|
|
|
- sm.prs.prs[3].Match, sm.prs.prs[3].Next = sm.raftLog.lastIndex(), sm.raftLog.lastIndex()+1
|
|
|
+ sm.prs.Progress[3].Match, sm.prs.Progress[3].Next = sm.raftLog.lastIndex(), sm.raftLog.lastIndex()+1
|
|
|
|
|
|
sm.Step(pb.Message{Type: pb.MsgBeat})
|
|
|
msgs := sm.readMessages()
|
|
|
@@ -2689,8 +2478,8 @@ func TestBcastBeat(t *testing.T) {
|
|
|
t.Fatalf("len(msgs) = %v, want 2", len(msgs))
|
|
|
}
|
|
|
wantCommitMap := map[uint64]uint64{
|
|
|
- 2: min(sm.raftLog.committed, sm.prs.prs[2].Match),
|
|
|
- 3: min(sm.raftLog.committed, sm.prs.prs[3].Match),
|
|
|
+ 2: min(sm.raftLog.committed, sm.prs.Progress[2].Match),
|
|
|
+ 3: min(sm.raftLog.committed, sm.prs.Progress[3].Match),
|
|
|
}
|
|
|
for i, m := range msgs {
|
|
|
if m.Type != pb.MsgHeartbeat {
|
|
|
@@ -2759,16 +2548,16 @@ func TestLeaderIncreaseNext(t *testing.T) {
|
|
|
previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 1, Index: 2}, {Term: 1, Index: 3}}
|
|
|
tests := []struct {
|
|
|
// progress
|
|
|
- state ProgressStateType
|
|
|
+ state tracker.StateType
|
|
|
next uint64
|
|
|
|
|
|
wnext uint64
|
|
|
}{
|
|
|
// state replicate, optimistically increase next
|
|
|
// previous entries + noop entry + propose + 1
|
|
|
- {ProgressStateReplicate, 2, uint64(len(previousEnts) + 1 + 1 + 1)},
|
|
|
+ {tracker.StateReplicate, 2, uint64(len(previousEnts) + 1 + 1 + 1)},
|
|
|
// state probe, not optimistically increase next
|
|
|
- {ProgressStateProbe, 2, 2},
|
|
|
+ {tracker.StateProbe, 2, 2},
|
|
|
}
|
|
|
|
|
|
for i, tt := range tests {
|
|
|
@@ -2776,11 +2565,11 @@ func TestLeaderIncreaseNext(t *testing.T) {
|
|
|
sm.raftLog.append(previousEnts...)
|
|
|
sm.becomeCandidate()
|
|
|
sm.becomeLeader()
|
|
|
- sm.prs.prs[2].State = tt.state
|
|
|
- sm.prs.prs[2].Next = tt.next
|
|
|
+ sm.prs.Progress[2].State = tt.state
|
|
|
+ sm.prs.Progress[2].Next = tt.next
|
|
|
sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
|
|
|
|
|
- p := sm.prs.prs[2]
|
|
|
+ p := sm.prs.Progress[2]
|
|
|
if p.Next != tt.wnext {
|
|
|
t.Errorf("#%d next = %d, want %d", i, p.Next, tt.wnext)
|
|
|
}
|
|
|
@@ -2792,7 +2581,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
|
- r.prs.prs[2].becomeProbe()
|
|
|
+ r.prs.Progress[2].BecomeProbe()
|
|
|
|
|
|
// each round is a heartbeat
|
|
|
for i := 0; i < 3; i++ {
|
|
|
@@ -2811,8 +2600,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if !r.prs.prs[2].Paused {
|
|
|
- t.Errorf("paused = %v, want true", r.prs.prs[2].Paused)
|
|
|
+ if !r.prs.Progress[2].ProbeSent {
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.Progress[2].ProbeSent)
|
|
|
}
|
|
|
for j := 0; j < 10; j++ {
|
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
|
@@ -2826,8 +2615,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
for j := 0; j < r.heartbeatTimeout; j++ {
|
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
|
}
|
|
|
- if !r.prs.prs[2].Paused {
|
|
|
- t.Errorf("paused = %v, want true", r.prs.prs[2].Paused)
|
|
|
+ if !r.prs.Progress[2].ProbeSent {
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.Progress[2].ProbeSent)
|
|
|
}
|
|
|
|
|
|
// consume the heartbeat
|
|
|
@@ -2849,8 +2638,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
if msg[0].Index != 0 {
|
|
|
t.Errorf("index = %d, want %d", msg[0].Index, 0)
|
|
|
}
|
|
|
- if !r.prs.prs[2].Paused {
|
|
|
- t.Errorf("paused = %v, want true", r.prs.prs[2].Paused)
|
|
|
+ if !r.prs.Progress[2].ProbeSent {
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.Progress[2].ProbeSent)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -2859,7 +2648,7 @@ func TestSendAppendForProgressReplicate(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
|
- r.prs.prs[2].becomeReplicate()
|
|
|
+ r.prs.Progress[2].BecomeReplicate()
|
|
|
|
|
|
for i := 0; i < 10; i++ {
|
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
|
@@ -2876,7 +2665,7 @@ func TestSendAppendForProgressSnapshot(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
|
- r.prs.prs[2].becomeSnapshot(10)
|
|
|
+ r.prs.Progress[2].BecomeSnapshot(10)
|
|
|
|
|
|
for i := 0; i < 10; i++ {
|
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
|
@@ -2897,17 +2686,17 @@ func TestRecvMsgUnreachable(t *testing.T) {
|
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
|
// set node 2 to state replicate
|
|
|
- r.prs.prs[2].Match = 3
|
|
|
- r.prs.prs[2].becomeReplicate()
|
|
|
- r.prs.prs[2].optimisticUpdate(5)
|
|
|
+ r.prs.Progress[2].Match = 3
|
|
|
+ r.prs.Progress[2].BecomeReplicate()
|
|
|
+ r.prs.Progress[2].OptimisticUpdate(5)
|
|
|
|
|
|
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgUnreachable})
|
|
|
|
|
|
- if r.prs.prs[2].State != ProgressStateProbe {
|
|
|
- t.Errorf("state = %s, want %s", r.prs.prs[2].State, ProgressStateProbe)
|
|
|
+ if r.prs.Progress[2].State != tracker.StateProbe {
|
|
|
+ t.Errorf("state = %s, want %s", r.prs.Progress[2].State, tracker.StateProbe)
|
|
|
}
|
|
|
- if wnext := r.prs.prs[2].Match + 1; r.prs.prs[2].Next != wnext {
|
|
|
- t.Errorf("next = %d, want %d", r.prs.prs[2].Next, wnext)
|
|
|
+ if wnext := r.prs.Progress[2].Match + 1; r.prs.Progress[2].Next != wnext {
|
|
|
+ t.Errorf("next = %d, want %d", r.prs.Progress[2].Next, wnext)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -2932,7 +2721,7 @@ func TestRestore(t *testing.T) {
|
|
|
if mustTerm(sm.raftLog.term(s.Metadata.Index)) != s.Metadata.Term {
|
|
|
t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term)
|
|
|
}
|
|
|
- sg := sm.prs.voterNodes()
|
|
|
+ sg := sm.prs.VoterNodes()
|
|
|
if !reflect.DeepEqual(sg, s.Metadata.ConfState.Nodes) {
|
|
|
t.Errorf("sm.Nodes = %+v, want %+v", sg, s.Metadata.ConfState.Nodes)
|
|
|
}
|
|
|
@@ -2964,22 +2753,22 @@ func TestRestoreWithLearner(t *testing.T) {
|
|
|
if mustTerm(sm.raftLog.term(s.Metadata.Index)) != s.Metadata.Term {
|
|
|
t.Errorf("log.lastTerm = %d, want %d", mustTerm(sm.raftLog.term(s.Metadata.Index)), s.Metadata.Term)
|
|
|
}
|
|
|
- sg := sm.prs.voterNodes()
|
|
|
+ sg := sm.prs.VoterNodes()
|
|
|
if len(sg) != len(s.Metadata.ConfState.Nodes) {
|
|
|
t.Errorf("sm.Nodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Nodes)
|
|
|
}
|
|
|
- lns := sm.prs.learnerNodes()
|
|
|
+ lns := sm.prs.LearnerNodes()
|
|
|
if len(lns) != len(s.Metadata.ConfState.Learners) {
|
|
|
t.Errorf("sm.LearnerNodes = %+v, length not equal with %+v", sg, s.Metadata.ConfState.Learners)
|
|
|
}
|
|
|
for _, n := range s.Metadata.ConfState.Nodes {
|
|
|
- if sm.prs.prs[n].IsLearner {
|
|
|
- t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.prs[n], false)
|
|
|
+ if sm.prs.Progress[n].IsLearner {
|
|
|
+ t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.Progress[n], false)
|
|
|
}
|
|
|
}
|
|
|
for _, n := range s.Metadata.ConfState.Learners {
|
|
|
- if !sm.prs.prs[n].IsLearner {
|
|
|
- t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.prs[n], true)
|
|
|
+ if !sm.prs.Progress[n].IsLearner {
|
|
|
+ t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.Progress[n], true)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -3121,8 +2910,8 @@ func TestProvideSnap(t *testing.T) {
|
|
|
sm.becomeLeader()
|
|
|
|
|
|
// force set the next of node 2, so that node 2 needs a snapshot
|
|
|
- sm.prs.prs[2].Next = sm.raftLog.firstIndex()
|
|
|
- sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs.prs[2].Next - 1, Reject: true})
|
|
|
+ sm.prs.Progress[2].Next = sm.raftLog.firstIndex()
|
|
|
+ sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs.Progress[2].Next - 1, Reject: true})
|
|
|
|
|
|
msgs := sm.readMessages()
|
|
|
if len(msgs) != 1 {
|
|
|
@@ -3152,8 +2941,8 @@ func TestIgnoreProvidingSnap(t *testing.T) {
|
|
|
|
|
|
// force set the next of node 2, so that node 2 needs a snapshot
|
|
|
// change node 2 to be inactive, expect node 1 ignore sending snapshot to 2
|
|
|
- sm.prs.prs[2].Next = sm.raftLog.firstIndex() - 1
|
|
|
- sm.prs.prs[2].RecentActive = false
|
|
|
+ sm.prs.Progress[2].Next = sm.raftLog.firstIndex() - 1
|
|
|
+ sm.prs.Progress[2].RecentActive = false
|
|
|
|
|
|
sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
|
|
|
|
|
@@ -3193,7 +2982,7 @@ func TestSlowNodeRestore(t *testing.T) {
|
|
|
}
|
|
|
lead := nt.peers[1].(*raft)
|
|
|
nextEnts(lead, nt.storage[1])
|
|
|
- nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.prs.voterNodes()}, nil)
|
|
|
+ nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.prs.VoterNodes()}, nil)
|
|
|
nt.storage[1].Compact(lead.raftLog.applied)
|
|
|
|
|
|
nt.recover()
|
|
|
@@ -3201,7 +2990,7 @@ func TestSlowNodeRestore(t *testing.T) {
|
|
|
// node 3 will only be considered as active when node 1 receives a reply from it.
|
|
|
for {
|
|
|
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
|
- if lead.prs.prs[3].RecentActive {
|
|
|
+ if lead.prs.Progress[3].RecentActive {
|
|
|
break
|
|
|
}
|
|
|
}
|
|
|
@@ -3288,7 +3077,7 @@ func TestNewLeaderPendingConfig(t *testing.T) {
|
|
|
func TestAddNode(t *testing.T) {
|
|
|
r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
|
|
r.addNode(2)
|
|
|
- nodes := r.prs.voterNodes()
|
|
|
+ nodes := r.prs.VoterNodes()
|
|
|
wnodes := []uint64{1, 2}
|
|
|
if !reflect.DeepEqual(nodes, wnodes) {
|
|
|
t.Errorf("nodes = %v, want %v", nodes, wnodes)
|
|
|
@@ -3299,13 +3088,13 @@ func TestAddNode(t *testing.T) {
|
|
|
func TestAddLearner(t *testing.T) {
|
|
|
r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
|
|
|
r.addLearner(2)
|
|
|
- nodes := r.prs.learnerNodes()
|
|
|
+ nodes := r.prs.LearnerNodes()
|
|
|
wnodes := []uint64{2}
|
|
|
if !reflect.DeepEqual(nodes, wnodes) {
|
|
|
t.Errorf("nodes = %v, want %v", nodes, wnodes)
|
|
|
}
|
|
|
- if !r.prs.prs[2].IsLearner {
|
|
|
- t.Errorf("node 2 is learner %t, want %t", r.prs.prs[2].IsLearner, true)
|
|
|
+ if !r.prs.Progress[2].IsLearner {
|
|
|
+ t.Errorf("node 2 is learner %t, want %t", r.prs.Progress[2].IsLearner, true)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -3349,14 +3138,14 @@ func TestRemoveNode(t *testing.T) {
|
|
|
r := newTestRaft(1, []uint64{1, 2}, 10, 1, NewMemoryStorage())
|
|
|
r.removeNode(2)
|
|
|
w := []uint64{1}
|
|
|
- if g := r.prs.voterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
+ if g := r.prs.VoterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
t.Errorf("nodes = %v, want %v", g, w)
|
|
|
}
|
|
|
|
|
|
// remove all nodes from cluster
|
|
|
r.removeNode(1)
|
|
|
w = []uint64{}
|
|
|
- if g := r.prs.voterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
+ if g := r.prs.VoterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
t.Errorf("nodes = %v, want %v", g, w)
|
|
|
}
|
|
|
}
|
|
|
@@ -3367,18 +3156,18 @@ func TestRemoveLearner(t *testing.T) {
|
|
|
r := newTestLearnerRaft(1, []uint64{1}, []uint64{2}, 10, 1, NewMemoryStorage())
|
|
|
r.removeNode(2)
|
|
|
w := []uint64{1}
|
|
|
- if g := r.prs.voterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
+ if g := r.prs.VoterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
t.Errorf("nodes = %v, want %v", g, w)
|
|
|
}
|
|
|
|
|
|
w = []uint64{}
|
|
|
- if g := r.prs.learnerNodes(); !reflect.DeepEqual(g, w) {
|
|
|
+ if g := r.prs.LearnerNodes(); !reflect.DeepEqual(g, w) {
|
|
|
t.Errorf("nodes = %v, want %v", g, w)
|
|
|
}
|
|
|
|
|
|
// remove all nodes from cluster
|
|
|
r.removeNode(1)
|
|
|
- if g := r.prs.voterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
+ if g := r.prs.VoterNodes(); !reflect.DeepEqual(g, w) {
|
|
|
t.Errorf("nodes = %v, want %v", g, w)
|
|
|
}
|
|
|
}
|
|
|
@@ -3417,8 +3206,8 @@ func TestRaftNodes(t *testing.T) {
|
|
|
}
|
|
|
for i, tt := range tests {
|
|
|
r := newTestRaft(1, tt.ids, 10, 1, NewMemoryStorage())
|
|
|
- if !reflect.DeepEqual(r.prs.voterNodes(), tt.wids) {
|
|
|
- t.Errorf("#%d: nodes = %+v, want %+v", i, r.prs.voterNodes(), tt.wids)
|
|
|
+ if !reflect.DeepEqual(r.prs.VoterNodes(), tt.wids) {
|
|
|
+ t.Errorf("#%d: nodes = %+v, want %+v", i, r.prs.VoterNodes(), tt.wids)
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
@@ -3619,8 +3408,8 @@ func TestLeaderTransferToSlowFollower(t *testing.T) {
|
|
|
|
|
|
nt.recover()
|
|
|
lead := nt.peers[1].(*raft)
|
|
|
- if lead.prs.prs[3].Match != 1 {
|
|
|
- t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs.prs[3].Match, 1)
|
|
|
+ if lead.prs.Progress[3].Match != 1 {
|
|
|
+ t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs.Progress[3].Match, 1)
|
|
|
}
|
|
|
|
|
|
// Transfer leadership to 3 when node 3 is lack of log.
|
|
|
@@ -3638,12 +3427,12 @@ func TestLeaderTransferAfterSnapshot(t *testing.T) {
|
|
|
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}})
|
|
|
lead := nt.peers[1].(*raft)
|
|
|
nextEnts(lead, nt.storage[1])
|
|
|
- nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.prs.voterNodes()}, nil)
|
|
|
+ nt.storage[1].CreateSnapshot(lead.raftLog.applied, &pb.ConfState{Nodes: lead.prs.VoterNodes()}, nil)
|
|
|
nt.storage[1].Compact(lead.raftLog.applied)
|
|
|
|
|
|
nt.recover()
|
|
|
- if lead.prs.prs[3].Match != 1 {
|
|
|
- t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs.prs[3].Match, 1)
|
|
|
+ if lead.prs.Progress[3].Match != 1 {
|
|
|
+ t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs.Progress[3].Match, 1)
|
|
|
}
|
|
|
|
|
|
// Transfer leadership to 3 when node 3 is lack of snapshot.
|
|
|
@@ -3722,8 +3511,8 @@ func TestLeaderTransferIgnoreProposal(t *testing.T) {
|
|
|
t.Fatalf("should return drop proposal error while transferring")
|
|
|
}
|
|
|
|
|
|
- if lead.prs.prs[1].Match != 1 {
|
|
|
- t.Fatalf("node 1 has match %x, want %x", lead.prs.prs[1].Match, 1)
|
|
|
+ if lead.prs.Progress[1].Match != 1 {
|
|
|
+ t.Fatalf("node 1 has match %x, want %x", lead.prs.Progress[1].Match, 1)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@@ -4329,24 +4118,21 @@ func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *netw
|
|
|
sm := newRaft(cfg)
|
|
|
npeers[id] = sm
|
|
|
case *raft:
|
|
|
- learners := make(map[uint64]bool, len(v.prs.learners))
|
|
|
- for i := range v.prs.learners {
|
|
|
+ learners := make(map[uint64]bool, len(v.prs.Learners))
|
|
|
+ for i := range v.prs.Learners {
|
|
|
learners[i] = true
|
|
|
}
|
|
|
v.id = id
|
|
|
- v.prs.voters[0] = make(map[uint64]struct{})
|
|
|
- v.prs.voters[1] = make(map[uint64]struct{})
|
|
|
- v.prs.learners = make(map[uint64]struct{})
|
|
|
- v.prs.prs = make(map[uint64]*Progress)
|
|
|
+ v.prs = tracker.MakeProgressTracker(v.prs.MaxInflight)
|
|
|
for i := 0; i < size; i++ {
|
|
|
- pr := &Progress{}
|
|
|
+ pr := &tracker.Progress{}
|
|
|
if _, ok := learners[peerAddrs[i]]; ok {
|
|
|
pr.IsLearner = true
|
|
|
- v.prs.learners[peerAddrs[i]] = struct{}{}
|
|
|
+ v.prs.Learners[peerAddrs[i]] = struct{}{}
|
|
|
} else {
|
|
|
- v.prs.voters[0][peerAddrs[i]] = struct{}{}
|
|
|
+ v.prs.Voters[0][peerAddrs[i]] = struct{}{}
|
|
|
}
|
|
|
- v.prs.prs[peerAddrs[i]] = pr
|
|
|
+ v.prs.Progress[peerAddrs[i]] = pr
|
|
|
}
|
|
|
v.reset(v.Term)
|
|
|
npeers[id] = v
|