nim-raft/raft/raft_api.nim

149 lines
6.5 KiB
Nim
Raw Normal View History

# 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
# those terms.
2023-09-04 09:47:27 +00:00
{.hint[XDeclaredButNotUsed]: off.}
2023-09-01 03:28:43 +00:00
import types
import protocol
2023-08-14 20:49:21 +00:00
import consensus_module
2023-09-03 03:27:27 +00:00
import log_ops
import ../db/kvstore_mdbx
2023-09-04 09:47:27 +00:00
import chronicles
2023-09-04 09:47:27 +00:00
export
types,
protocol,
consensus_module,
log_ops,
chronicles
2023-09-04 09:47:27 +00:00
# Forward declarations
2023-08-31 14:05:41 +00:00
proc RaftNodeSmInit[SmCommandType, SmStateType](stateMachine: var RaftNodeStateMachine[SmCommandType, SmStateType])
proc RaftNodeSendHeartBeat*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType])
2023-09-03 00:53:48 +00:00
proc RaftNodeScheduleHeartBeatTimeout*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): Future[void] {.async.}
2023-08-31 14:05:41 +00:00
2023-09-04 09:47:27 +00:00
# Raft Node Public API
proc new*[SmCommandType, SmStateType](T: type RaftNode[SmCommandType, SmStateType]; # Create New Raft Node
id: RaftNodeId; peersIds: seq[RaftNodeId];
2023-08-31 20:52:52 +00:00
# persistentStorage: RaftNodePersistentStorage,
msgSendCallback: RaftMessageSendCallback): T =
2023-08-31 14:05:41 +00:00
var
2023-08-31 20:52:52 +00:00
peers: RaftNodePeers
for peerId in peersIds:
peers.add(RaftNodePeer(id: peerId, nextIndex: 0, matchIndex: 0, hasVoted: false, canVote: true))
result = T(
2023-08-31 20:52:52 +00:00
id: id, state: rnsFollower, currentTerm: 0, peers: peers, commitIndex: 0, lastApplied: 0,
2023-09-02 21:16:26 +00:00
msgSendCallback: msgSendCallback, votedFor: DefaultUUID, currentLeaderId: DefaultUUID
2023-08-31 14:05:41 +00:00
)
2023-09-02 21:16:26 +00:00
RaftNodeSmInit(result.stateMachine)
2023-08-31 20:52:52 +00:00
initLock(result.raftStateMutex)
2023-08-14 20:49:21 +00:00
proc RaftNodeLoad*[SmCommandType, SmStateType](
persistentStorage: RaftNodePersistentStorage, # Load Raft Node From Storage
2023-08-14 20:49:21 +00:00
msgSendCallback: RaftMessageSendCallback): Result[RaftNode[SmCommandType, SmStateType], string] =
discard
proc RaftNodeIdGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): RaftNodeId {.gcsafe.} = # Get Raft Node ID
result = node.id
proc RaftNodeStateGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): RaftNodeState = # Get Raft Node State
node.state
proc RaftNodeTermGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): RaftNodeTerm = # Get Raft Node Term
2023-08-31 14:05:41 +00:00
node.currentTerm
2023-08-14 20:49:21 +00:00
func RaftNodePeersGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): RaftNodePeers = # Get Raft Node Peers
2023-08-31 14:05:41 +00:00
node.peers
2023-08-14 20:49:21 +00:00
func RaftNodeIsLeader*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): bool = # Check if Raft Node is Leader
2023-08-31 14:05:41 +00:00
node.state == rnsLeader
2023-08-14 20:49:21 +00:00
# Deliver Raft Message to the Raft Node and dispatch it
2023-09-01 02:55:55 +00:00
proc RaftNodeMessageDeliver*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], raftMessage: RaftMessageBase): Future[RaftMessageResponseBase] {.async, gcsafe.} =
2023-09-04 09:47:27 +00:00
case raftMessage.op
of rmoRequestVote: # Dispatch different Raft Message types based on the operation code
discard
of rmoAppendLogEntry:
discard
else: discard
2023-09-03 00:53:48 +00:00
discard
2023-08-14 20:49:21 +00:00
# Process RaftNodeClientRequests
2023-09-04 09:47:27 +00:00
proc RaftNodeServeClientRequest*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], req: RaftNodeClientRequest[SmCommandType]): Future[RaftNodeClientResponse[SmStateType]] {.async, gcsafe.} =
2023-09-03 00:53:48 +00:00
case req.op
2023-09-02 21:16:26 +00:00
of rncroExecSmCommand:
# TODO: implemenmt command handling
discard
of rncroRequestSmState:
if RaftNodeIsLeader(node):
2023-09-04 09:47:27 +00:00
return RaftNodeClientResponse(nodeId: node.id, error: rncreSuccess, state: RaftNodeStateGet(node))
else:
return RaftNodeClientResponse(nodeId: node.id, error: rncreNotLeader, currentLeaderId: node.currentLeaderId)
else:
raiseAssert "Unknown client request operation."
# Abstract State Machine Ops
2023-08-14 20:49:21 +00:00
func RaftNodeSmStateGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): SmStateType =
node.stateMachine.state
2023-08-14 20:49:21 +00:00
proc RaftNodeSmInit[SmCommandType, SmStateType](stateMachine: var RaftNodeStateMachine[SmCommandType, SmStateType]) =
mixin RaftSmInit
RaftSmInit(stateMachine)
2023-08-14 20:49:21 +00:00
proc RaftNodeSmApply[SmCommandType, SmStateType](stateMachine: RaftNodeStateMachine[SmCommandType, SmStateType], command: SmCommandType) =
mixin RaftSmApply
2023-08-14 20:49:21 +00:00
RaftSmApply(stateMachine, command)
2023-08-14 20:49:21 +00:00
# Private Abstract Timer manipulation Ops
2023-08-25 09:00:40 +00:00
template RaftTimerCreate(timerInterval: int, oneshot: bool, timerCallback: RaftTimerCallback): RaftTimer =
2023-08-10 08:38:09 +00:00
mixin RaftTimerCreateCustomImpl
2023-08-25 09:00:40 +00:00
RaftTimerCreateCustomImpl(timerInterval, oneshot, timerCallback)
# Timers scheduling stuff etc.
proc RaftNodeScheduleHeartBeat*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]) =
var fut = sleepAsync(node.heartBeatTimeout)
fut.callback = proc () = RaftNodeSendHeartBeat(node)
2023-09-03 00:53:48 +00:00
proc RaftNodeScheduleHeartBeatTimeout*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): Future[void] {.async.} =
node.heartBeatTimeoutTimer = sleepAsync(node.heartBeatTimeout)
2023-09-03 00:53:48 +00:00
await node.heartBeatTimeoutTimer
node.state = rnsCandidate # Transition to candidate state and initiate new Election
var f = RaftNodeStartElection(node)
cancel(f)
proc RaftNodeSendHeartBeat*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]) =
for raftPeer in node.peers:
let msgHrtBt = RaftMessageAppendEntries(
senderId: node.id, receiverId: raftPeer.id,
senderTerm: RaftNodeTermGet(node), commitIndex: node.commitIndex,
prevLogIndex: RaftNodeLogIndexGet(node) - 1, prevLogTerm: if RaftNodeLogIndexGet(node) > 0: RaftNodeLogEntry(node, RaftNodeLogIndexGet(node) - 1).term else: 0
)
2023-09-04 09:47:27 +00:00
asyncSpawn node.msgSendCallback(msgHrtBt)
RaftNodeScheduleHeartBeat(node)
# Raft Node Control
2023-09-03 00:53:48 +00:00
proc RaftNodeCancelAllTimers[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]) =
node.requestVotesTimer.fail(newException(Exception, "fail"))
node.heartBeatTimer.fail(newException(Exception, "fail"))
node.heartBeatTimeoutTimer.fail(newException(Exception, "fail"))
node.appendEntriesTimer.fail(newException(Exception, "fail"))
proc RaftNodeStop*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]) =
2023-09-03 00:53:48 +00:00
# Try to stop gracefully
node.state = rnsStopped
# Cancel pending timers (if any)
var f = RaftNodeCancelAllTimers(node)
proc RaftNodeStart*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]) =
2023-09-03 00:53:48 +00:00
node.state = rnsFollower
asyncSpawn RaftNodeScheduleHeartBeatTimeout(node)
debug "Start Raft Node with ID: ", nodeid=node.id