2023-08-09 13:06:34 +03:00
|
|
|
# nim-raft
|
|
|
|
# Copyright (c) 2023 Status Research & Development GmbH
|
|
|
|
# Licensed under either of
|
|
|
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
|
|
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
|
|
|
# at your option.
|
|
|
|
# This file may not be copied, modified, or distributed except according to
|
2023-08-14 23:49:21 +03:00
|
|
|
# those terms.
|
|
|
|
|
2023-09-04 12:47:27 +03:00
|
|
|
{.hint[XDeclaredButNotUsed]: off.}
|
|
|
|
|
|
|
|
import types
|
|
|
|
import protocol
|
|
|
|
import log_ops
|
|
|
|
import chronicles
|
2023-08-14 23:49:21 +03:00
|
|
|
|
2023-09-06 19:18:02 +03:00
|
|
|
proc RaftNodeQuorumMin[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): bool =
|
|
|
|
result = false
|
2023-09-06 22:27:22 +03:00
|
|
|
withRLock(node.raftStateMutex):
|
2023-09-06 19:18:02 +03:00
|
|
|
var cnt = 0
|
|
|
|
for peer in node.peers:
|
|
|
|
if peer.hasVoted:
|
|
|
|
cnt.inc
|
|
|
|
if cnt >= (node.peers.len div 2 + 1):
|
|
|
|
result = true
|
|
|
|
|
2023-09-03 20:52:35 +03:00
|
|
|
proc RaftNodeStartElection*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]) {.async.} =
|
2023-09-06 22:27:22 +03:00
|
|
|
withRLock(node.raftStateMutex):
|
2023-09-04 12:47:27 +03:00
|
|
|
node.currentTerm.inc
|
|
|
|
node.state = rnsCandidate
|
|
|
|
node.votedFor = node.id
|
2023-09-06 22:27:22 +03:00
|
|
|
debug "Raft Node started election", node_id=node.id, state=node.state, voted_for=node.votedFor
|
2023-09-03 06:27:27 +03:00
|
|
|
|
2023-09-04 12:47:27 +03:00
|
|
|
for peer in node.peers:
|
|
|
|
peer.hasVoted = false
|
|
|
|
node.votesFuts.add(node.msgSendCallback(
|
2023-09-06 19:18:02 +03:00
|
|
|
RaftMessageRequestVote(
|
|
|
|
op: rmoRequestVote, msgId: genUUID(), senderId: node.id,
|
|
|
|
receiverId: peer.id, lastLogTerm: RaftNodeLogEntryGet(node, RaftNodeLogIndexGet(node)).term,
|
|
|
|
lastLogIndex: RaftNodeLogIndexGet(node), senderTerm: node.currentTerm)
|
2023-09-04 12:47:27 +03:00
|
|
|
)
|
2023-09-03 06:27:27 +03:00
|
|
|
)
|
|
|
|
|
2023-09-03 20:52:35 +03:00
|
|
|
# Process votes (if any)
|
|
|
|
for voteFut in node.votesFuts:
|
2023-09-06 19:18:02 +03:00
|
|
|
let r = await voteFut
|
|
|
|
let respVote = RaftMessageRequestVoteResponse(r)
|
2023-09-06 22:27:22 +03:00
|
|
|
debug "Received vote", node_id=node.id, sender_id=respVote.senderId, granted=respVote.granted
|
2023-09-04 12:47:27 +03:00
|
|
|
|
2023-09-06 22:27:22 +03:00
|
|
|
withRLock(node.raftStateMutex):
|
2023-09-03 06:27:27 +03:00
|
|
|
for p in node.peers:
|
2023-09-06 19:18:02 +03:00
|
|
|
if p.id == respVote.senderId:
|
|
|
|
p.hasVoted = respVote.granted
|
2023-09-03 20:52:35 +03:00
|
|
|
|
2023-09-06 22:27:22 +03:00
|
|
|
withRLock(node.raftStateMutex):
|
2023-09-06 19:18:02 +03:00
|
|
|
while node.votesFuts.len > 0:
|
|
|
|
discard node.votesFuts.pop
|
2023-08-14 23:49:21 +03:00
|
|
|
|
2023-09-06 19:18:02 +03:00
|
|
|
if node.state == rnsCandidate:
|
|
|
|
if RaftNodeQuorumMin(node):
|
2023-09-07 05:04:27 +03:00
|
|
|
asyncSpawn cancelAndWait(node.electionTimeoutTimer)
|
2023-09-06 22:27:22 +03:00
|
|
|
debug "Raft Node transition to leader", node_id=node.id
|
2023-09-06 19:18:02 +03:00
|
|
|
node.state = rnsLeader # Transition to leader and send Heart-Beat to establish this node as the cluster leader
|
2023-09-06 22:27:22 +03:00
|
|
|
asyncSpawn RaftNodeSendHeartBeat(node)
|
2023-09-06 19:18:02 +03:00
|
|
|
|
|
|
|
proc RaftNodeHandleRequestVote*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], msg: RaftMessageRequestVote): RaftMessageRequestVoteResponse =
|
2023-09-06 22:27:22 +03:00
|
|
|
withRLock(node.raftStateMutex):
|
2023-09-06 19:18:02 +03:00
|
|
|
result = RaftMessageRequestVoteResponse(msgId: msg.msgId, senderId: node.id, receiverId: msg.senderId, granted: false)
|
2023-09-06 22:27:22 +03:00
|
|
|
if node.state != rnsCandidate and node.state != rnsStopped and msg.senderTerm > node.currentTerm and node.votedFor == DefaultUUID:
|
2023-09-07 05:04:27 +03:00
|
|
|
# if msg.lastLogIndex >= RaftNodeLogIndexGet(node) and msg.lastLogTerm >= RaftNodeLogEntryGet(node, RaftNodeLogIndexGet(node)).term:
|
|
|
|
asyncSpawn cancelAndWait(node.electionTimeoutTimer)
|
|
|
|
node.votedFor = msg.senderId
|
|
|
|
result.granted = true
|
|
|
|
RaftNodeScheduleElectionTimeout(node)
|
2023-08-14 23:49:21 +03:00
|
|
|
|
2023-09-06 19:18:02 +03:00
|
|
|
proc RaftNodeHandleAppendEntries*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], msg: RaftMessageAppendEntries): RaftMessageAppendEntriesResponse[SmStateType] =
|
2023-09-01 06:28:43 +03:00
|
|
|
discard
|
|
|
|
|
|
|
|
proc RaftNodeReplicateSmCommand*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], cmd: SmCommandType) =
|
|
|
|
discard
|
|
|
|
|
2023-09-03 03:53:48 +03:00
|
|
|
proc RaftNodeLogAppend[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], logEntry: RaftNodeLogEntry[SmCommandType]) =
|
|
|
|
discard
|
|
|
|
|
|
|
|
proc RaftNodeLogTruncate[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], truncateIndex: uint64) =
|
2023-08-14 23:49:21 +03:00
|
|
|
discard
|