From 5bd50081dae82f0ae26fc4b64a9b7efe59847e16 Mon Sep 17 00:00:00 2001 From: Raycho Mukelov Date: Fri, 13 Oct 2023 09:03:42 +0300 Subject: [PATCH] Fix compilation --- raft/async_util.nim | 2 +- raft/consensus_module.nim | 4 ++-- raft/log_ops.nim | 8 +++++--- raft/raft_api.nim | 12 +++++------- raft/types.nim | 8 ++++---- tests/basic_cluster.nim | 2 +- tests/test_basic_cluster_election.nim | 2 +- 7 files changed, 19 insertions(+), 19 deletions(-) diff --git a/raft/async_util.nim b/raft/async_util.nim index 143dd50..3cd08d7 100644 --- a/raft/async_util.nim +++ b/raft/async_util.nim @@ -11,7 +11,7 @@ import chronos template awaitWithTimeout*[T](operation: Future[T], deadline: Future[void], - body: untyped): T = + body: untyped) = let f {.inject.} = operation await f or deadline if not f.finished: diff --git a/raft/consensus_module.nim b/raft/consensus_module.nim index 0286346..8b8e6e8 100644 --- a/raft/consensus_module.nim +++ b/raft/consensus_module.nim @@ -136,8 +136,8 @@ proc raftNodeHandleAppendEntries*[SmCommandType, SmStateType](node: RaftNode[SmC raftNodeLogTruncate(node, msg.prevLogIndex) return - if msg.entries.len > 0: - for entry in msg.entries: + if msg.logEntries.isSome: + for entry in msg.logEntries.get: raftNodeLogAppend(node, entry) if msg.commitIndex > node.commitIndex: diff --git a/raft/log_ops.nim b/raft/log_ops.nim index 924c22e..6dfc6e0 100644 --- a/raft/log_ops.nim +++ b/raft/log_ops.nim @@ -8,6 +8,7 @@ # those terms. import types +import chronicles # Private Log Ops proc raftNodeLogIndexGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType]): RaftLogIndex = @@ -20,10 +21,11 @@ proc raftNodeLogEntryGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandTy proc raftNodeLogAppend*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], logEntry: RaftNodeLogEntry[SmCommandType]) = node.log.logData.add(logEntry) -proc raftNodeLogTruncate*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], truncateIndex: uint64) = - node.log.logData.truncate(truncateIndex) +proc raftNodeLogTruncate*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], truncateIndex: RaftLogIndex) = + debug "Truncating log to index: ", truncateIndex=truncateIndex, ld=repr(node.log.logData) + # node.log.logData = node.log.logData[:truncateIndex] proc raftNodeApplyLogEntry*[SmCommandType, SmStateType](node: RaftNode[SmCommandType, SmStateType], logEntry: RaftNodeLogEntry[SmCommandType]) = mixin raftNodeSmApply - raftNodeSmApply(node.stateMachine, logEntry.command) \ No newline at end of file + raftNodeSmApply(node.stateMachine, logEntry.data.get) \ No newline at end of file diff --git a/raft/raft_api.nim b/raft/raft_api.nim index e4adcca..2369b28 100644 --- a/raft/raft_api.nim +++ b/raft/raft_api.nim @@ -23,7 +23,7 @@ export chronicles # Forward declarations -proc raftNodeSmInit[SmCommandType, SmStateType](stateMachine: var RaftNodeStateMachine[SmCommandType, SmStateType]) +proc raftNodeSmInit*[SmCommandType, SmStateType](stateMachine: var RaftNodeStateMachine[SmCommandType, SmStateType]) # Raft Node Public API proc new*[SmCommandType, SmStateType](T: type RaftNode[SmCommandType, SmStateType]; @@ -112,17 +112,15 @@ func raftNodeSmStateGet*[SmCommandType, SmStateType](node: RaftNode[SmCommandTyp withRLock(node.raftStateMutex): node.stateMachine.state -proc raftNodeSmInit[SmCommandType, SmStateType](stateMachine: var RaftNodeStateMachine[SmCommandType, SmStateType]) = +proc raftNodeSmInit*[SmCommandType, SmStateType](stateMachine: var RaftNodeStateMachine[SmCommandType, SmStateType]) = mixin raftSmInit - withRLock(node.raftStateMutex): - raftSmInit(stateMachine) + raftSmInit(stateMachine) -proc raftNodeSmApply[SmCommandType, SmStateType](stateMachine: RaftNodeStateMachine[SmCommandType, SmStateType], command: SmCommandType) = +proc raftNodeSmApply*[SmCommandType, SmStateType](stateMachine: RaftNodeStateMachine[SmCommandType, SmStateType], command: SmCommandType) = mixin raftSmApply - withRLock(node.raftStateMutex): - raftSmApply(stateMachine, command) + raftSmApply(stateMachine, command) # Private Abstract Timer creation template raftTimerCreate*(timerInterval: int, timerCallback: RaftTimerCallback): Future[void] = diff --git a/raft/types.nim b/raft/types.nim index 4c46ae4..702af62 100644 --- a/raft/types.nim +++ b/raft/types.nim @@ -37,7 +37,7 @@ type RaftNodeTerm* = int # Raft Node Term Type RaftLogIndex* = int # Raft Node Log Index Type - RaftNodePeer* = ref object # Raft Node Peer object + RaftNodePeer* = ref object # Raft Node Peer object id*: RaftNodeId nextIndex*: RaftLogIndex # For each peer Raft Node, index of the next log entry to send to that Node # (initialized to leader last log index + 1) @@ -51,7 +51,7 @@ type # Raft Node Abstract State Machine type - RaftNodeStateMachine*[SmCommandType, SmStateType] = ref object # Some opaque State Machine Impelementation to be used by the Raft Node + RaftNodeStateMachine*[SmCommandType, SmStateType] = ref object # Some opaque State Machine Impelementation to be used by the Raft Node # providing at minimum operations for initialization, querying the current state # and RaftNodeLogEntry (SmCommandType) application state*: ref SmStateType @@ -147,8 +147,8 @@ type currentTerm*: RaftNodeTerm # Latest term this Raft Node has seen (initialized to 0 on first boot, increases monotonically) votedFor*: RaftNodeId # Candidate RaftNodeId that received vote in current term (or DefaultUUID if none), # also used to redirect Client Requests in case this Raft Node is not the leader - log*: RaftNodeLog[SmCommandType] # This Raft Node Log - stateMachine*: RaftNodeStateMachine[SmCommandType, SmStateType] # Not sure for now putting it here. I assume that persisting the State Machine's + log*: RaftNodeLog[SmCommandType] # This Raft Node Log + stateMachine*: RaftNodeStateMachine[SmCommandType, SmStateType] # Not sure for now putting it here. I assume that persisting the State Machine's # state is enough to consider it 'persisted' peers*: RaftNodePeers # This Raft Node Peers IDs. I am not sure if this must be persistent or volatile but making it persistent # makes sense for the moment diff --git a/tests/basic_cluster.nim b/tests/basic_cluster.nim index ca9076f..7d0953b 100644 --- a/tests/basic_cluster.nim +++ b/tests/basic_cluster.nim @@ -44,7 +44,7 @@ proc basicRaftClusterClientRequest*(cluster: BasicRaftCluster, req: RaftNodeClie of rncroExecSmCommand: discard -proc basicRaftClusterInit*(nodesIds: seq[RaftNodeId], electionTimeout=5, heartBeatTimeout=5): BasicRaftCluster = +proc basicRaftClusterInit*(nodesIds: seq[RaftNodeId], electionTimeout=150, heartBeatTimeout=150): BasicRaftCluster = new(result) for nodeId in nodesIds: var diff --git a/tests/test_basic_cluster_election.nim b/tests/test_basic_cluster_election.nim index c32b95d..8634371 100644 --- a/tests/test_basic_cluster_election.nim +++ b/tests/test_basic_cluster_election.nim @@ -20,7 +20,7 @@ proc basicClusterElectionMain*() = test "Basic Raft Cluster Init (5 nodes)": for i in 0..4: nodesIds[i] = genUUID() - cluster = basicRaftClusterInit(nodesIds, 3, 3) + cluster = basicRaftClusterInit(nodesIds, 150, 150) check cluster != nil test "Start Basic Raft Cluster and wait it to converge for a 2 seconds interval (Elect a Leader)":