Add preliminary API definitions and project roadmap

This commit is contained in:
Raycho Mukelov 2023-08-09 13:06:34 +03:00 committed by GitHub
parent 2533fb346b
commit 4fa562c4a1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 362 additions and 2 deletions

8
.gitignore vendored Normal file
View File

@ -0,0 +1,8 @@
# ignore all executable files
*
!*.*
!*/
*.exe
*.out
nimcache/
build/

View File

@ -1,4 +1,4 @@
The Nimbus GUI is licensed under the Apache License version 2
nim-raft is licensed under the Apache License version 2
Copyright (c) 2023 Status Research & Development GmbH
-----------------------------------------------------

View File

@ -1,4 +1,4 @@
The Nimbus GUI is licensed under the MIT License
nim-raft is licensed under the MIT License
Copyright (c) 2023 Status Research & Development GmbH
-----------------------------------------------------

33
doc/road-map.md Normal file
View File

@ -0,0 +1,33 @@
# RAFT Consensus Nim library Road-map
## Proposed milestones during the library development
1. Create Nim library package. Implement basic functionality: fully functional RAFT Node and its API. The RAFT Node should be abstract working without network communication by the means of API calls and Callback calls only. The RAFT Node should cover all the functionality described in the RAFT Paper excluding Dynamic Adding/Removing of RAFT Node Peers and Log Compaction. Create appropriate tests.
*Duration: 3 weeks (2 weeks for implementation, 1 week for test creation/testing)*
2. Implement advanced functionality: Log Compaction and Dynamic Adding/Removing of RAFT Node Peers and the corresponding tests. Implement Anti Entropy measures observed in other projects (if appropriate).
*Duration: 3 weeks (2 weeks for implementation, 1 week for test creation/testing)*
3. Integrate the RAFT library in the Nimbus project - define p2p networking deal with serialization etc. Create relevant tests. I guess it is a good idea to add some kind of basic RAFT Node metrics. Optionally implement some of the following enhancements (if appropriate):
- Optimistic pipelining to reduce log replication latency
- Writing to leader's disk in parallel
- Automatic stepping down when the leader loses quorum
- Leadership transfer extension
- Pre-vote protocol
*Duration: 1+ week (?)[^note]*
4. Final testing of the solution. Fix unexpected bugs.
*Duration: 1 week (?)[^note]*
5. Implement any new requirements aroused after milestone 4 completion.
*Duration: 0+ week(s) (?)[^note]*
6. End
---
[^note] Durations marked with an (?) means I am not pretty sure how much this will take.

14
raft.nim Normal file
View File

@ -0,0 +1,14 @@
# 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.
import
raft/raft_api
export
raft_api, types, protocol

27
raft.nimble Normal file
View File

@ -0,0 +1,27 @@
# 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.
mode = ScriptMode.Verbose
packageName = "raft"
version = "0.0.1"
author = "Status Research & Development GmbH"
description = "raft consensus in nim"
license = "Apache License 2.0"
skipDirs = @["tests"]
requires "nim >= 1.6.0"
requires "stew >= 0.1.0"
requires "nimcrypto >= 0.5.4"
requires "unittest2 >= 0.0.4"
requires "chronicles >= 0.10.2"
requires "eth >= 1.0.0"
requires "chronos >= 3.2.0"
# Helper functions

8
raft/anti_entropy.nim Normal file
View File

@ -0,0 +1,8 @@
# 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.

View File

@ -0,0 +1,8 @@
# 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.

View File

@ -0,0 +1,8 @@
# 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.

View File

@ -0,0 +1,8 @@
# 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.

48
raft/protocol.nim Normal file
View File

@ -0,0 +1,48 @@
# 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.
# #
# Raft Messages Protocol definition #
# #
import types
import options
type
# Raft Node Messages OPs
RaftMessageOps* = enum
rmoRequestVote = 0,
rmoAppendLogEntry = 1,
rmoInstallSnapshot = 2 # For dynamic adding of new Raft Nodes
RaftMessagePayloadChecksum* = object # Checksum probably will be a SHA3 hash not sure about this at this point
RaftMessagePayload*[LogEntryDataType] = ref object
data*: RaftNodeLogEntry[LogEntryDataType]
checksum*: RaftMessagePayloadChecksum
RaftMessage*[LogEntryDataType] = ref object of RaftMessageBase
op*: RaftMessageOps # Message Op - Ask For Votes, Append Entry(ies), Install Snapshot etc.
payload*: Option[seq[RaftMessagePayload[LogEntryDataType]]] # Optional Message Payload(s) - e.g. log entry(ies). Will be empty for a Heart-Beat # Heart-Beat will be a message with Append Entry(ies) Op and empty payload
RaftMessageResponse*[SmStateType] = ref object of RaftMessageBase
success*: bool # Indicates success/failure
state*: Option[SmStateType] # Raft Abstract State Machine State
# Raft Node Client Request/Response definitions
RaftNodeClientRequestOps = enum
rncroRequestState = 0,
rncroAppendNewEntry = 1
RaftNodeClientRequest*[LogEntryDataType] = ref object
op*: RaftNodeClientRequestOps
payload*: Option[RaftMessagePayload[LogEntryDataType]] # Optional RaftMessagePayload carrying a Log Entry
RaftNodeClientResponse*[SmStateType] = ref object
success*: bool # Indicate succcess
state*: Option[SmStateType] # Optional Raft Abstract State Machine State
raftNodeRedirectId*: Option[RaftNodeId] # Optional Raft Node ID to redirect the request to in case of failure

70
raft/raft_api.nim Normal file
View File

@ -0,0 +1,70 @@
# 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.
import types
import protocol
export types, protocol
# Raft Node Public API procedures / functions
proc RaftNodeCreateNew*[LogEntryDataType, SmStateType]( # Create New Raft Node
id: RaftNodeId, peers: RaftNodePeers,
persistentStorage: RaftNodePersistentStorage,
msgSendCallback: RaftMessageSendCallback): RaftNode[LogEntryDataType, SmStateType] =
discard
proc RaftNodeLoad*[LogEntryDataType, SmStateType](
persistentStorage: RaftNodePersistentStorage, # Load Raft Node From Storage
msgSendCallback: RaftMessageSendCallback): Result[RaftNode[LogEntryDataType, SmStateType], string] =
discard
proc RaftNodeStop*(node: RaftNode) =
discard
proc RaftNodeStart*(node: RaftNode) =
discard
func RaftNodeIdGet*(node: RaftNode): RaftNodeId = # Get Raft Node ID
discard
func RaftNodeStateGet*(node: RaftNode): RaftNodeState = # Get Raft Node State
discard
func RaftNodeTermGet*(node: RaftNode): RaftNodeTerm = # Get Raft Node Term
discard
func RaftNodePeersGet*(node: RaftNode): RaftNodePeers = # Get Raft Node Peers
discard
func RaftNodeIsLeader*(node: RaftNode): bool = # Check if Raft Node is Leader
discard
proc RaftNodeMessageDeliver*(node: RaftNode, raftMessage: RaftMessageBase): RaftMessageResponse {.discardable.} = # Deliver Raft Message to the Raft Node
discard
proc RaftNodeRequest*(node: RaftNode, req: RaftNodeClientRequest): RaftNodeClientResponse = # Process RaftNodeClientRequest
discard
proc RaftNodeLogIndexGet*(node: RaftNode): RaftLogIndex =
discard
proc RaftNodeLogEntryGet*(node: RaftNode, logIndex: RaftLogIndex): Result[RaftNodeLogEntry, string] =
discard
# Abstract State Machine Ops
func RaftNodeSmStateGet*[LogEntryDataType, SmStateType](node: RaftNode[LogEntryDataType, SmStateType]): SmStateType =
node.stateMachine.state
proc RaftNodeSmInit[LogEntryDataType, SmStateType](stateMachine: var RaftNodeStateMachine[LogEntryDataType, SmStateType]) =
mixin RaftSmInit
RaftSmInit(stateMachine)
proc RaftNodeSmApply[LogEntryDataType, SmStateType](stateMachine: RaftNodeStateMachine[LogEntryDataType, SmStateType], logEntry: LogEntryDataType) =
mixin RaftSmApply
RaftSmApply(stateMachine, logEntry)

120
raft/types.nim Normal file
View File

@ -0,0 +1,120 @@
# 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.
# Raft Node Public Types.
# I guess that at some point these can be moved to a separate file called raft_consensus_types.nim for example
import std/locks
import stew/results
import eth/keyfile
export results
type
# Raft Node basic definitions
Blob* = seq[byte]
RaftNodeState* = enum
UNKNOWN = 0,
FOLLOWER = 1,
LEADER = 2
RaftNodeId* = UUID # UUID uniquely identifying every Raft Node
RaftNodePeers* = seq[RaftNodeId] # List of Raft Node Peers IDs
RaftNodeTerm* = uint64 # Raft Node Term Type
RaftLogIndex* = uint64 # Raft Node Log Index Type
# Raft Node Abstract State Machine type
RaftNodeStateMachine*[LogEntryDataType, SmStateType] = ref object # Some probably opaque State Machine Impelementation to be used by the Raft Node
# providing at minimum operations for initialization, querying the current state
# and RaftNodeLogEntry application
state: SmStateType
# Raft Node Persistent Storage basic definition
RaftNodePersistentStorage* = ref object # Should be some kind of Persistent Transactional Store Wrapper
# Basic modules (algos) definitions
RaftNodeAccessCallback[LogEntryDataType] = proc: RaftNode[LogEntryDataType] {.nimcall, gcsafe.} # This should be implementes as a closure holding the RaftNode
RaftConsensusModule*[LogEntryDataType] = object of RootObj
stateTransitionsFsm: seq[byte] # I plan to use nim.fsm https://github.com/ba0f3/fsm.nim
raftNodeAccessCallback: RaftNodeAccessCallback[LogEntryDataType]
RaftLogCompactionModule*[LogEntryDataType] = object of RootObj
raftNodeAccessCallback: RaftNodeAccessCallback[LogEntryDataType]
RaftMembershipChangeModule*[LogEntryDataType] = object of RootObj
raftNodeAccessCallback: RaftNodeAccessCallback[LogEntryDataType]
# Callback for sending messages out of this Raft Node
RaftMessageId* = UUID # UUID assigned to every Raft Node Message,
# so it can be matched with it's corresponding response etc.
RaftMessageSendCallback* = proc (raftMessage: RaftMessageBase) {.nimcall, gcsafe.} # Callback for Sending Raft Node Messages
# out of this Raft Node. Can be used for broadcasting
# (a Heart-Beat for example)
# Raft Node basic Log definitions
RaftNodeLogEntry*[LogEntryDataType] = ref object # Abstarct Raft Node Log entry containing opaque binary data (Blob etc.)
term*: RaftNodeTerm
data*: LogEntryDataType
RaftNodeLog*[LogEntryDataType] = ref object # Needs more elaborate definition.
# Probably this will be a RocksDB/MDBX/SQLite Store Wrapper etc.
logData*: seq[RaftNodeLogEntry[LogEntryDataType]] # Raft Node Log Data
# Base type for Raft message objects
RaftMessageBase* = ref object of RootObj # Base Type for Raft Node Messages
msgId*: RaftMessageId # Message UUID
senderId*: RaftNodeId # Sender Raft Node ID
senderTerm*: RaftNodeTerm # Sender Raft Node Term
peers*: RaftNodePeers # List of Raft Node IDs, which should receive this message
# Raft Node Object type
RaftNode*[LogEntryDataType, SmStateType] = ref object
# Timers
votingTimout: uint64
heartBeatTimeout: uint64
# etc. timers
# Mtx definitions go here
raftStateMutex: Lock
raftLogMutex: Lock
raftCommMutexReceiveMsg: Lock
raftCommMutexClientResponse: Lock
# Modules (Algos)
consensusModule: RaftConsensusModule[LogEntryDataType]
logCompactionModule: RaftLogCompactionModule[LogEntryDataType]
membershipChangeModule: RaftMembershipChangeModule[LogEntryDataType]
# Misc
msgSendCallback: RaftMessageSendCallback
persistentStorage: RaftNodePersistentStorage
# Persistent state
id: RaftNodeId # This Raft Node ID
state: RaftNodeState # This Raft Node State
currentTerm: RaftNodeTerm # Latest term this Raft Node has seen (initialized to 0 on first boot, increases monotonically)
log: RaftNodeLog[LogEntryDataType] # This Raft Node Log
votedFor: RaftNodeId # Candidate RaftNodeId that received vote in current term (or nil/zero if none),
# also used to redirect Client Requests in case this Raft Node is not the leader
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
stateMachine: RaftNodeStateMachine[LogEntryDataType, SmStateType] # Not sure for now putting it here. I assume that persisting the State Machine's
# state is enough to consider it 'persisted'
# Volatile state
commitIndex: RaftLogIndex # Index of highest log entry known to be committed (initialized to 0, increases monotonically)
lastApplied: RaftLogIndex # Index of highest log entry applied to state machine (initialized to 0, increases monotonically)
# Volatile state on leaders
nextIndex: seq[RaftLogIndex] # For each peer Raft Node, index of the next log entry to send to that Node
# (initialized to leader last log index + 1)
matchIndex: seq[RaftLogIndex] # For each peer Raft Node, index of highest log entry known to be replicated on Node
# (initialized to 0, increases monotonically)

8
tests/all_tests.nim Normal file
View File

@ -0,0 +1,8 @@
# 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.