Refactoring

This commit is contained in:
Marto 2024-02-07 15:54:25 +02:00
parent 2ecc344aa1
commit 47e1e36294
3 changed files with 43 additions and 59 deletions

View File

@ -20,9 +20,9 @@ randomize()
type type
RaftRpcMessageType* = enum RaftRpcMessageType* = enum
Vote = 0, VoteRequest = 0,
VoteReplay = 1, VoteReplay = 1,
Append = 2, AppendRequest = 2,
AppendReplay = 3 AppendReplay = 3
RaftRpcCode* = enum RaftRpcCode* = enum
@ -54,20 +54,17 @@ type
currentTerm*: RaftNodeTerm currentTerm*: RaftNodeTerm
lastLogIndex*: RaftLogIndex lastLogIndex*: RaftLogIndex
lastLogTerm*: RaftNodeTerm lastLogTerm*: RaftNodeTerm
isPrevote*: bool
force*: bool force*: bool
RaftRpcVoteReplay* = object RaftRpcVoteReplay* = object
currentTerm*: RaftNodeTerm currentTerm*: RaftNodeTerm
voteGranted*: bool voteGranted*: bool
isPrevote*: bool
LeaderState* = object LeaderState* = object
tracker: RaftTracker tracker: RaftTracker
CandidateState* = object CandidateState* = object
votes: RaftVotes votes: RaftVotes
isPrevote: bool
FollowerState* = object FollowerState* = object
leader: RaftNodeId leader: RaftNodeId
@ -77,9 +74,9 @@ type
sender*: RaftNodeId sender*: RaftNodeId
receiver*: RaftNodeId receiver*: RaftNodeId
case kind*: RaftRpcMessageType case kind*: RaftRpcMessageType
of Vote: voteRequest*: RaftRpcVoteRequest of VoteRequest: voteRequest*: RaftRpcVoteRequest
of VoteReplay: voteReplay*: RaftRpcVoteReplay of VoteReplay: voteReplay*: RaftRpcVoteReplay
of Append: appendRequest*: RaftRpcAppendRequest of AppendRequest: appendRequest*: RaftRpcAppendRequest
of AppendReplay: appendReplay*: RaftRpcAppendReplay of AppendReplay: appendReplay*: RaftRpcAppendReplay
RaftStateMachineOutput* = object RaftStateMachineOutput* = object
@ -176,13 +173,13 @@ func findFollowerProggressById(sm: var RaftStateMachine, id: RaftNodeId): Option
return sm.leader.tracker.find(id) return sm.leader.tracker.find(id)
func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcAppendRequest) = func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcAppendRequest) =
sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.Append, appendRequest: request)) sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.AppendRequest, appendRequest: request))
func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcAppendReplay) = func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcAppendReplay) =
sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.AppendReplay, appendReplay: request)) sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.AppendReplay, appendReplay: request))
func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcVoteRequest) = func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcVoteRequest) =
sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.Vote, voteRequest: request)) sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.VoteRequest, voteRequest: request))
func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcVoteReplay) = func sendToImpl*(sm: var RaftStateMachine, id: RaftNodeId, request: RaftRpcVoteReplay) =
sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.VoteReplay, voteReplay: request)) sm.output.messages.add(RaftRpcMessage(currentTerm: sm.term, receiver: id, sender: sm.myId, kind: RaftRpcMessageType.VoteReplay, voteReplay: request))
@ -199,7 +196,7 @@ func sendTo[MsgType](sm: var RaftStateMachine, id: RaftNodeId, request: MsgType)
sm.sendToImpl(id, request) sm.sendToImpl(id, request)
func createVoteRequest*(sm: var RaftStateMachine): RaftRpcMessage = func createVoteRequest*(sm: var RaftStateMachine): RaftRpcMessage =
return RaftRpcMessage(currentTerm: sm.term, sender: sm.myId, kind: Vote, voteRequest: RaftRpcVoteRequest()) return RaftRpcMessage(currentTerm: sm.term, sender: sm.myId, kind: VoteRequest, voteRequest: RaftRpcVoteRequest())
func replicateTo*(sm: var RaftStateMachine, follower: RaftFollowerProgressTracker) = func replicateTo*(sm: var RaftStateMachine, follower: RaftFollowerProgressTracker) =
if follower.nextIndex > sm.log.lastIndex: if follower.nextIndex > sm.log.lastIndex:
@ -268,7 +265,7 @@ func becomeLeader*(sm: var RaftStateMachine) =
#TODO: Update last election time #TODO: Update last election time
return return
func becomeCandidate*(sm: var RaftStateMachine, isPrevote: bool) = func becomeCandidate*(sm: var RaftStateMachine) =
#TODO: implement #TODO: implement
if not sm.state.isCandidate: if not sm.state.isCandidate:
sm.output.stateChange = true sm.output.stateChange = true
@ -285,14 +282,10 @@ func becomeCandidate*(sm: var RaftStateMachine, isPrevote: bool) =
sm.votedFor = nodeId sm.votedFor = nodeId
continue continue
let request = RaftRpcVoteRequest(currentTerm: sm.term, lastLogIndex: sm.log.lastIndex, lastLogTerm: sm.log.lastTerm, isPrevote: isPrevote, force: false) let request = RaftRpcVoteRequest(currentTerm: sm.term, lastLogIndex: sm.log.lastIndex, lastLogTerm: sm.log.lastTerm, force: false)
sm.sendTo(nodeId, request) sm.sendTo(nodeId, request)
sm.debug "Elecation won" & $(sm.candidate.votes) & $sm.myId sm.debug "Elecation won" & $(sm.candidate.votes) & $sm.myId
if sm.candidate.votes.tallyVote == RaftElectionResult.Won: if sm.candidate.votes.tallyVote == RaftElectionResult.Won:
if isPrevote:
sm.becomeCandidate(false)
else:
sm.becomeLeader() sm.becomeLeader()
return return
@ -310,7 +303,7 @@ func tickLeader*(sm: var RaftStateMachine, now: times.DateTime) =
sm.lastElectionTime = now sm.lastElectionTime = now
if not sm.state.isLeader: if not sm.state.isLeader:
sm.debug "tick_leader can be called only on the leader" sm.debug "tickLeader can be called only on the leader"
return return
for followerIndex in 0..<sm.leader.tracker.progress.len: for followerIndex in 0..<sm.leader.tracker.progress.len:
var follower = sm.leader.tracker.progress[followerIndex] var follower = sm.leader.tracker.progress[followerIndex]
@ -332,7 +325,7 @@ func tick*(sm: var RaftStateMachine, now: times.DateTime) =
sm.tickLeader(now); sm.tickLeader(now);
elif sm.state.isFollower and sm.timeNow - sm.lastElectionTime > sm.randomizedElectionTime: elif sm.state.isFollower and sm.timeNow - sm.lastElectionTime > sm.randomizedElectionTime:
sm.debug "Become candidate" sm.debug "Become candidate"
sm.becomeCandidate(false) sm.becomeCandidate()
func poll*(sm: var RaftStateMachine): RaftStateMachineOutput = func poll*(sm: var RaftStateMachine): RaftStateMachineOutput =
@ -351,17 +344,17 @@ func poll*(sm: var RaftStateMachine): RaftStateMachineOutput =
func commit*(sm: var RaftStateMachine) = func commit*(sm: var RaftStateMachine) =
if not sm.state.isLeader: if not sm.state.isLeader:
return return
var new_index = sm.commitIndex var newIndex = sm.commitIndex
var next_index = sm.commitIndex + 1 var nextIndex = sm.commitIndex + 1
while next_index < sm.log.lastIndex: while nextIndex < sm.log.lastIndex:
var replicationCnt = 0 var replicationCnt = 0
for p in sm.leader.tracker.progress: for p in sm.leader.tracker.progress:
if p.matchIndex > new_index: if p.matchIndex > newIndex:
replicationCnt += 1 replicationCnt += 1
if replicationCnt >= (sm.leader.tracker.progress.len div 2 + 1): if replicationCnt >= (sm.leader.tracker.progress.len div 2 + 1):
sm.output.committed.add(sm.log.getEntryByIndex(next_index)) sm.output.committed.add(sm.log.getEntryByIndex(nextIndex))
sm.commitIndex += next_index; sm.commitIndex += nextIndex;
next_index += 1 nextIndex += 1
else: else:
break break
@ -386,7 +379,7 @@ func appendEntryReplay*(sm: var RaftStateMachine, fromId: RaftNodeId, replay: Ra
of RaftRpcCode.Rejected: of RaftRpcCode.Rejected:
if replay.rejected.nonMatchingIndex == 0 and replay.rejected.lastIdx == 0: if replay.rejected.nonMatchingIndex == 0 and replay.rejected.lastIdx == 0:
sm.replicateTo(follower.get()) sm.replicateTo(follower.get())
follower.get().next_index = min(replay.rejected.nonMatchingIndex, replay.rejected.lastIdx + 1) follower.get().nextIndex = min(replay.rejected.nonMatchingIndex, replay.rejected.lastIdx + 1)
# if commit apply configuration that removes current follower # if commit apply configuration that removes current follower
# we should take it again # we should take it again
var follower2 = sm.findFollowerProggressById(fromId) var follower2 = sm.findFollowerProggressById(fromId)
@ -394,9 +387,9 @@ func appendEntryReplay*(sm: var RaftStateMachine, fromId: RaftNodeId, replay: Ra
sm.replicateTo(follower2.get()) sm.replicateTo(follower2.get())
func advanceCommitIdx(sm: var RaftStateMachine, leaderIdx: RaftLogIndex) = func advanceCommitIdx(sm: var RaftStateMachine, leaderIdx: RaftLogIndex) =
let new_idx = min(leaderIdx, sm.log.lastIndex) let newIdx = min(leaderIdx, sm.log.lastIndex)
if new_idx > sm.commitIndex: if newIdx > sm.commitIndex:
sm.commitIndex = new_idx sm.commitIndex = newIdx
# TODO: signal the output for the update # TODO: signal the output for the update
@ -418,52 +411,44 @@ func appendEntry*(sm: var RaftStateMachine, fromId: RaftNodeId, request: RaftRpc
sm.sendTo(fromId, responce) sm.sendTo(fromId, responce)
func requestVote*(sm: var RaftStateMachine, fromId: RaftNodeId, request: RaftRpcVoteRequest) = func requestVote*(sm: var RaftStateMachine, fromId: RaftNodeId, request: RaftRpcVoteRequest) =
let canVote = sm.votedFor == fromId or (sm.votedFor == RaftNodeId() and sm.currentLeader == RaftNodeId()) or (request.isPrevote and request.currentTerm > sm.term) let canVote = sm.votedFor == fromId or (sm.votedFor == RaftNodeId() and sm.currentLeader == RaftNodeId())
if canVote and sm.log.isUpToDate(request.lastLogIndex, request.lastLogTerm): if canVote and sm.log.isUpToDate(request.lastLogIndex, request.lastLogTerm):
if not request.is_prevote: let responce = RaftRpcVoteReplay(currentTerm: sm.term, voteGranted: true)
# TODO: Update election time
sm.votedFor = fromId
let responce = RaftRpcVoteReplay(currentTerm: sm.term, voteGranted: true, isPrevote: request.is_prevote)
sm.sendTo(fromId, responce) sm.sendTo(fromId, responce)
else: else:
let responce: RaftRpcVoteReplay = RaftRpcVoteReplay(currentTerm: sm.term, voteGranted: false, isPrevote: request.is_prevote) let responce: RaftRpcVoteReplay = RaftRpcVoteReplay(currentTerm: sm.term, voteGranted: false)
sm.sendTo(fromId, responce) sm.sendTo(fromId, responce)
func requestVoteReply*(sm: var RaftStateMachine, from_id: RaftNodeId, request: RaftRpcVoteReplay) = func requestVoteReply*(sm: var RaftStateMachine, fromId: RaftNodeId, request: RaftRpcVoteReplay) =
if not sm.state.isCandidate: if not sm.state.isCandidate:
sm.debug "Non candidate can't handle votes" sm.debug "Non candidate can't handle votes"
return return
discard sm.candidate.votes.registerVote(from_id, request.voteGranted) discard sm.candidate.votes.registerVote(fromId, request.voteGranted)
case sm.candidate.votes.tallyVote: case sm.candidate.votes.tallyVote:
of RaftElectionResult.Unknown: of RaftElectionResult.Unknown:
return return
of RaftElectionResult.Won: of RaftElectionResult.Won:
sm.debug "Win election" sm.debug "Win election"
if (sm.candidate.isPrevote): sm.becomeLeader()
sm.becomeCandidate(false)
else:
sm.becomeLeader()
of RaftElectionResult.Lost: of RaftElectionResult.Lost:
sm.debug "Lost election" sm.debug "Lost election"
sm.becomeFollower(RaftNodeId()) sm.becomeFollower(RaftNodeId())
# TODO: become foller
func advance*(sm: var RaftStateMachine, msg: RaftRpcMessage, now: times.DateTime) = func advance*(sm: var RaftStateMachine, msg: RaftRpcMessage, now: times.DateTime) =
#sm.debug $msg #sm.debug $msg
if msg.currentTerm > sm.term: if msg.currentTerm > sm.term:
sm.debug "Current node is behind" sm.debug "Current node is behind"
var leaderId = RaftnodeId() var leaderId = RaftnodeId()
if msg.kind == RaftRpcMessageType.Append: if msg.kind == RaftRpcMessageType.AppendRequest:
leaderId = msg.sender leaderId = msg.sender
sm.becomeFollower(leaderId) sm.becomeFollower(leaderId)
# TODO: implement pre vote # TODO: implement pre vote
sm.term = msg.currentTerm sm.term = msg.currentTerm
sm.votedFor = RaftnodeId() sm.votedFor = RaftnodeId()
elif msg.currentTerm < sm.term: elif msg.currentTerm < sm.term:
if msg.kind == RaftRpcMessageType.Append: if msg.kind == RaftRpcMessageType.AppendRequest:
# Instruct leader to step down # Instruct leader to step down
let rejected = RaftRpcAppendReplayRejected(nonMatchingIndex: 0, lastIdx: sm.log.lastIndex) let rejected = RaftRpcAppendReplayRejected(nonMatchingIndex: 0, lastIdx: sm.log.lastIndex)
let responce = RaftRpcAppendReplay(term: sm.term, commitIndex: sm.commitIndex, result: RaftRpcCode.Rejected, rejected: rejected) let responce = RaftRpcAppendReplay(term: sm.term, commitIndex: sm.commitIndex, result: RaftRpcCode.Rejected, rejected: rejected)
@ -472,14 +457,14 @@ func advance*(sm: var RaftStateMachine, msg: RaftRpcMessage, now: times.DateTime
sm.debug "Ignore message with lower term" sm.debug "Ignore message with lower term"
else: else:
# TODO: add also snapshot # TODO: add also snapshot
if msg.kind == RaftRpcMessageType.Append: if msg.kind == RaftRpcMessageType.AppendRequest:
if sm.state.isCandidate: if sm.state.isCandidate:
sm.becomeFollower(msg.sender) sm.becomeFollower(msg.sender)
elif sm.state.isFollower: elif sm.state.isFollower:
sm.follower.leader = msg.sender sm.follower.leader = msg.sender
# TODO: fix time # TODO: fix time
if sm.state.isCandidate: if sm.state.isCandidate:
if msg.kind == RaftRpcMessageType.Vote: if msg.kind == RaftRpcMessageType.VoteRequest:
sm.requestVote(msg.sender, msg.voteRequest) sm.requestVote(msg.sender, msg.voteRequest)
elif msg.kind == RaftRpcMessageType.VoteReplay: elif msg.kind == RaftRpcMessageType.VoteReplay:
sm.debug "Apply vote" sm.debug "Apply vote"
@ -489,19 +474,19 @@ func advance*(sm: var RaftStateMachine, msg: RaftRpcMessage, now: times.DateTime
elif sm.state.isFollower: elif sm.state.isFollower:
if msg.sender == sm.follower.leader: if msg.sender == sm.follower.leader:
sm.lastElectionTime = now sm.lastElectionTime = now
if msg.kind == RaftRpcMessageType.Append: if msg.kind == RaftRpcMessageType.AppendRequest:
sm.appendEntry(msg.sender, msg.appendRequest) sm.appendEntry(msg.sender, msg.appendRequest)
elif msg.kind == RaftRpcMessageType.Vote: elif msg.kind == RaftRpcMessageType.VoteRequest:
sm.requestVote(msg.sender, msg.voteRequest) sm.requestVote(msg.sender, msg.voteRequest)
else: else:
sm.debug "Follower ignore message" & $msg sm.debug "Follower ignore message" & $msg
# TODO: imelement the rest of the state transitions # TODO: imelement the rest of the state transitions
elif sm.state.isLeader: elif sm.state.isLeader:
if msg.kind == RaftRpcMessageType.Append: if msg.kind == RaftRpcMessageType.AppendRequest:
sm.debug "Ignore message leader append his entries directly" sm.debug "Ignore message leader append his entries directly"
elif msg.kind == RaftRpcMessageType.AppendReplay: elif msg.kind == RaftRpcMessageType.AppendReplay:
sm.appendEntryReplay(msg.sender, msg.appendReplay) sm.appendEntryReplay(msg.sender, msg.appendReplay)
elif msg.kind == RaftRpcMessageType.Vote: elif msg.kind == RaftRpcMessageType.VoteRequest:
sm.requestVote(msg.sender, msg.voteRequest) sm.requestVote(msg.sender, msg.voteRequest)
else: else:
sm.debug "Leader ignore message" sm.debug "Leader ignore message"

View File

@ -67,7 +67,6 @@ func appendAsFollower*(rf: var RaftLog, entry: LogEntry) =
if entry.index <= currentIdx: if entry.index <= currentIdx:
# TODO: The indexing hold only if we keep all entries in memory # TODO: The indexing hold only if we keep all entries in memory
# we should change it when we add support for snapshots # we should change it when we add support for snapshots
if entry.index >= rf.firstIndex or entry.term != rf.getEntryByIndex(entry.index).term: if entry.index >= rf.firstIndex or entry.term != rf.getEntryByIndex(entry.index).term:
rf.truncateUncomitted(entry.index) rf.truncateUncomitted(entry.index)
rf.logEntries.add(entry) rf.logEntries.add(entry)

View File

@ -238,7 +238,7 @@ proc consensusstatemachineMain*() =
timeNow += 1.milliseconds timeNow += 1.milliseconds
block: block:
let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: true, isPrevote: false) let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: true)
let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:id1, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay) let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:id1, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay)
check sm.state.isCandidate check sm.state.isCandidate
sm.advance(msg, timeNow) sm.advance(msg, timeNow)
@ -250,7 +250,7 @@ proc consensusstatemachineMain*() =
# Older messages should be ignored # Older messages should be ignored
block: block:
let voteRaplay = RaftRpcVoteReplay(currentTerm: (output.term - 1), voteGranted: true, isPrevote: false) let voteRaplay = RaftRpcVoteReplay(currentTerm: (output.term - 1), voteGranted: true)
let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:id1, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay) let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:id1, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay)
sm.advance(msg, timeNow) sm.advance(msg, timeNow)
output = sm.poll() output = sm.poll()
@ -269,7 +269,7 @@ proc consensusstatemachineMain*() =
output = sm.poll() output = sm.poll()
let entry = LogEntry(term: (output.term + 1), index: 101, kind: RaftLogEntryType.rletEmpty, empty: true) let entry = LogEntry(term: (output.term + 1), index: 101, kind: RaftLogEntryType.rletEmpty, empty: true)
let appendRequest = RaftRpcAppendRequest(previousTerm: (output.term + 1), previousLogIndex: 100, commitIndex: 99, entries: @[entry]) let appendRequest = RaftRpcAppendRequest(previousTerm: (output.term + 1), previousLogIndex: 100, commitIndex: 99, entries: @[entry])
let msg = RaftRpcMessage(currentTerm: (output.term + 1), sender: id2, receiver:id1, kind: RaftRpcMessageType.Append, appendRequest: appendRequest) let msg = RaftRpcMessage(currentTerm: (output.term + 1), sender: id2, receiver:id1, kind: RaftRpcMessageType.AppendRequest, appendRequest: appendRequest)
sm.advance(msg, timeNow) sm.advance(msg, timeNow)
output = sm.poll() output = sm.poll()
check output.stateChange == true check output.stateChange == true
@ -292,7 +292,7 @@ proc consensusstatemachineMain*() =
check output.votedFor.get() == mainNodeId check output.votedFor.get() == mainNodeId
timeNow += 1.milliseconds timeNow += 1.milliseconds
block: block:
let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: false, isPrevote: false) let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: false)
let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay) let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay)
check sm.state.isCandidate check sm.state.isCandidate
sm.advance(msg, timeNow) sm.advance(msg, timeNow)
@ -302,7 +302,7 @@ proc consensusstatemachineMain*() =
timeNow += 1.milliseconds timeNow += 1.milliseconds
block: block:
let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: false, isPrevote: false) let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: false)
let msg = RaftRpcMessage(currentTerm: output.term, sender: id3, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay) let msg = RaftRpcMessage(currentTerm: output.term, sender: id3, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay)
check sm.state.isCandidate check sm.state.isCandidate
sm.advance(msg, timeNow) sm.advance(msg, timeNow)
@ -329,7 +329,7 @@ proc consensusstatemachineMain*() =
check output.votedFor.get() == mainNodeId check output.votedFor.get() == mainNodeId
timeNow += 1.milliseconds timeNow += 1.milliseconds
block: block:
let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: false, isPrevote: false) let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: false)
let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay) let msg = RaftRpcMessage(currentTerm: output.term, sender: id2, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay)
check sm.state.isCandidate check sm.state.isCandidate
sm.advance(msg, timeNow) sm.advance(msg, timeNow)
@ -339,7 +339,7 @@ proc consensusstatemachineMain*() =
timeNow += 1.milliseconds timeNow += 1.milliseconds
block: block:
let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: true, isPrevote: false) let voteRaplay = RaftRpcVoteReplay(currentTerm: output.term, voteGranted: true)
let msg = RaftRpcMessage(currentTerm: output.term, sender: id3, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay) let msg = RaftRpcMessage(currentTerm: output.term, sender: id3, receiver:mainNodeId, kind: RaftRpcMessageType.VoteReplay, voteReplay: voteRaplay)
check sm.state.isCandidate check sm.state.isCandidate
sm.advance(msg, timeNow) sm.advance(msg, timeNow)