|
@@ -271,12 +271,12 @@ func TestProgressLeader(t *testing.T) {
|
|
|
r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
|
|
r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
|
|
|
r.becomeCandidate()
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
|
- r.prs[2].becomeReplicate()
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeReplicate()
|
|
|
|
|
|
|
|
// Send proposals to r1. The first 5 entries should be appended to the log.
|
|
// 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")}}}
|
|
propMsg := pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("foo")}}}
|
|
|
for i := 0; i < 5; i++ {
|
|
for i := 0; i < 5; i++ {
|
|
|
- if pr := r.prs[r.id]; pr.State != ProgressStateReplicate || pr.Match != uint64(i+1) || pr.Next != pr.Match+1 {
|
|
|
|
|
|
|
+ if pr := r.prs.nodes[r.id]; pr.State != ProgressStateReplicate || pr.Match != uint64(i+1) || pr.Next != pr.Match+1 {
|
|
|
t.Errorf("unexpected progress %v", pr)
|
|
t.Errorf("unexpected progress %v", pr)
|
|
|
}
|
|
}
|
|
|
if err := r.Step(propMsg); err != nil {
|
|
if err := r.Step(propMsg); err != nil {
|
|
@@ -291,17 +291,17 @@ func TestProgressResumeByHeartbeatResp(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
|
|
|
|
|
|
- r.prs[2].Paused = true
|
|
|
|
|
|
|
+ r.prs.nodes[2].Paused = true
|
|
|
|
|
|
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
|
- if !r.prs[2].Paused {
|
|
|
|
|
- t.Errorf("paused = %v, want true", r.prs[2].Paused)
|
|
|
|
|
|
|
+ if !r.prs.nodes[2].Paused {
|
|
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.nodes[2].Paused)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- r.prs[2].becomeReplicate()
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeReplicate()
|
|
|
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
|
|
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
|
|
|
- if r.prs[2].Paused {
|
|
|
|
|
- t.Errorf("paused = %v, want false", r.prs[2].Paused)
|
|
|
|
|
|
|
+ if r.prs.nodes[2].Paused {
|
|
|
|
|
+ t.Errorf("paused = %v, want false", r.prs.nodes[2].Paused)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -331,7 +331,7 @@ func TestProgressFlowControl(t *testing.T) {
|
|
|
r.readMessages()
|
|
r.readMessages()
|
|
|
|
|
|
|
|
// While node 2 is in probe state, propose a bunch of entries.
|
|
// While node 2 is in probe state, propose a bunch of entries.
|
|
|
- r.prs[2].becomeProbe()
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeProbe()
|
|
|
blob := []byte(strings.Repeat("a", 1000))
|
|
blob := []byte(strings.Repeat("a", 1000))
|
|
|
for i := 0; i < 10; i++ {
|
|
for i := 0; i < 10; i++ {
|
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: blob}}})
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: blob}}})
|
|
@@ -409,8 +409,8 @@ func TestUncommittedEntryLimit(t *testing.T) {
|
|
|
|
|
|
|
|
// Set the two followers to the replicate state. Commit to tail of log.
|
|
// Set the two followers to the replicate state. Commit to tail of log.
|
|
|
const numFollowers = 2
|
|
const numFollowers = 2
|
|
|
- r.prs[2].becomeReplicate()
|
|
|
|
|
- r.prs[3].becomeReplicate()
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeReplicate()
|
|
|
|
|
+ r.prs.nodes[3].becomeReplicate()
|
|
|
r.uncommittedSize = 0
|
|
r.uncommittedSize = 0
|
|
|
|
|
|
|
|
// Send proposals to r1. The first 5 entries should be appended to the log.
|
|
// Send proposals to r1. The first 5 entries should be appended to the log.
|
|
@@ -2137,7 +2137,7 @@ func TestNonPromotableVoterWithCheckQuorum(t *testing.T) {
|
|
|
nt := newNetwork(a, b)
|
|
nt := newNetwork(a, b)
|
|
|
setRandomizedElectionTimeout(b, b.electionTimeout+1)
|
|
setRandomizedElectionTimeout(b, b.electionTimeout+1)
|
|
|
// Need to remove 2 again to make it a non-promotable node since newNetwork overwritten some internal states
|
|
// Need to remove 2 again to make it a non-promotable node since newNetwork overwritten some internal states
|
|
|
- b.delProgress(2)
|
|
|
|
|
|
|
+ b.prs.removeAny(2)
|
|
|
|
|
|
|
|
if b.promotable() {
|
|
if b.promotable() {
|
|
|
t.Fatalf("promotable = %v, want false", b.promotable())
|
|
t.Fatalf("promotable = %v, want false", b.promotable())
|
|
@@ -2631,7 +2631,7 @@ func TestLeaderAppResp(t *testing.T) {
|
|
|
sm.readMessages()
|
|
sm.readMessages()
|
|
|
sm.Step(pb.Message{From: 2, Type: pb.MsgAppResp, Index: tt.index, Term: sm.Term, Reject: tt.reject, RejectHint: tt.index})
|
|
sm.Step(pb.Message{From: 2, Type: pb.MsgAppResp, Index: tt.index, Term: sm.Term, Reject: tt.reject, RejectHint: tt.index})
|
|
|
|
|
|
|
|
- p := sm.prs[2]
|
|
|
|
|
|
|
+ p := sm.prs.nodes[2]
|
|
|
if p.Match != tt.wmatch {
|
|
if p.Match != tt.wmatch {
|
|
|
t.Errorf("#%d match = %d, want %d", i, p.Match, tt.wmatch)
|
|
t.Errorf("#%d match = %d, want %d", i, p.Match, tt.wmatch)
|
|
|
}
|
|
}
|
|
@@ -2678,9 +2678,9 @@ func TestBcastBeat(t *testing.T) {
|
|
|
mustAppendEntry(sm, pb.Entry{Index: uint64(i) + 1})
|
|
mustAppendEntry(sm, pb.Entry{Index: uint64(i) + 1})
|
|
|
}
|
|
}
|
|
|
// slow follower
|
|
// slow follower
|
|
|
- sm.prs[2].Match, sm.prs[2].Next = 5, 6
|
|
|
|
|
|
|
+ sm.prs.nodes[2].Match, sm.prs.nodes[2].Next = 5, 6
|
|
|
// normal follower
|
|
// normal follower
|
|
|
- sm.prs[3].Match, sm.prs[3].Next = sm.raftLog.lastIndex(), sm.raftLog.lastIndex()+1
|
|
|
|
|
|
|
+ sm.prs.nodes[3].Match, sm.prs.nodes[3].Next = sm.raftLog.lastIndex(), sm.raftLog.lastIndex()+1
|
|
|
|
|
|
|
|
sm.Step(pb.Message{Type: pb.MsgBeat})
|
|
sm.Step(pb.Message{Type: pb.MsgBeat})
|
|
|
msgs := sm.readMessages()
|
|
msgs := sm.readMessages()
|
|
@@ -2688,8 +2688,8 @@ func TestBcastBeat(t *testing.T) {
|
|
|
t.Fatalf("len(msgs) = %v, want 2", len(msgs))
|
|
t.Fatalf("len(msgs) = %v, want 2", len(msgs))
|
|
|
}
|
|
}
|
|
|
wantCommitMap := map[uint64]uint64{
|
|
wantCommitMap := map[uint64]uint64{
|
|
|
- 2: min(sm.raftLog.committed, sm.prs[2].Match),
|
|
|
|
|
- 3: min(sm.raftLog.committed, sm.prs[3].Match),
|
|
|
|
|
|
|
+ 2: min(sm.raftLog.committed, sm.prs.nodes[2].Match),
|
|
|
|
|
+ 3: min(sm.raftLog.committed, sm.prs.nodes[3].Match),
|
|
|
}
|
|
}
|
|
|
for i, m := range msgs {
|
|
for i, m := range msgs {
|
|
|
if m.Type != pb.MsgHeartbeat {
|
|
if m.Type != pb.MsgHeartbeat {
|
|
@@ -2775,11 +2775,11 @@ func TestLeaderIncreaseNext(t *testing.T) {
|
|
|
sm.raftLog.append(previousEnts...)
|
|
sm.raftLog.append(previousEnts...)
|
|
|
sm.becomeCandidate()
|
|
sm.becomeCandidate()
|
|
|
sm.becomeLeader()
|
|
sm.becomeLeader()
|
|
|
- sm.prs[2].State = tt.state
|
|
|
|
|
- sm.prs[2].Next = tt.next
|
|
|
|
|
|
|
+ sm.prs.nodes[2].State = tt.state
|
|
|
|
|
+ sm.prs.nodes[2].Next = tt.next
|
|
|
sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
|
sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
|
|
|
|
|
|
|
- p := sm.prs[2]
|
|
|
|
|
|
|
+ p := sm.prs.nodes[2]
|
|
|
if p.Next != tt.wnext {
|
|
if p.Next != tt.wnext {
|
|
|
t.Errorf("#%d next = %d, want %d", i, p.Next, tt.wnext)
|
|
t.Errorf("#%d next = %d, want %d", i, p.Next, tt.wnext)
|
|
|
}
|
|
}
|
|
@@ -2791,7 +2791,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
r.readMessages()
|
|
|
- r.prs[2].becomeProbe()
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeProbe()
|
|
|
|
|
|
|
|
// each round is a heartbeat
|
|
// each round is a heartbeat
|
|
|
for i := 0; i < 3; i++ {
|
|
for i := 0; i < 3; i++ {
|
|
@@ -2810,8 +2810,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- if !r.prs[2].Paused {
|
|
|
|
|
- t.Errorf("paused = %v, want true", r.prs[2].Paused)
|
|
|
|
|
|
|
+ if !r.prs.nodes[2].Paused {
|
|
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.nodes[2].Paused)
|
|
|
}
|
|
}
|
|
|
for j := 0; j < 10; j++ {
|
|
for j := 0; j < 10; j++ {
|
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
@@ -2825,8 +2825,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
for j := 0; j < r.heartbeatTimeout; j++ {
|
|
for j := 0; j < r.heartbeatTimeout; j++ {
|
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
|
}
|
|
}
|
|
|
- if !r.prs[2].Paused {
|
|
|
|
|
- t.Errorf("paused = %v, want true", r.prs[2].Paused)
|
|
|
|
|
|
|
+ if !r.prs.nodes[2].Paused {
|
|
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.nodes[2].Paused)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// consume the heartbeat
|
|
// consume the heartbeat
|
|
@@ -2848,8 +2848,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
|
|
if msg[0].Index != 0 {
|
|
if msg[0].Index != 0 {
|
|
|
t.Errorf("index = %d, want %d", msg[0].Index, 0)
|
|
t.Errorf("index = %d, want %d", msg[0].Index, 0)
|
|
|
}
|
|
}
|
|
|
- if !r.prs[2].Paused {
|
|
|
|
|
- t.Errorf("paused = %v, want true", r.prs[2].Paused)
|
|
|
|
|
|
|
+ if !r.prs.nodes[2].Paused {
|
|
|
|
|
+ t.Errorf("paused = %v, want true", r.prs.nodes[2].Paused)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -2858,7 +2858,7 @@ func TestSendAppendForProgressReplicate(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
r.readMessages()
|
|
|
- r.prs[2].becomeReplicate()
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeReplicate()
|
|
|
|
|
|
|
|
for i := 0; i < 10; i++ {
|
|
for i := 0; i < 10; i++ {
|
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
@@ -2875,7 +2875,7 @@ func TestSendAppendForProgressSnapshot(t *testing.T) {
|
|
|
r.becomeCandidate()
|
|
r.becomeCandidate()
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
r.readMessages()
|
|
|
- r.prs[2].becomeSnapshot(10)
|
|
|
|
|
|
|
+ r.prs.nodes[2].becomeSnapshot(10)
|
|
|
|
|
|
|
|
for i := 0; i < 10; i++ {
|
|
for i := 0; i < 10; i++ {
|
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
|
@@ -2896,17 +2896,17 @@ func TestRecvMsgUnreachable(t *testing.T) {
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
|
r.readMessages()
|
|
r.readMessages()
|
|
|
// set node 2 to state replicate
|
|
// set node 2 to state replicate
|
|
|
- r.prs[2].Match = 3
|
|
|
|
|
- r.prs[2].becomeReplicate()
|
|
|
|
|
- r.prs[2].optimisticUpdate(5)
|
|
|
|
|
|
|
+ r.prs.nodes[2].Match = 3
|
|
|
|
|
+ r.prs.nodes[2].becomeReplicate()
|
|
|
|
|
+ r.prs.nodes[2].optimisticUpdate(5)
|
|
|
|
|
|
|
|
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgUnreachable})
|
|
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgUnreachable})
|
|
|
|
|
|
|
|
- if r.prs[2].State != ProgressStateProbe {
|
|
|
|
|
- t.Errorf("state = %s, want %s", r.prs[2].State, ProgressStateProbe)
|
|
|
|
|
|
|
+ if r.prs.nodes[2].State != ProgressStateProbe {
|
|
|
|
|
+ t.Errorf("state = %s, want %s", r.prs.nodes[2].State, ProgressStateProbe)
|
|
|
}
|
|
}
|
|
|
- if wnext := r.prs[2].Match + 1; r.prs[2].Next != wnext {
|
|
|
|
|
- t.Errorf("next = %d, want %d", r.prs[2].Next, wnext)
|
|
|
|
|
|
|
+ if wnext := r.prs.nodes[2].Match + 1; r.prs.nodes[2].Next != wnext {
|
|
|
|
|
+ t.Errorf("next = %d, want %d", r.prs.nodes[2].Next, wnext)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -2972,13 +2972,13 @@ func TestRestoreWithLearner(t *testing.T) {
|
|
|
t.Errorf("sm.LearnerNodes = %+v, length not equal with %+v", sg, 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 {
|
|
for _, n := range s.Metadata.ConfState.Nodes {
|
|
|
- if sm.prs[n].IsLearner {
|
|
|
|
|
- t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs[n], false)
|
|
|
|
|
|
|
+ if sm.prs.nodes[n].IsLearner {
|
|
|
|
|
+ t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.nodes[n], false)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
for _, n := range s.Metadata.ConfState.Learners {
|
|
for _, n := range s.Metadata.ConfState.Learners {
|
|
|
- if !sm.learnerPrs[n].IsLearner {
|
|
|
|
|
- t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs[n], true)
|
|
|
|
|
|
|
+ if !sm.prs.learners[n].IsLearner {
|
|
|
|
|
+ t.Errorf("sm.Node %x isLearner = %s, want %t", n, sm.prs.nodes[n], true)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -3120,8 +3120,8 @@ func TestProvideSnap(t *testing.T) {
|
|
|
sm.becomeLeader()
|
|
sm.becomeLeader()
|
|
|
|
|
|
|
|
// force set the next of node 2, so that node 2 needs a snapshot
|
|
// force set the next of node 2, so that node 2 needs a snapshot
|
|
|
- sm.prs[2].Next = sm.raftLog.firstIndex()
|
|
|
|
|
- sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs[2].Next - 1, Reject: true})
|
|
|
|
|
|
|
+ sm.prs.nodes[2].Next = sm.raftLog.firstIndex()
|
|
|
|
|
+ sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs.nodes[2].Next - 1, Reject: true})
|
|
|
|
|
|
|
|
msgs := sm.readMessages()
|
|
msgs := sm.readMessages()
|
|
|
if len(msgs) != 1 {
|
|
if len(msgs) != 1 {
|
|
@@ -3151,8 +3151,8 @@ func TestIgnoreProvidingSnap(t *testing.T) {
|
|
|
|
|
|
|
|
// force set the next of node 2, so that node 2 needs a snapshot
|
|
// 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
|
|
// change node 2 to be inactive, expect node 1 ignore sending snapshot to 2
|
|
|
- sm.prs[2].Next = sm.raftLog.firstIndex() - 1
|
|
|
|
|
- sm.prs[2].RecentActive = false
|
|
|
|
|
|
|
+ sm.prs.nodes[2].Next = sm.raftLog.firstIndex() - 1
|
|
|
|
|
+ sm.prs.nodes[2].RecentActive = false
|
|
|
|
|
|
|
|
sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
|
sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
|
|
|
|
|
|
|
@@ -3200,7 +3200,7 @@ func TestSlowNodeRestore(t *testing.T) {
|
|
|
// node 3 will only be considered as active when node 1 receives a reply from it.
|
|
// node 3 will only be considered as active when node 1 receives a reply from it.
|
|
|
for {
|
|
for {
|
|
|
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
|
|
- if lead.prs[3].RecentActive {
|
|
|
|
|
|
|
+ if lead.prs.nodes[3].RecentActive {
|
|
|
break
|
|
break
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
@@ -3303,8 +3303,8 @@ func TestAddLearner(t *testing.T) {
|
|
|
if !reflect.DeepEqual(nodes, wnodes) {
|
|
if !reflect.DeepEqual(nodes, wnodes) {
|
|
|
t.Errorf("nodes = %v, want %v", nodes, wnodes)
|
|
t.Errorf("nodes = %v, want %v", nodes, wnodes)
|
|
|
}
|
|
}
|
|
|
- if !r.learnerPrs[2].IsLearner {
|
|
|
|
|
- t.Errorf("node 2 is learner %t, want %t", r.prs[2].IsLearner, true)
|
|
|
|
|
|
|
+ if !r.prs.learners[2].IsLearner {
|
|
|
|
|
+ t.Errorf("node 2 is learner %t, want %t", r.prs.nodes[2].IsLearner, true)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -3618,8 +3618,8 @@ func TestLeaderTransferToSlowFollower(t *testing.T) {
|
|
|
|
|
|
|
|
nt.recover()
|
|
nt.recover()
|
|
|
lead := nt.peers[1].(*raft)
|
|
lead := nt.peers[1].(*raft)
|
|
|
- if lead.prs[3].Match != 1 {
|
|
|
|
|
- t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs[3].Match, 1)
|
|
|
|
|
|
|
+ if lead.prs.nodes[3].Match != 1 {
|
|
|
|
|
+ t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs.nodes[3].Match, 1)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// Transfer leadership to 3 when node 3 is lack of log.
|
|
// Transfer leadership to 3 when node 3 is lack of log.
|
|
@@ -3641,8 +3641,8 @@ func TestLeaderTransferAfterSnapshot(t *testing.T) {
|
|
|
nt.storage[1].Compact(lead.raftLog.applied)
|
|
nt.storage[1].Compact(lead.raftLog.applied)
|
|
|
|
|
|
|
|
nt.recover()
|
|
nt.recover()
|
|
|
- if lead.prs[3].Match != 1 {
|
|
|
|
|
- t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs[3].Match, 1)
|
|
|
|
|
|
|
+ if lead.prs.nodes[3].Match != 1 {
|
|
|
|
|
+ t.Fatalf("node 1 has match %x for node 3, want %x", lead.prs.nodes[3].Match, 1)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// Transfer leadership to 3 when node 3 is lack of snapshot.
|
|
// Transfer leadership to 3 when node 3 is lack of snapshot.
|
|
@@ -3721,8 +3721,8 @@ func TestLeaderTransferIgnoreProposal(t *testing.T) {
|
|
|
t.Fatalf("should return drop proposal error while transferring")
|
|
t.Fatalf("should return drop proposal error while transferring")
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- if lead.prs[1].Match != 1 {
|
|
|
|
|
- t.Fatalf("node 1 has match %x, want %x", lead.prs[1].Match, 1)
|
|
|
|
|
|
|
+ if lead.prs.nodes[1].Match != 1 {
|
|
|
|
|
+ t.Fatalf("node 1 has match %x, want %x", lead.prs.nodes[1].Match, 1)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
@@ -4294,18 +4294,18 @@ func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *netw
|
|
|
sm := newRaft(cfg)
|
|
sm := newRaft(cfg)
|
|
|
npeers[id] = sm
|
|
npeers[id] = sm
|
|
|
case *raft:
|
|
case *raft:
|
|
|
- learners := make(map[uint64]bool, len(v.learnerPrs))
|
|
|
|
|
- for i := range v.learnerPrs {
|
|
|
|
|
|
|
+ learners := make(map[uint64]bool, len(v.prs.learners))
|
|
|
|
|
+ for i := range v.prs.learners {
|
|
|
learners[i] = true
|
|
learners[i] = true
|
|
|
}
|
|
}
|
|
|
v.id = id
|
|
v.id = id
|
|
|
- v.prs = make(map[uint64]*Progress)
|
|
|
|
|
- v.learnerPrs = make(map[uint64]*Progress)
|
|
|
|
|
|
|
+ v.prs.nodes = make(map[uint64]*Progress)
|
|
|
|
|
+ v.prs.learners = make(map[uint64]*Progress)
|
|
|
for i := 0; i < size; i++ {
|
|
for i := 0; i < size; i++ {
|
|
|
if _, ok := learners[peerAddrs[i]]; ok {
|
|
if _, ok := learners[peerAddrs[i]]; ok {
|
|
|
- v.learnerPrs[peerAddrs[i]] = &Progress{IsLearner: true}
|
|
|
|
|
|
|
+ v.prs.learners[peerAddrs[i]] = &Progress{IsLearner: true}
|
|
|
} else {
|
|
} else {
|
|
|
- v.prs[peerAddrs[i]] = &Progress{}
|
|
|
|
|
|
|
+ v.prs.nodes[peerAddrs[i]] = &Progress{}
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
v.reset(v.Term)
|
|
v.reset(v.Term)
|