mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
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.
This commit is contained in:
parent
71881a423f
commit
dbac67e7a8
@ -14,7 +14,10 @@
|
||||
|
||||
package raft
|
||||
|
||||
import "fmt"
|
||||
import (
|
||||
"fmt"
|
||||
"sort"
|
||||
)
|
||||
|
||||
const (
|
||||
ProgressStateProbe ProgressStateType = iota
|
||||
@ -283,3 +286,44 @@ func (in *inflights) reset() {
|
||||
in.count = 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)
|
||||
}
|
||||
|
90
raft/raft.go
90
raft/raft.go
@ -262,9 +262,7 @@ type raft struct {
|
||||
maxMsgSize uint64
|
||||
maxUncommittedSize uint64
|
||||
maxInflight int
|
||||
prs map[uint64]*Progress
|
||||
learnerPrs map[uint64]*Progress
|
||||
matchBuf uint64Slice
|
||||
prs prs
|
||||
|
||||
state StateType
|
||||
|
||||
@ -350,8 +348,7 @@ func newRaft(c *Config) *raft {
|
||||
maxMsgSize: c.MaxSizePerMsg,
|
||||
maxInflight: c.MaxInflightMsgs,
|
||||
maxUncommittedSize: c.MaxUncommittedEntriesSize,
|
||||
prs: make(map[uint64]*Progress),
|
||||
learnerPrs: make(map[uint64]*Progress),
|
||||
prs: makePRS(),
|
||||
electionTimeout: c.ElectionTick,
|
||||
heartbeatTimeout: c.HeartbeatTick,
|
||||
logger: c.Logger,
|
||||
@ -361,13 +358,13 @@ func newRaft(c *Config) *raft {
|
||||
disableProposalForwarding: c.DisableProposalForwarding,
|
||||
}
|
||||
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 {
|
||||
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))
|
||||
}
|
||||
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 {
|
||||
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 {
|
||||
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)
|
||||
}
|
||||
sort.Sort(uint64Slice(nodes))
|
||||
@ -415,8 +410,8 @@ func (r *raft) nodes() []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)
|
||||
}
|
||||
sort.Sort(uint64Slice(nodes))
|
||||
@ -458,11 +453,11 @@ func (r *raft) send(m pb.Message) {
|
||||
}
|
||||
|
||||
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 r.learnerPrs[id]
|
||||
return r.prs.learners[id]
|
||||
}
|
||||
|
||||
// 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)) {
|
||||
for id, pr := range r.prs {
|
||||
for id, pr := range r.prs.nodes {
|
||||
f(id, pr)
|
||||
}
|
||||
|
||||
for id, pr := range r.learnerPrs {
|
||||
for id, pr := range r.prs.learners {
|
||||
f(id, pr)
|
||||
}
|
||||
}
|
||||
@ -602,19 +597,7 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
|
||||
// the commit index changed (in which case the caller should call
|
||||
// r.bcastAppend).
|
||||
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)
|
||||
}
|
||||
|
||||
@ -755,7 +738,7 @@ func (r *raft) becomeLeader() {
|
||||
// (perhaps after having received a snapshot as a result). The leader is
|
||||
// trivially in this state. Note that r.reset() has initialized this
|
||||
// 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
|
||||
// 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
|
||||
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
|
||||
// this is a single-node cluster). Advance to the next state.
|
||||
if t == campaignPreElection {
|
||||
@ -800,7 +783,7 @@ func (r *raft) campaign(t CampaignType) {
|
||||
}
|
||||
return
|
||||
}
|
||||
for id := range r.prs {
|
||||
for id := range r.prs.nodes {
|
||||
if id == r.id {
|
||||
continue
|
||||
}
|
||||
@ -994,7 +977,7 @@ func stepLeader(r *raft, m pb.Message) error {
|
||||
if len(m.Entries) == 0 {
|
||||
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
|
||||
// was removed from the configuration while serving as leader),
|
||||
// drop any new proposals.
|
||||
@ -1024,7 +1007,7 @@ func stepLeader(r *raft, m pb.Message) error {
|
||||
r.bcastAppend()
|
||||
return nil
|
||||
case pb.MsgReadIndex:
|
||||
if r.quorum() > 1 {
|
||||
if r.prs.quorum() > 1 {
|
||||
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.
|
||||
return nil
|
||||
@ -1134,7 +1117,7 @@ func stepLeader(r *raft, m pb.Message) error {
|
||||
}
|
||||
|
||||
ackCount := r.readOnly.recvAck(m)
|
||||
if ackCount < r.quorum() {
|
||||
if ackCount < r.prs.quorum() {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -1232,8 +1215,8 @@ func stepCandidate(r *raft, m pb.Message) error {
|
||||
r.handleSnapshot(m)
|
||||
case myVoteRespType:
|
||||
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:
|
||||
if r.state == StatePreCandidate {
|
||||
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.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.Learners, 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,
|
||||
// which is true when its own id is in progress list.
|
||||
func (r *raft) promotable() bool {
|
||||
_, ok := r.prs[r.id]
|
||||
_, ok := r.prs.nodes[r.id]
|
||||
return ok
|
||||
}
|
||||
|
||||
@ -1422,9 +1405,9 @@ func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) {
|
||||
}
|
||||
|
||||
// change Learner to Voter, use origin Learner progress
|
||||
delete(r.learnerPrs, id)
|
||||
delete(r.prs.learners, id)
|
||||
pr.IsLearner = false
|
||||
r.prs[id] = pr
|
||||
r.prs.nodes[id] = pr
|
||||
}
|
||||
|
||||
if r.id == id {
|
||||
@ -1439,10 +1422,10 @@ func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) {
|
||||
}
|
||||
|
||||
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.
|
||||
if len(r.prs) == 0 && len(r.learnerPrs) == 0 {
|
||||
if len(r.prs.nodes) == 0 && len(r.prs.learners) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
@ -1459,20 +1442,15 @@ func (r *raft) removeNode(id uint64) {
|
||||
|
||||
func (r *raft) setProgress(id, match, next uint64, isLearner bool) {
|
||||
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
|
||||
}
|
||||
|
||||
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))
|
||||
}
|
||||
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) {
|
||||
@ -1515,7 +1493,7 @@ func (r *raft) checkQuorumActive() bool {
|
||||
pr.RecentActive = false
|
||||
})
|
||||
|
||||
return act >= r.quorum()
|
||||
return act >= r.prs.quorum()
|
||||
}
|
||||
|
||||
func (r *raft) sendTimeoutNow(to uint64) {
|
||||
|
@ -29,7 +29,7 @@ func TestMsgAppFlowControlFull(t *testing.T) {
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
|
||||
pr2 := r.prs[2]
|
||||
pr2 := r.prs.nodes[2]
|
||||
// force the progress to be in replicate state
|
||||
pr2.becomeReplicate()
|
||||
// fill in the inflights window
|
||||
@ -65,7 +65,7 @@ func TestMsgAppFlowControlMoveForward(t *testing.T) {
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
|
||||
pr2 := r.prs[2]
|
||||
pr2 := r.prs.nodes[2]
|
||||
// force the progress to be in replicate state
|
||||
pr2.becomeReplicate()
|
||||
// fill in the inflights window
|
||||
@ -110,7 +110,7 @@ func TestMsgAppFlowControlRecvHeartbeat(t *testing.T) {
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
|
||||
pr2 := r.prs[2]
|
||||
pr2 := r.prs.nodes[2]
|
||||
// force the progress to be in replicate state
|
||||
pr2.becomeReplicate()
|
||||
// fill in the inflights window
|
||||
|
@ -40,11 +40,11 @@ func TestSendingSnapshotSetPendingSnapshot(t *testing.T) {
|
||||
|
||||
// force set the next of node 2, so that
|
||||
// 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.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")}}})
|
||||
msgs := sm.readMessages()
|
||||
@ -73,18 +73,18 @@ func TestSnapshotFailure(t *testing.T) {
|
||||
sm.becomeCandidate()
|
||||
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})
|
||||
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.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})
|
||||
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)
|
||||
|
||||
// Leader has correct state for follower.
|
||||
pr := n1.prs[2]
|
||||
pr := n1.prs.nodes[2]
|
||||
if pr.State != ProgressStateReplicate {
|
||||
t.Fatalf("unexpected state %v", pr)
|
||||
}
|
||||
@ -227,23 +227,23 @@ func TestSnapshotAbort(t *testing.T) {
|
||||
sm.becomeCandidate()
|
||||
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
|
||||
// pending snapshot should abort the pending snapshot.
|
||||
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
|
||||
// and optimistically updated the progress (so we see 13 instead of 12).
|
||||
// There is something to append because the leader appended an empty entry
|
||||
// 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)
|
||||
}
|
||||
}
|
||||
|
@ -271,12 +271,12 @@ func TestProgressLeader(t *testing.T) {
|
||||
r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
|
||||
r.becomeCandidate()
|
||||
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.
|
||||
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[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)
|
||||
}
|
||||
if err := r.Step(propMsg); err != nil {
|
||||
@ -291,17 +291,17 @@ func TestProgressResumeByHeartbeatResp(t *testing.T) {
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
|
||||
r.prs[2].Paused = true
|
||||
r.prs.nodes[2].Paused = true
|
||||
|
||||
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})
|
||||
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()
|
||||
|
||||
// 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))
|
||||
for i := 0; i < 10; i++ {
|
||||
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.
|
||||
const numFollowers = 2
|
||||
r.prs[2].becomeReplicate()
|
||||
r.prs[3].becomeReplicate()
|
||||
r.prs.nodes[2].becomeReplicate()
|
||||
r.prs.nodes[3].becomeReplicate()
|
||||
r.uncommittedSize = 0
|
||||
|
||||
// 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)
|
||||
setRandomizedElectionTimeout(b, b.electionTimeout+1)
|
||||
// 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() {
|
||||
t.Fatalf("promotable = %v, want false", b.promotable())
|
||||
@ -2631,7 +2631,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[2]
|
||||
p := sm.prs.nodes[2]
|
||||
if 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})
|
||||
}
|
||||
// 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
|
||||
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})
|
||||
msgs := sm.readMessages()
|
||||
@ -2688,8 +2688,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[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 {
|
||||
if m.Type != pb.MsgHeartbeat {
|
||||
@ -2775,11 +2775,11 @@ func TestLeaderIncreaseNext(t *testing.T) {
|
||||
sm.raftLog.append(previousEnts...)
|
||||
sm.becomeCandidate()
|
||||
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")}}})
|
||||
|
||||
p := sm.prs[2]
|
||||
p := sm.prs.nodes[2]
|
||||
if 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.becomeLeader()
|
||||
r.readMessages()
|
||||
r.prs[2].becomeProbe()
|
||||
r.prs.nodes[2].becomeProbe()
|
||||
|
||||
// each round is a heartbeat
|
||||
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++ {
|
||||
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
||||
@ -2825,8 +2825,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[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
|
||||
@ -2848,8 +2848,8 @@ func TestSendAppendForProgressProbe(t *testing.T) {
|
||||
if 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.becomeLeader()
|
||||
r.readMessages()
|
||||
r.prs[2].becomeReplicate()
|
||||
r.prs.nodes[2].becomeReplicate()
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
||||
@ -2875,7 +2875,7 @@ func TestSendAppendForProgressSnapshot(t *testing.T) {
|
||||
r.becomeCandidate()
|
||||
r.becomeLeader()
|
||||
r.readMessages()
|
||||
r.prs[2].becomeSnapshot(10)
|
||||
r.prs.nodes[2].becomeSnapshot(10)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
|
||||
@ -2896,17 +2896,17 @@ func TestRecvMsgUnreachable(t *testing.T) {
|
||||
r.becomeLeader()
|
||||
r.readMessages()
|
||||
// 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})
|
||||
|
||||
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)
|
||||
}
|
||||
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 {
|
||||
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()
|
||||
|
||||
// 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()
|
||||
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
|
||||
// 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")}}})
|
||||
|
||||
@ -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.
|
||||
for {
|
||||
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
|
||||
if lead.prs[3].RecentActive {
|
||||
if lead.prs.nodes[3].RecentActive {
|
||||
break
|
||||
}
|
||||
}
|
||||
@ -3303,8 +3303,8 @@ func TestAddLearner(t *testing.T) {
|
||||
if !reflect.DeepEqual(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()
|
||||
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.
|
||||
@ -3641,8 +3641,8 @@ func TestLeaderTransferAfterSnapshot(t *testing.T) {
|
||||
nt.storage[1].Compact(lead.raftLog.applied)
|
||||
|
||||
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.
|
||||
@ -3721,8 +3721,8 @@ func TestLeaderTransferIgnoreProposal(t *testing.T) {
|
||||
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)
|
||||
npeers[id] = sm
|
||||
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
|
||||
}
|
||||
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++ {
|
||||
if _, ok := learners[peerAddrs[i]]; ok {
|
||||
v.learnerPrs[peerAddrs[i]] = &Progress{IsLearner: true}
|
||||
v.prs.learners[peerAddrs[i]] = &Progress{IsLearner: true}
|
||||
} else {
|
||||
v.prs[peerAddrs[i]] = &Progress{}
|
||||
v.prs.nodes[peerAddrs[i]] = &Progress{}
|
||||
}
|
||||
}
|
||||
v.reset(v.Term)
|
||||
|
@ -257,12 +257,12 @@ const (
|
||||
// WithProgress is a helper to introspect the Progress for this node and its
|
||||
// peers.
|
||||
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.ins = nil
|
||||
visitor(id, ProgressTypePeer, pr)
|
||||
}
|
||||
for id, pr := range rn.raft.learnerPrs {
|
||||
for id, pr := range rn.raft.prs.learners {
|
||||
pr := *pr
|
||||
pr.ins = nil
|
||||
visitor(id, ProgressTypeLearner, pr)
|
||||
|
@ -34,11 +34,11 @@ type Status struct {
|
||||
|
||||
func getProgressCopy(r *raft) map[uint64]Progress {
|
||||
prs := make(map[uint64]Progress)
|
||||
for id, p := range r.prs {
|
||||
for id, p := range r.prs.nodes {
|
||||
prs[id] = *p
|
||||
}
|
||||
|
||||
for id, p := range r.learnerPrs {
|
||||
for id, p := range r.prs.learners {
|
||||
prs[id] = *p
|
||||
}
|
||||
return prs
|
||||
|
Loading…
x
Reference in New Issue
Block a user