Browse Source

raft: extract progress tracking into own component

The Progress maps contain both the active configuration and information
about the replication status. By pulling it into its own component, this
becomes easier to unit test and also clarifies the code, which will see
changes as etcd-io/etcd#7625 is addressed.

More functionality will move into `prs` in self-contained follow-up commits.
Tobias Schottdorf 6 years ago
parent
commit
dbac67e7a8
7 changed files with 174 additions and 152 deletions
  1. 45 1
      raft/progress.go
  2. 34 56
      raft/raft.go
  3. 3 3
      raft/raft_flow_control_test.go
  4. 29 29
      raft/raft_snap_test.go
  5. 59 59
      raft/raft_test.go
  6. 2 2
      raft/rawnode.go
  7. 2 2
      raft/status.go

+ 45 - 1
raft/progress.go

@@ -14,7 +14,10 @@
 
 
 package raft
 package raft
 
 
-import "fmt"
+import (
+	"fmt"
+	"sort"
+)
 
 
 const (
 const (
 	ProgressStateProbe ProgressStateType = iota
 	ProgressStateProbe ProgressStateType = iota
@@ -283,3 +286,44 @@ func (in *inflights) reset() {
 	in.count = 0
 	in.count = 0
 	in.start = 0
 	in.start = 0
 }
 }
+
+// prs tracks the currently active configuration and the information known about
+// the nodes and learners in it. In particular, it tracks the match index for
+// each peer which in turn allows reasoning about the committed index.
+type prs struct {
+	nodes    map[uint64]*Progress
+	learners map[uint64]*Progress
+	matchBuf uint64Slice
+}
+
+func makePRS() prs {
+	return prs{
+		nodes:    map[uint64]*Progress{},
+		learners: map[uint64]*Progress{},
+	}
+}
+
+func (p *prs) quorum() int {
+	return len(p.nodes)/2 + 1
+}
+
+func (p *prs) committed() uint64 {
+	// Preserving matchBuf across calls is an optimization
+	// used to avoid allocating a new slice on each call.
+	if cap(p.matchBuf) < len(p.nodes) {
+		p.matchBuf = make(uint64Slice, len(p.nodes))
+	}
+	p.matchBuf = p.matchBuf[:len(p.nodes)]
+	idx := 0
+	for _, pr := range p.nodes {
+		p.matchBuf[idx] = pr.Match
+		idx++
+	}
+	sort.Sort(&p.matchBuf)
+	return p.matchBuf[len(p.matchBuf)-p.quorum()]
+}
+
+func (p *prs) removeAny(id uint64) {
+	delete(p.nodes, id)
+	delete(p.learners, id)
+}

+ 34 - 56
raft/raft.go

@@ -262,9 +262,7 @@ type raft struct {
 	maxMsgSize         uint64
 	maxMsgSize         uint64
 	maxUncommittedSize uint64
 	maxUncommittedSize uint64
 	maxInflight        int
 	maxInflight        int
-	prs                map[uint64]*Progress
-	learnerPrs         map[uint64]*Progress
-	matchBuf           uint64Slice
+	prs                prs
 
 
 	state StateType
 	state StateType
 
 
@@ -350,8 +348,7 @@ func newRaft(c *Config) *raft {
 		maxMsgSize:                c.MaxSizePerMsg,
 		maxMsgSize:                c.MaxSizePerMsg,
 		maxInflight:               c.MaxInflightMsgs,
 		maxInflight:               c.MaxInflightMsgs,
 		maxUncommittedSize:        c.MaxUncommittedEntriesSize,
 		maxUncommittedSize:        c.MaxUncommittedEntriesSize,
-		prs:                       make(map[uint64]*Progress),
-		learnerPrs:                make(map[uint64]*Progress),
+		prs:                       makePRS(),
 		electionTimeout:           c.ElectionTick,
 		electionTimeout:           c.ElectionTick,
 		heartbeatTimeout:          c.HeartbeatTick,
 		heartbeatTimeout:          c.HeartbeatTick,
 		logger:                    c.Logger,
 		logger:                    c.Logger,
@@ -361,13 +358,13 @@ func newRaft(c *Config) *raft {
 		disableProposalForwarding: c.DisableProposalForwarding,
 		disableProposalForwarding: c.DisableProposalForwarding,
 	}
 	}
 	for _, p := range peers {
 	for _, p := range peers {
-		r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
+		r.prs.nodes[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
 	}
 	}
 	for _, p := range learners {
 	for _, p := range learners {
-		if _, ok := r.prs[p]; ok {
+		if _, ok := r.prs.nodes[p]; ok {
 			panic(fmt.Sprintf("node %x is in both learner and peer list", p))
 			panic(fmt.Sprintf("node %x is in both learner and peer list", p))
 		}
 		}
-		r.learnerPrs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight), IsLearner: true}
+		r.prs.learners[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight), IsLearner: true}
 		if r.id == p {
 		if r.id == p {
 			r.isLearner = true
 			r.isLearner = true
 		}
 		}
@@ -403,11 +400,9 @@ func (r *raft) hardState() pb.HardState {
 	}
 	}
 }
 }
 
 
-func (r *raft) quorum() int { return len(r.prs)/2 + 1 }
-
 func (r *raft) nodes() []uint64 {
 func (r *raft) nodes() []uint64 {
-	nodes := make([]uint64, 0, len(r.prs))
-	for id := range r.prs {
+	nodes := make([]uint64, 0, len(r.prs.nodes))
+	for id := range r.prs.nodes {
 		nodes = append(nodes, id)
 		nodes = append(nodes, id)
 	}
 	}
 	sort.Sort(uint64Slice(nodes))
 	sort.Sort(uint64Slice(nodes))
@@ -415,8 +410,8 @@ func (r *raft) nodes() []uint64 {
 }
 }
 
 
 func (r *raft) learnerNodes() []uint64 {
 func (r *raft) learnerNodes() []uint64 {
-	nodes := make([]uint64, 0, len(r.learnerPrs))
-	for id := range r.learnerPrs {
+	nodes := make([]uint64, 0, len(r.prs.learners))
+	for id := range r.prs.learners {
 		nodes = append(nodes, id)
 		nodes = append(nodes, id)
 	}
 	}
 	sort.Sort(uint64Slice(nodes))
 	sort.Sort(uint64Slice(nodes))
@@ -458,11 +453,11 @@ func (r *raft) send(m pb.Message) {
 }
 }
 
 
 func (r *raft) getProgress(id uint64) *Progress {
 func (r *raft) getProgress(id uint64) *Progress {
-	if pr, ok := r.prs[id]; ok {
+	if pr, ok := r.prs.nodes[id]; ok {
 		return pr
 		return pr
 	}
 	}
 
 
-	return r.learnerPrs[id]
+	return r.prs.learners[id]
 }
 }
 
 
 // sendAppend sends an append RPC with new entries (if any) and the
 // sendAppend sends an append RPC with new entries (if any) and the
@@ -558,11 +553,11 @@ func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
 }
 }
 
 
 func (r *raft) forEachProgress(f func(id uint64, pr *Progress)) {
 func (r *raft) forEachProgress(f func(id uint64, pr *Progress)) {
-	for id, pr := range r.prs {
+	for id, pr := range r.prs.nodes {
 		f(id, pr)
 		f(id, pr)
 	}
 	}
 
 
-	for id, pr := range r.learnerPrs {
+	for id, pr := range r.prs.learners {
 		f(id, pr)
 		f(id, pr)
 	}
 	}
 }
 }
@@ -602,19 +597,7 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
 // the commit index changed (in which case the caller should call
 // the commit index changed (in which case the caller should call
 // r.bcastAppend).
 // r.bcastAppend).
 func (r *raft) maybeCommit() bool {
 func (r *raft) maybeCommit() bool {
-	// Preserving matchBuf across calls is an optimization
-	// used to avoid allocating a new slice on each call.
-	if cap(r.matchBuf) < len(r.prs) {
-		r.matchBuf = make(uint64Slice, len(r.prs))
-	}
-	r.matchBuf = r.matchBuf[:len(r.prs)]
-	idx := 0
-	for _, p := range r.prs {
-		r.matchBuf[idx] = p.Match
-		idx++
-	}
-	sort.Sort(&r.matchBuf)
-	mci := r.matchBuf[len(r.matchBuf)-r.quorum()]
+	mci := r.prs.committed()
 	return r.raftLog.maybeCommit(mci, r.Term)
 	return r.raftLog.maybeCommit(mci, r.Term)
 }
 }
 
 
@@ -755,7 +738,7 @@ func (r *raft) becomeLeader() {
 	// (perhaps after having received a snapshot as a result). The leader is
 	// (perhaps after having received a snapshot as a result). The leader is
 	// trivially in this state. Note that r.reset() has initialized this
 	// trivially in this state. Note that r.reset() has initialized this
 	// progress with the last index already.
 	// progress with the last index already.
-	r.prs[r.id].becomeReplicate()
+	r.prs.nodes[r.id].becomeReplicate()
 
 
 	// Conservatively set the pendingConfIndex to the last index in the
 	// Conservatively set the pendingConfIndex to the last index in the
 	// log. There may or may not be a pending config change, but it's
 	// log. There may or may not be a pending config change, but it's
@@ -790,7 +773,7 @@ func (r *raft) campaign(t CampaignType) {
 		voteMsg = pb.MsgVote
 		voteMsg = pb.MsgVote
 		term = r.Term
 		term = r.Term
 	}
 	}
-	if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
+	if r.prs.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
 		// We won the election after voting for ourselves (which must mean that
 		// We won the election after voting for ourselves (which must mean that
 		// this is a single-node cluster). Advance to the next state.
 		// this is a single-node cluster). Advance to the next state.
 		if t == campaignPreElection {
 		if t == campaignPreElection {
@@ -800,7 +783,7 @@ func (r *raft) campaign(t CampaignType) {
 		}
 		}
 		return
 		return
 	}
 	}
-	for id := range r.prs {
+	for id := range r.prs.nodes {
 		if id == r.id {
 		if id == r.id {
 			continue
 			continue
 		}
 		}
@@ -994,7 +977,7 @@ func stepLeader(r *raft, m pb.Message) error {
 		if len(m.Entries) == 0 {
 		if len(m.Entries) == 0 {
 			r.logger.Panicf("%x stepped empty MsgProp", r.id)
 			r.logger.Panicf("%x stepped empty MsgProp", r.id)
 		}
 		}
-		if _, ok := r.prs[r.id]; !ok {
+		if _, ok := r.prs.nodes[r.id]; !ok {
 			// If we are not currently a member of the range (i.e. this node
 			// If we are not currently a member of the range (i.e. this node
 			// was removed from the configuration while serving as leader),
 			// was removed from the configuration while serving as leader),
 			// drop any new proposals.
 			// drop any new proposals.
@@ -1024,7 +1007,7 @@ func stepLeader(r *raft, m pb.Message) error {
 		r.bcastAppend()
 		r.bcastAppend()
 		return nil
 		return nil
 	case pb.MsgReadIndex:
 	case pb.MsgReadIndex:
-		if r.quorum() > 1 {
+		if r.prs.quorum() > 1 {
 			if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
 			if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
 				// Reject read only request when this leader has not committed any log entry at its term.
 				// Reject read only request when this leader has not committed any log entry at its term.
 				return nil
 				return nil
@@ -1134,7 +1117,7 @@ func stepLeader(r *raft, m pb.Message) error {
 		}
 		}
 
 
 		ackCount := r.readOnly.recvAck(m)
 		ackCount := r.readOnly.recvAck(m)
-		if ackCount < r.quorum() {
+		if ackCount < r.prs.quorum() {
 			return nil
 			return nil
 		}
 		}
 
 
@@ -1232,8 +1215,8 @@ func stepCandidate(r *raft, m pb.Message) error {
 		r.handleSnapshot(m)
 		r.handleSnapshot(m)
 	case myVoteRespType:
 	case myVoteRespType:
 		gr := r.poll(m.From, m.Type, !m.Reject)
 		gr := r.poll(m.From, m.Type, !m.Reject)
-		r.logger.Infof("%x [quorum:%d] has received %d %s votes and %d vote rejections", r.id, r.quorum(), gr, m.Type, len(r.votes)-gr)
-		switch r.quorum() {
+		r.logger.Infof("%x [quorum:%d] has received %d %s votes and %d vote rejections", r.id, r.prs.quorum(), gr, m.Type, len(r.votes)-gr)
+		switch r.prs.quorum() {
 		case gr:
 		case gr:
 			if r.state == StatePreCandidate {
 			if r.state == StatePreCandidate {
 				r.campaign(campaignElection)
 				r.campaign(campaignElection)
@@ -1370,8 +1353,8 @@ func (r *raft) restore(s pb.Snapshot) bool {
 		r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
 		r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
 
 
 	r.raftLog.restore(s)
 	r.raftLog.restore(s)
-	r.prs = make(map[uint64]*Progress)
-	r.learnerPrs = make(map[uint64]*Progress)
+	r.prs.nodes = make(map[uint64]*Progress)
+	r.prs.learners = make(map[uint64]*Progress)
 	r.restoreNode(s.Metadata.ConfState.Nodes, false)
 	r.restoreNode(s.Metadata.ConfState.Nodes, false)
 	r.restoreNode(s.Metadata.ConfState.Learners, true)
 	r.restoreNode(s.Metadata.ConfState.Learners, true)
 	return true
 	return true
@@ -1392,7 +1375,7 @@ func (r *raft) restoreNode(nodes []uint64, isLearner bool) {
 // promotable indicates whether state machine can be promoted to leader,
 // promotable indicates whether state machine can be promoted to leader,
 // which is true when its own id is in progress list.
 // which is true when its own id is in progress list.
 func (r *raft) promotable() bool {
 func (r *raft) promotable() bool {
-	_, ok := r.prs[r.id]
+	_, ok := r.prs.nodes[r.id]
 	return ok
 	return ok
 }
 }
 
 
@@ -1422,9 +1405,9 @@ func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) {
 		}
 		}
 
 
 		// change Learner to Voter, use origin Learner progress
 		// change Learner to Voter, use origin Learner progress
-		delete(r.learnerPrs, id)
+		delete(r.prs.learners, id)
 		pr.IsLearner = false
 		pr.IsLearner = false
-		r.prs[id] = pr
+		r.prs.nodes[id] = pr
 	}
 	}
 
 
 	if r.id == id {
 	if r.id == id {
@@ -1439,10 +1422,10 @@ func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) {
 }
 }
 
 
 func (r *raft) removeNode(id uint64) {
 func (r *raft) removeNode(id uint64) {
-	r.delProgress(id)
+	r.prs.removeAny(id)
 
 
 	// do not try to commit or abort transferring if there is no nodes in the cluster.
 	// do not try to commit or abort transferring if there is no nodes in the cluster.
-	if len(r.prs) == 0 && len(r.learnerPrs) == 0 {
+	if len(r.prs.nodes) == 0 && len(r.prs.learners) == 0 {
 		return
 		return
 	}
 	}
 
 
@@ -1459,20 +1442,15 @@ func (r *raft) removeNode(id uint64) {
 
 
 func (r *raft) setProgress(id, match, next uint64, isLearner bool) {
 func (r *raft) setProgress(id, match, next uint64, isLearner bool) {
 	if !isLearner {
 	if !isLearner {
-		delete(r.learnerPrs, id)
-		r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
+		delete(r.prs.learners, id)
+		r.prs.nodes[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
 		return
 		return
 	}
 	}
 
 
-	if _, ok := r.prs[id]; ok {
+	if _, ok := r.prs.nodes[id]; ok {
 		panic(fmt.Sprintf("%x unexpected changing from voter to learner for %x", r.id, id))
 		panic(fmt.Sprintf("%x unexpected changing from voter to learner for %x", r.id, id))
 	}
 	}
-	r.learnerPrs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), IsLearner: true}
-}
-
-func (r *raft) delProgress(id uint64) {
-	delete(r.prs, id)
-	delete(r.learnerPrs, id)
+	r.prs.learners[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), IsLearner: true}
 }
 }
 
 
 func (r *raft) loadState(state pb.HardState) {
 func (r *raft) loadState(state pb.HardState) {
@@ -1515,7 +1493,7 @@ func (r *raft) checkQuorumActive() bool {
 		pr.RecentActive = false
 		pr.RecentActive = false
 	})
 	})
 
 
-	return act >= r.quorum()
+	return act >= r.prs.quorum()
 }
 }
 
 
 func (r *raft) sendTimeoutNow(to uint64) {
 func (r *raft) sendTimeoutNow(to uint64) {

+ 3 - 3
raft/raft_flow_control_test.go

@@ -29,7 +29,7 @@ func TestMsgAppFlowControlFull(t *testing.T) {
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 
 
-	pr2 := r.prs[2]
+	pr2 := r.prs.nodes[2]
 	// force the progress to be in replicate state
 	// force the progress to be in replicate state
 	pr2.becomeReplicate()
 	pr2.becomeReplicate()
 	// fill in the inflights window
 	// fill in the inflights window
@@ -65,7 +65,7 @@ func TestMsgAppFlowControlMoveForward(t *testing.T) {
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 
 
-	pr2 := r.prs[2]
+	pr2 := r.prs.nodes[2]
 	// force the progress to be in replicate state
 	// force the progress to be in replicate state
 	pr2.becomeReplicate()
 	pr2.becomeReplicate()
 	// fill in the inflights window
 	// fill in the inflights window
@@ -110,7 +110,7 @@ func TestMsgAppFlowControlRecvHeartbeat(t *testing.T) {
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 
 
-	pr2 := r.prs[2]
+	pr2 := r.prs.nodes[2]
 	// force the progress to be in replicate state
 	// force the progress to be in replicate state
 	pr2.becomeReplicate()
 	pr2.becomeReplicate()
 	// fill in the inflights window
 	// fill in the inflights window

+ 29 - 29
raft/raft_snap_test.go

@@ -40,11 +40,11 @@ func TestSendingSnapshotSetPendingSnapshot(t *testing.T) {
 
 
 	// force set the next of node 2, so that
 	// force set the next of node 2, so that
 	// node 2 needs a snapshot
 	// node 2 needs a snapshot
-	sm.prs[2].Next = sm.raftLog.firstIndex()
+	sm.prs.nodes[2].Next = sm.raftLog.firstIndex()
 
 
-	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs[2].Next - 1, Reject: true})
-	if sm.prs[2].PendingSnapshot != 11 {
-		t.Fatalf("PendingSnapshot = %d, want 11", sm.prs[2].PendingSnapshot)
+	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: sm.prs.nodes[2].Next - 1, Reject: true})
+	if sm.prs.nodes[2].PendingSnapshot != 11 {
+		t.Fatalf("PendingSnapshot = %d, want 11", sm.prs.nodes[2].PendingSnapshot)
 	}
 	}
 }
 }
 
 
@@ -56,7 +56,7 @@ func TestPendingSnapshotPauseReplication(t *testing.T) {
 	sm.becomeCandidate()
 	sm.becomeCandidate()
 	sm.becomeLeader()
 	sm.becomeLeader()
 
 
-	sm.prs[2].becomeSnapshot(11)
+	sm.prs.nodes[2].becomeSnapshot(11)
 
 
 	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")}}})
 	msgs := sm.readMessages()
 	msgs := sm.readMessages()
@@ -73,18 +73,18 @@ func TestSnapshotFailure(t *testing.T) {
 	sm.becomeCandidate()
 	sm.becomeCandidate()
 	sm.becomeLeader()
 	sm.becomeLeader()
 
 
-	sm.prs[2].Next = 1
-	sm.prs[2].becomeSnapshot(11)
+	sm.prs.nodes[2].Next = 1
+	sm.prs.nodes[2].becomeSnapshot(11)
 
 
 	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgSnapStatus, Reject: true})
 	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgSnapStatus, Reject: true})
-	if sm.prs[2].PendingSnapshot != 0 {
-		t.Fatalf("PendingSnapshot = %d, want 0", sm.prs[2].PendingSnapshot)
+	if sm.prs.nodes[2].PendingSnapshot != 0 {
+		t.Fatalf("PendingSnapshot = %d, want 0", sm.prs.nodes[2].PendingSnapshot)
 	}
 	}
-	if sm.prs[2].Next != 1 {
-		t.Fatalf("Next = %d, want 1", sm.prs[2].Next)
+	if sm.prs.nodes[2].Next != 1 {
+		t.Fatalf("Next = %d, want 1", sm.prs.nodes[2].Next)
 	}
 	}
-	if !sm.prs[2].Paused {
-		t.Errorf("Paused = %v, want true", sm.prs[2].Paused)
+	if !sm.prs.nodes[2].Paused {
+		t.Errorf("Paused = %v, want true", sm.prs.nodes[2].Paused)
 	}
 	}
 }
 }
 
 
@@ -96,18 +96,18 @@ func TestSnapshotSucceed(t *testing.T) {
 	sm.becomeCandidate()
 	sm.becomeCandidate()
 	sm.becomeLeader()
 	sm.becomeLeader()
 
 
-	sm.prs[2].Next = 1
-	sm.prs[2].becomeSnapshot(11)
+	sm.prs.nodes[2].Next = 1
+	sm.prs.nodes[2].becomeSnapshot(11)
 
 
 	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgSnapStatus, Reject: false})
 	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgSnapStatus, Reject: false})
-	if sm.prs[2].PendingSnapshot != 0 {
-		t.Fatalf("PendingSnapshot = %d, want 0", sm.prs[2].PendingSnapshot)
+	if sm.prs.nodes[2].PendingSnapshot != 0 {
+		t.Fatalf("PendingSnapshot = %d, want 0", sm.prs.nodes[2].PendingSnapshot)
 	}
 	}
-	if sm.prs[2].Next != 12 {
-		t.Fatalf("Next = %d, want 12", sm.prs[2].Next)
+	if sm.prs.nodes[2].Next != 12 {
+		t.Fatalf("Next = %d, want 12", sm.prs.nodes[2].Next)
 	}
 	}
-	if !sm.prs[2].Paused {
-		t.Errorf("Paused = %v, want true", sm.prs[2].Paused)
+	if !sm.prs.nodes[2].Paused {
+		t.Errorf("Paused = %v, want true", sm.prs.nodes[2].Paused)
 	}
 	}
 }
 }
 
 
@@ -206,7 +206,7 @@ func TestSnapshotSucceedViaAppResp(t *testing.T) {
 	mustSend(n2, n1, pb.MsgAppResp)
 	mustSend(n2, n1, pb.MsgAppResp)
 
 
 	// Leader has correct state for follower.
 	// Leader has correct state for follower.
-	pr := n1.prs[2]
+	pr := n1.prs.nodes[2]
 	if pr.State != ProgressStateReplicate {
 	if pr.State != ProgressStateReplicate {
 		t.Fatalf("unexpected state %v", pr)
 		t.Fatalf("unexpected state %v", pr)
 	}
 	}
@@ -227,23 +227,23 @@ func TestSnapshotAbort(t *testing.T) {
 	sm.becomeCandidate()
 	sm.becomeCandidate()
 	sm.becomeLeader()
 	sm.becomeLeader()
 
 
-	sm.prs[2].Next = 1
-	sm.prs[2].becomeSnapshot(11)
+	sm.prs.nodes[2].Next = 1
+	sm.prs.nodes[2].becomeSnapshot(11)
 
 
 	// A successful msgAppResp that has a higher/equal index than the
 	// A successful msgAppResp that has a higher/equal index than the
 	// pending snapshot should abort the pending snapshot.
 	// pending snapshot should abort the pending snapshot.
 	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: 11})
 	sm.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: 11})
-	if sm.prs[2].PendingSnapshot != 0 {
-		t.Fatalf("PendingSnapshot = %d, want 0", sm.prs[2].PendingSnapshot)
+	if sm.prs.nodes[2].PendingSnapshot != 0 {
+		t.Fatalf("PendingSnapshot = %d, want 0", sm.prs.nodes[2].PendingSnapshot)
 	}
 	}
 	// The follower entered ProgressStateReplicate and the leader send an append
 	// The follower entered ProgressStateReplicate and the leader send an append
 	// and optimistically updated the progress (so we see 13 instead of 12).
 	// and optimistically updated the progress (so we see 13 instead of 12).
 	// There is something to append because the leader appended an empty entry
 	// There is something to append because the leader appended an empty entry
 	// to the log at index 12 when it assumed leadership.
 	// to the log at index 12 when it assumed leadership.
-	if sm.prs[2].Next != 13 {
-		t.Fatalf("Next = %d, want 13", sm.prs[2].Next)
+	if sm.prs.nodes[2].Next != 13 {
+		t.Fatalf("Next = %d, want 13", sm.prs.nodes[2].Next)
 	}
 	}
-	if n := sm.prs[2].ins.count; n != 1 {
+	if n := sm.prs.nodes[2].ins.count; n != 1 {
 		t.Fatalf("expected an inflight message, got %d", n)
 		t.Fatalf("expected an inflight message, got %d", n)
 	}
 	}
 }
 }

+ 59 - 59
raft/raft_test.go

@@ -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)

+ 2 - 2
raft/rawnode.go

@@ -257,12 +257,12 @@ const (
 // WithProgress is a helper to introspect the Progress for this node and its
 // WithProgress is a helper to introspect the Progress for this node and its
 // peers.
 // peers.
 func (rn *RawNode) WithProgress(visitor func(id uint64, typ ProgressType, pr Progress)) {
 func (rn *RawNode) WithProgress(visitor func(id uint64, typ ProgressType, pr Progress)) {
-	for id, pr := range rn.raft.prs {
+	for id, pr := range rn.raft.prs.nodes {
 		pr := *pr
 		pr := *pr
 		pr.ins = nil
 		pr.ins = nil
 		visitor(id, ProgressTypePeer, pr)
 		visitor(id, ProgressTypePeer, pr)
 	}
 	}
-	for id, pr := range rn.raft.learnerPrs {
+	for id, pr := range rn.raft.prs.learners {
 		pr := *pr
 		pr := *pr
 		pr.ins = nil
 		pr.ins = nil
 		visitor(id, ProgressTypeLearner, pr)
 		visitor(id, ProgressTypeLearner, pr)

+ 2 - 2
raft/status.go

@@ -34,11 +34,11 @@ type Status struct {
 
 
 func getProgressCopy(r *raft) map[uint64]Progress {
 func getProgressCopy(r *raft) map[uint64]Progress {
 	prs := make(map[uint64]Progress)
 	prs := make(map[uint64]Progress)
-	for id, p := range r.prs {
+	for id, p := range r.prs.nodes {
 		prs[id] = *p
 		prs[id] = *p
 	}
 	}
 
 
-	for id, p := range r.learnerPrs {
+	for id, p := range r.prs.learners {
 		prs[id] = *p
 		prs[id] = *p
 	}
 	}
 	return prs
 	return prs