|
@@ -17,7 +17,6 @@ package raft
|
|
|
import (
|
|
import (
|
|
|
"errors"
|
|
"errors"
|
|
|
"fmt"
|
|
"fmt"
|
|
|
- "log"
|
|
|
|
|
"math/rand"
|
|
"math/rand"
|
|
|
"sort"
|
|
"sort"
|
|
|
"strings"
|
|
"strings"
|
|
@@ -229,7 +228,7 @@ func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage
|
|
|
nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
|
|
nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- log.Printf("raft: newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
|
|
|
r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
|
|
r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
|
|
|
return r
|
|
return r
|
|
|
}
|
|
}
|
|
@@ -286,10 +285,10 @@ func (r *raft) sendAppend(to uint64) {
|
|
|
}
|
|
}
|
|
|
m.Snapshot = snapshot
|
|
m.Snapshot = snapshot
|
|
|
sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
|
|
sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
|
|
|
- log.Printf("raft: %x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
|
|
|
r.id, r.raftLog.firstIndex(), r.Commit, sindex, sterm, to, pr)
|
|
r.id, r.raftLog.firstIndex(), r.Commit, sindex, sterm, to, pr)
|
|
|
pr.setPendingSnapshot(sindex)
|
|
pr.setPendingSnapshot(sindex)
|
|
|
- log.Printf("raft: %x paused sending replication messages to %x [%s]", r.id, to, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x paused sending replication messages to %x [%s]", r.id, to, pr)
|
|
|
} else {
|
|
} else {
|
|
|
m.Type = pb.MsgApp
|
|
m.Type = pb.MsgApp
|
|
|
m.Index = pr.Next - 1
|
|
m.Index = pr.Next - 1
|
|
@@ -413,7 +412,7 @@ func (r *raft) becomeFollower(term uint64, lead uint64) {
|
|
|
r.tick = r.tickElection
|
|
r.tick = r.tickElection
|
|
|
r.lead = lead
|
|
r.lead = lead
|
|
|
r.state = StateFollower
|
|
r.state = StateFollower
|
|
|
- log.Printf("raft: %x became follower at term %d", r.id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x became follower at term %d", r.id, r.Term)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (r *raft) becomeCandidate() {
|
|
func (r *raft) becomeCandidate() {
|
|
@@ -426,7 +425,7 @@ func (r *raft) becomeCandidate() {
|
|
|
r.tick = r.tickElection
|
|
r.tick = r.tickElection
|
|
|
r.Vote = r.id
|
|
r.Vote = r.id
|
|
|
r.state = StateCandidate
|
|
r.state = StateCandidate
|
|
|
- log.Printf("raft: %x became candidate at term %d", r.id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x became candidate at term %d", r.id, r.Term)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (r *raft) becomeLeader() {
|
|
func (r *raft) becomeLeader() {
|
|
@@ -449,7 +448,7 @@ func (r *raft) becomeLeader() {
|
|
|
r.pendingConf = true
|
|
r.pendingConf = true
|
|
|
}
|
|
}
|
|
|
r.appendEntry(pb.Entry{Data: nil})
|
|
r.appendEntry(pb.Entry{Data: nil})
|
|
|
- log.Printf("raft: %x became leader at term %d", r.id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x became leader at term %d", r.id, r.Term)
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (r *raft) campaign() {
|
|
func (r *raft) campaign() {
|
|
@@ -462,7 +461,7 @@ func (r *raft) campaign() {
|
|
|
if i == r.id {
|
|
if i == r.id {
|
|
|
continue
|
|
continue
|
|
|
}
|
|
}
|
|
|
- log.Printf("raft: %x [logterm: %d, index: %d] sent vote request to %x at term %d",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [logterm: %d, index: %d] sent vote request to %x at term %d",
|
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), i, r.Term)
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), i, r.Term)
|
|
|
r.send(pb.Message{To: i, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm()})
|
|
r.send(pb.Message{To: i, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm()})
|
|
|
}
|
|
}
|
|
@@ -470,9 +469,9 @@ func (r *raft) campaign() {
|
|
|
|
|
|
|
|
func (r *raft) poll(id uint64, v bool) (granted int) {
|
|
func (r *raft) poll(id uint64, v bool) (granted int) {
|
|
|
if v {
|
|
if v {
|
|
|
- log.Printf("raft: %x received vote from %x at term %d", r.id, id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x received vote from %x at term %d", r.id, id, r.Term)
|
|
|
} else {
|
|
} else {
|
|
|
- log.Printf("raft: %x received vote rejection from %x at term %d", r.id, id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x received vote rejection from %x at term %d", r.id, id, r.Term)
|
|
|
}
|
|
}
|
|
|
if _, ok := r.votes[id]; !ok {
|
|
if _, ok := r.votes[id]; !ok {
|
|
|
r.votes[id] = v
|
|
r.votes[id] = v
|
|
@@ -487,7 +486,7 @@ func (r *raft) poll(id uint64, v bool) (granted int) {
|
|
|
|
|
|
|
|
func (r *raft) Step(m pb.Message) error {
|
|
func (r *raft) Step(m pb.Message) error {
|
|
|
if m.Type == pb.MsgHup {
|
|
if m.Type == pb.MsgHup {
|
|
|
- log.Printf("raft: %x is starting a new election at term %d", r.id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x is starting a new election at term %d", r.id, r.Term)
|
|
|
r.campaign()
|
|
r.campaign()
|
|
|
r.Commit = r.raftLog.committed
|
|
r.Commit = r.raftLog.committed
|
|
|
return nil
|
|
return nil
|
|
@@ -501,12 +500,12 @@ func (r *raft) Step(m pb.Message) error {
|
|
|
if m.Type == pb.MsgVote {
|
|
if m.Type == pb.MsgVote {
|
|
|
lead = None
|
|
lead = None
|
|
|
}
|
|
}
|
|
|
- log.Printf("raft: %x [term: %d] received a %s message with higher term from %x [term: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [term: %d] received a %s message with higher term from %x [term: %d]",
|
|
|
r.id, r.Term, m.Type, m.From, m.Term)
|
|
r.id, r.Term, m.Type, m.From, m.Term)
|
|
|
r.becomeFollower(m.Term, lead)
|
|
r.becomeFollower(m.Term, lead)
|
|
|
case m.Term < r.Term:
|
|
case m.Term < r.Term:
|
|
|
// ignore
|
|
// ignore
|
|
|
- log.Printf("raft: %x [term: %d] ignored a %s message with lower term from %x [term: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [term: %d] ignored a %s message with lower term from %x [term: %d]",
|
|
|
r.id, r.Term, m.Type, m.From, m.Term)
|
|
r.id, r.Term, m.Type, m.From, m.Term)
|
|
|
return nil
|
|
return nil
|
|
|
}
|
|
}
|
|
@@ -525,7 +524,7 @@ func stepLeader(r *raft, m pb.Message) {
|
|
|
r.bcastHeartbeat()
|
|
r.bcastHeartbeat()
|
|
|
case pb.MsgProp:
|
|
case pb.MsgProp:
|
|
|
if len(m.Entries) == 0 {
|
|
if len(m.Entries) == 0 {
|
|
|
- log.Panicf("raft: %x stepped empty MsgProp", r.id)
|
|
|
|
|
|
|
+ raftLogger.Panicf("raft: %x stepped empty MsgProp", r.id)
|
|
|
}
|
|
}
|
|
|
for i, e := range m.Entries {
|
|
for i, e := range m.Entries {
|
|
|
if e.Type == pb.EntryConfChange {
|
|
if e.Type == pb.EntryConfChange {
|
|
@@ -540,20 +539,20 @@ func stepLeader(r *raft, m pb.Message) {
|
|
|
case pb.MsgAppResp:
|
|
case pb.MsgAppResp:
|
|
|
if pr.isUnreachable() {
|
|
if pr.isUnreachable() {
|
|
|
pr.reachable()
|
|
pr.reachable()
|
|
|
- log.Printf("raft: %x received msgAppResp from %x and changed it to be reachable [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x received msgAppResp from %x and changed it to be reachable [%s]", r.id, m.From, pr)
|
|
|
}
|
|
}
|
|
|
if m.Reject {
|
|
if m.Reject {
|
|
|
- log.Printf("raft: %x received msgApp rejection(lastindex: %d) from %x for index %d",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x received msgApp rejection(lastindex: %d) from %x for index %d",
|
|
|
r.id, m.RejectHint, m.From, m.Index)
|
|
r.id, m.RejectHint, m.From, m.Index)
|
|
|
if pr.maybeDecrTo(m.Index, m.RejectHint) {
|
|
if pr.maybeDecrTo(m.Index, m.RejectHint) {
|
|
|
- log.Printf("raft: %x decreased progress of %x to [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x decreased progress of %x to [%s]", r.id, m.From, pr)
|
|
|
r.sendAppend(m.From)
|
|
r.sendAppend(m.From)
|
|
|
}
|
|
}
|
|
|
} else {
|
|
} else {
|
|
|
oldWait := pr.shouldWait()
|
|
oldWait := pr.shouldWait()
|
|
|
pr.update(m.Index)
|
|
pr.update(m.Index)
|
|
|
if r.prs[m.From].maybeSnapshotAbort() {
|
|
if r.prs[m.From].maybeSnapshotAbort() {
|
|
|
- log.Printf("raft: %x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
|
|
|
}
|
|
}
|
|
|
if r.maybeCommit() {
|
|
if r.maybeCommit() {
|
|
|
r.bcastAppend()
|
|
r.bcastAppend()
|
|
@@ -566,13 +565,13 @@ func stepLeader(r *raft, m pb.Message) {
|
|
|
case pb.MsgHeartbeatResp:
|
|
case pb.MsgHeartbeatResp:
|
|
|
if pr.isUnreachable() {
|
|
if pr.isUnreachable() {
|
|
|
pr.reachable()
|
|
pr.reachable()
|
|
|
- log.Printf("raft: %x received msgHeartbeatResp from %x and changed it to be reachable [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x received msgHeartbeatResp from %x and changed it to be reachable [%s]", r.id, m.From, pr)
|
|
|
}
|
|
}
|
|
|
if pr.Match < r.raftLog.lastIndex() {
|
|
if pr.Match < r.raftLog.lastIndex() {
|
|
|
r.sendAppend(m.From)
|
|
r.sendAppend(m.From)
|
|
|
}
|
|
}
|
|
|
case pb.MsgVote:
|
|
case pb.MsgVote:
|
|
|
- log.Printf("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
|
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
|
|
|
case pb.MsgSnapStatus:
|
|
case pb.MsgSnapStatus:
|
|
@@ -581,10 +580,10 @@ func stepLeader(r *raft, m pb.Message) {
|
|
|
}
|
|
}
|
|
|
if m.Reject {
|
|
if m.Reject {
|
|
|
pr.snapshotFail()
|
|
pr.snapshotFail()
|
|
|
- log.Printf("raft: %x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
|
|
|
} else {
|
|
} else {
|
|
|
pr.snapshotFinish()
|
|
pr.snapshotFinish()
|
|
|
- log.Printf("raft: %x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
|
|
|
// wait for the msgAppResp from the remote node before sending
|
|
// wait for the msgAppResp from the remote node before sending
|
|
|
// out the next msgApp
|
|
// out the next msgApp
|
|
|
pr.waitSet(r.electionTimeout)
|
|
pr.waitSet(r.electionTimeout)
|
|
@@ -592,7 +591,7 @@ func stepLeader(r *raft, m pb.Message) {
|
|
|
case pb.MsgUnreachable:
|
|
case pb.MsgUnreachable:
|
|
|
if !pr.isUnreachable() {
|
|
if !pr.isUnreachable() {
|
|
|
pr.unreachable()
|
|
pr.unreachable()
|
|
|
- log.Printf("raft: %x failed to send message to %x and changed it to be unreachable [%s]", r.id, m.From, pr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x failed to send message to %x and changed it to be unreachable [%s]", r.id, m.From, pr)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
@@ -600,7 +599,7 @@ func stepLeader(r *raft, m pb.Message) {
|
|
|
func stepCandidate(r *raft, m pb.Message) {
|
|
func stepCandidate(r *raft, m pb.Message) {
|
|
|
switch m.Type {
|
|
switch m.Type {
|
|
|
case pb.MsgProp:
|
|
case pb.MsgProp:
|
|
|
- log.Printf("raft: %x no leader at term %d; dropping proposal", r.id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x no leader at term %d; dropping proposal", r.id, r.Term)
|
|
|
return
|
|
return
|
|
|
case pb.MsgApp:
|
|
case pb.MsgApp:
|
|
|
r.becomeFollower(r.Term, m.From)
|
|
r.becomeFollower(r.Term, m.From)
|
|
@@ -612,12 +611,12 @@ func stepCandidate(r *raft, m pb.Message) {
|
|
|
r.becomeFollower(m.Term, m.From)
|
|
r.becomeFollower(m.Term, m.From)
|
|
|
r.handleSnapshot(m)
|
|
r.handleSnapshot(m)
|
|
|
case pb.MsgVote:
|
|
case pb.MsgVote:
|
|
|
- log.Printf("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %x",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %x",
|
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
|
|
|
case pb.MsgVoteResp:
|
|
case pb.MsgVoteResp:
|
|
|
gr := r.poll(m.From, !m.Reject)
|
|
gr := r.poll(m.From, !m.Reject)
|
|
|
- log.Printf("raft: %x [q:%d] has received %d votes and %d vote rejections", r.id, r.q(), gr, len(r.votes)-gr)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [q:%d] has received %d votes and %d vote rejections", r.id, r.q(), gr, len(r.votes)-gr)
|
|
|
switch r.q() {
|
|
switch r.q() {
|
|
|
case gr:
|
|
case gr:
|
|
|
r.becomeLeader()
|
|
r.becomeLeader()
|
|
@@ -632,7 +631,7 @@ func stepFollower(r *raft, m pb.Message) {
|
|
|
switch m.Type {
|
|
switch m.Type {
|
|
|
case pb.MsgProp:
|
|
case pb.MsgProp:
|
|
|
if r.lead == None {
|
|
if r.lead == None {
|
|
|
- log.Printf("raft: %x no leader at term %d; dropping proposal", r.id, r.Term)
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x no leader at term %d; dropping proposal", r.id, r.Term)
|
|
|
return
|
|
return
|
|
|
}
|
|
}
|
|
|
m.To = r.lead
|
|
m.To = r.lead
|
|
@@ -651,12 +650,12 @@ func stepFollower(r *raft, m pb.Message) {
|
|
|
case pb.MsgVote:
|
|
case pb.MsgVote:
|
|
|
if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
|
|
if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
|
|
|
r.elapsed = 0
|
|
r.elapsed = 0
|
|
|
- log.Printf("raft: %x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
|
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
|
r.Vote = m.From
|
|
r.Vote = m.From
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
|
|
|
} else {
|
|
} else {
|
|
|
- log.Printf("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
|
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
|
|
|
}
|
|
}
|
|
@@ -667,7 +666,7 @@ func (r *raft) handleAppendEntries(m pb.Message) {
|
|
|
if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
|
|
if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
|
|
|
} else {
|
|
} else {
|
|
|
- log.Printf("raft: %x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
|
|
|
r.id, r.raftLog.term(m.Index), m.Index, m.LogTerm, m.Index, m.From)
|
|
r.id, r.raftLog.term(m.Index), m.Index, m.LogTerm, m.Index, m.From)
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
|
|
|
}
|
|
}
|
|
@@ -681,11 +680,11 @@ func (r *raft) handleHeartbeat(m pb.Message) {
|
|
|
func (r *raft) handleSnapshot(m pb.Message) {
|
|
func (r *raft) handleSnapshot(m pb.Message) {
|
|
|
sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
|
|
sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
|
|
|
if r.restore(m.Snapshot) {
|
|
if r.restore(m.Snapshot) {
|
|
|
- log.Printf("raft: %x [commit: %d] restored snapshot [index: %d, term: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [commit: %d] restored snapshot [index: %d, term: %d]",
|
|
|
r.id, r.Commit, sindex, sterm)
|
|
r.id, r.Commit, sindex, sterm)
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
|
|
|
} else {
|
|
} else {
|
|
|
- log.Printf("raft: %x [commit: %d] ignored snapshot [index: %d, term: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [commit: %d] ignored snapshot [index: %d, term: %d]",
|
|
|
r.id, r.Commit, sindex, sterm)
|
|
r.id, r.Commit, sindex, sterm)
|
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
|
|
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
|
|
|
}
|
|
}
|
|
@@ -698,13 +697,13 @@ func (r *raft) restore(s pb.Snapshot) bool {
|
|
|
return false
|
|
return false
|
|
|
}
|
|
}
|
|
|
if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
|
|
if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
|
|
|
- log.Printf("raft: %x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
|
|
|
r.id, r.Commit, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
|
|
r.id, r.Commit, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
|
|
|
r.raftLog.commitTo(s.Metadata.Index)
|
|
r.raftLog.commitTo(s.Metadata.Index)
|
|
|
return false
|
|
return false
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- log.Printf("raft: %x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
|
|
|
r.id, r.Commit, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
|
|
r.id, r.Commit, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
|
|
|
|
|
|
|
|
r.raftLog.restore(s)
|
|
r.raftLog.restore(s)
|
|
@@ -717,7 +716,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
|
|
|
match = 0
|
|
match = 0
|
|
|
}
|
|
}
|
|
|
r.setProgress(n, match, next)
|
|
r.setProgress(n, match, next)
|
|
|
- log.Printf("raft: %x restored progress of %x [%s]", r.id, n, r.prs[n])
|
|
|
|
|
|
|
+ raftLogger.Infof("raft: %x restored progress of %x [%s]", r.id, n, r.prs[n])
|
|
|
}
|
|
}
|
|
|
return true
|
|
return true
|
|
|
}
|
|
}
|
|
@@ -761,7 +760,7 @@ func (r *raft) delProgress(id uint64) {
|
|
|
|
|
|
|
|
func (r *raft) loadState(state pb.HardState) {
|
|
func (r *raft) loadState(state pb.HardState) {
|
|
|
if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
|
|
if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
|
|
|
- log.Panicf("raft: %x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
|
|
|
|
|
|
|
+ raftLogger.Panicf("raft: %x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
|
|
|
}
|
|
}
|
|
|
r.raftLog.committed = state.Commit
|
|
r.raftLog.committed = state.Commit
|
|
|
r.Term = state.Term
|
|
r.Term = state.Term
|