├── .gitignore
├── raft
├── src
│ ├── test
│ │ ├── resources
│ │ │ └── mockito-extensions
│ │ │ │ └── org.mockito.plugins.MockMaker
│ │ └── kotlin
│ │ │ └── edu
│ │ │ └── ucu
│ │ │ └── raft
│ │ │ ├── state
│ │ │ ├── StateTest.kt
│ │ │ ├── VoteRequestTest.kt
│ │ │ └── AppendRequestTest.kt
│ │ │ ├── actions
│ │ │ ├── VotingActionTest.kt
│ │ │ ├── CommitTest.kt
│ │ │ └── HeartbeatActionTest.kt
│ │ │ ├── clock
│ │ │ └── TermClockTest.kt
│ │ │ ├── utils.kt
│ │ │ ├── log
│ │ │ └── LogTest.kt
│ │ │ ├── RaftControllerTest.kt
│ │ │ └── RaftClusterTesting.kt
│ └── main
│ │ ├── kotlin
│ │ └── edu
│ │ │ └── ucu
│ │ │ └── raft
│ │ │ ├── state
│ │ │ ├── NodeState.kt
│ │ │ └── State.kt
│ │ │ ├── adapters
│ │ │ ├── RaftHandler.kt
│ │ │ └── ClusterNode.kt
│ │ │ ├── grpc
│ │ │ ├── ClusterNodeService.kt
│ │ │ └── GrpcClusterNode.kt
│ │ │ ├── RaftNode.kt
│ │ │ ├── actions
│ │ │ ├── CommitAction.kt
│ │ │ ├── VotingAction.kt
│ │ │ └── HeartbeatAction.kt
│ │ │ ├── log
│ │ │ ├── LogState.kt
│ │ │ └── Log.kt
│ │ │ ├── RaftConfiguration.kt
│ │ │ ├── clock
│ │ │ └── TermClock.kt
│ │ │ └── RaftController.kt
│ │ ├── resources
│ │ └── log4j2.xml
│ │ └── proto
│ │ └── raft.proto
└── build.gradle
├── gradle
└── wrapper
│ ├── gradle-wrapper.jar
│ └── gradle-wrapper.properties
├── Dockerfile
├── key-value-example
├── build.gradle
└── src
│ └── main
│ └── kotlin
│ └── edu
│ └── ucu
│ └── example
│ └── KeyValue.kt
├── settings.gradle
├── .travis.yml
├── LICENSE
├── docker-compose.yml
├── gradlew.bat
├── README.md
└── gradlew
/.gitignore:
--------------------------------------------------------------------------------
1 | .gradle/
2 | build/
3 | .idea/
4 |
5 | **/generated/**
--------------------------------------------------------------------------------
/raft/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker:
--------------------------------------------------------------------------------
1 | mock-maker-inline
--------------------------------------------------------------------------------
/gradle/wrapper/gradle-wrapper.jar:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/AChepurnoi/raft-kotlin/HEAD/gradle/wrapper/gradle-wrapper.jar
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/state/NodeState.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.state
2 |
3 | enum class NodeState {
4 | FOLLOWER, LEADER, CANDIDATE
5 | }
--------------------------------------------------------------------------------
/Dockerfile:
--------------------------------------------------------------------------------
1 | FROM openjdk:alpine
2 | COPY key-value-example/build/libs/key-value-example-1.jar /usr/src/app/
3 | WORKDIR /usr/src/app
4 | CMD java -XX:+PrintFlagsFinal $JAVA_OPTIONS -jar key-value-example-1.jar
--------------------------------------------------------------------------------
/gradle/wrapper/gradle-wrapper.properties:
--------------------------------------------------------------------------------
1 | #Mon Mar 11 22:12:15 EET 2019
2 | distributionBase=GRADLE_USER_HOME
3 | distributionPath=wrapper/dists
4 | zipStoreBase=GRADLE_USER_HOME
5 | zipStorePath=wrapper/dists
6 | distributionUrl=https\://services.gradle.org/distributions/gradle-4.8-all.zip
7 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/adapters/RaftHandler.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.adapters
2 |
3 | import edu.ucu.proto.AppendRequest
4 | import edu.ucu.proto.AppendResponse
5 | import edu.ucu.proto.VoteRequest
6 | import edu.ucu.proto.VoteResponse
7 |
8 | interface RaftHandler {
9 | suspend fun requestVote(request: VoteRequest): VoteResponse?
10 |
11 | suspend fun appendEntries(request: AppendRequest): AppendResponse?
12 | }
--------------------------------------------------------------------------------
/raft/src/main/resources/log4j2.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/adapters/ClusterNode.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.adapters
2 |
3 | import edu.ucu.raft.adapters.RaftHandler
4 |
5 | interface ClusterNode: RaftHandler {
6 |
7 | var nextIndex: Int
8 | var matchIndex: Int
9 |
10 | val nodeId: String
11 |
12 | fun decreaseIndex() {
13 | if (nextIndex > 0) {
14 | nextIndex -= 1
15 | }
16 | }
17 |
18 | fun reinitializeIndex(index: Int) {
19 | nextIndex = index
20 | matchIndex = 0
21 | }
22 | }
--------------------------------------------------------------------------------
/key-value-example/build.gradle:
--------------------------------------------------------------------------------
1 | plugins {
2 | id 'idea'
3 | }
4 |
5 | repositories {
6 | mavenCentral()
7 | mavenLocal()
8 | jcenter()
9 |
10 | }
11 |
12 |
13 | dependencies {
14 | implementation project(":raft")
15 | implementation "io.ktor:ktor-server-netty:1.1.3"
16 |
17 | }
18 |
19 | jar {
20 | manifest {
21 | attributes 'Main-Class': 'edu.ucu.example.KeyValue'
22 | }
23 | from {
24 | configurations.runtimeClasspath.collect {
25 | it.isDirectory() ? it : zipTree(it)
26 | }
27 | }
28 | }
--------------------------------------------------------------------------------
/settings.gradle:
--------------------------------------------------------------------------------
1 | pluginManagement {
2 | repositories {
3 | maven {
4 | url 'http://dl.bintray.com/kotlin/kotlin-eap'
5 | }
6 |
7 | mavenCentral()
8 |
9 | maven {
10 | url 'https://plugins.gradle.org/m2/'
11 | }
12 |
13 | maven {
14 | url 'http://dl.bintray.com/kotlin/kotlin-eap'
15 | }
16 |
17 | maven {
18 | url 'https://plugins.gradle.org/m2/'
19 | }
20 | }
21 | }
22 | rootProject.name = 'raft-kt'
23 | include 'raft'
24 | include 'key-value-example'
25 |
26 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/grpc/ClusterNodeService.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.grpc
2 |
3 | import edu.ucu.proto.*
4 | import edu.ucu.raft.adapters.RaftHandler
5 | import io.grpc.stub.StreamObserver
6 | import kotlinx.coroutines.runBlocking
7 | import mu.KotlinLogging
8 |
9 |
10 | class ClusterNodeService(val raft: RaftHandler) : ClusterNodeGrpc.ClusterNodeImplBase() {
11 |
12 | override fun requestVote(request: VoteRequest, responseObserver: StreamObserver) {
13 | val result = runBlocking { raft.requestVote(request) }
14 | responseObserver.onNext(result)
15 | responseObserver.onCompleted()
16 | }
17 |
18 |
19 | override fun appendEntries(request: AppendRequest, responseObserver: StreamObserver) {
20 | val result = runBlocking {
21 | raft.appendEntries(request)
22 | }
23 | responseObserver.onNext(result)
24 | responseObserver.onCompleted()
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language: java
2 | jdk:
3 | - oraclejdk8
4 |
5 | env:
6 | global:
7 | - secure: "de5pWDR/apqjFr1xbVP0m1SrQB2DiRl0LFGmxkP7j9cXyWpR51RzssqkDwL6gLqJ13+t1Hc0TgRInLA4RFD0GLTzINosQu7PeORncT7v0ihNqwXzLxhDBVAX50+zHGw386lc+DhfyLarkZ2cV12lDVzfr90h5zBQIitGDPCPChlfTKTj20fJmxGTk56h2Sj9WTr0Gpc/NIbZDfoDnlckgXNhN4p9sgDxJWLcSoKB7KUQHWQE2Gf0G93OieG3gsoKfAWydumsm3wj463w4NZ/jeeW3mO+rbzk2eP2FQwb8MOLOYJVWFZ3ExCNQmsu+kxj08KZZ1igptHpF+VBXBv0ZjQiGbWqAKpk0GSLd70nlfVgQXQTzSY6VoY4aQtOY6CJHKMqqovHNj/Y8O3KSd0VRHm07s7w/eXx3UI+nFJM7uwFEwe3Se73zvi2AH0OM6arleBlDXv3b5mdq+033opjtKvITPv0eznKEW6ISEKQS5vxvgzbzyE4F8bjzLJHkRrAfqxJSPkCNJU6ueA7qaOA2qkNXFngjnQyEgGw+/batrkqWG4D6LOfKqertnkqno2Gn7mfapgkmTzdpOTtt1enW2GPofqq4g61pNLt5Rm+/pkPNdERG685DtdVtz/7sMfQshayIh26sGb6m9MEWs//1cPvHK/yMo/CeLi3t25giz4="
8 |
9 |
10 | cache:
11 | directories:
12 | - "$HOME/.gradle/caches/"
13 | - "$HOME/.gradle/wrapper/"
14 |
15 | script:
16 | - "./gradlew test"
17 | - "./gradlew codeCoverageReport"
18 |
19 | after_success:
20 | - "bash <(curl -s https://codecov.io/bash)"
--------------------------------------------------------------------------------
/raft/src/main/proto/raft.proto:
--------------------------------------------------------------------------------
1 | syntax = "proto3";
2 |
3 | option java_multiple_files = true;
4 | option java_package = "edu.ucu.proto";
5 |
6 | package helloworld;
7 |
8 | service ClusterNode {
9 | rpc RequestVote (VoteRequest) returns (VoteResponse) {}
10 | rpc AppendEntries (AppendRequest) returns (AppendResponse){}
11 | }
12 |
13 |
14 | message VoteRequest {
15 | int64 term = 1;
16 | int32 candidateId = 2;
17 | int32 lastLogIndex = 3;
18 | int64 lastLogTerm = 4;
19 |
20 | }
21 |
22 | message VoteResponse {
23 | int64 term = 1;
24 | bool voteGranted = 2;
25 | }
26 |
27 | message AppendRequest{
28 | int64 term = 1;
29 | int32 leaderId = 2;
30 | int32 prevLogIndex = 3;
31 | int64 prevLogTerm = 4;
32 | int32 leaderCommit = 5;
33 | repeated LogEntry entries = 6;
34 |
35 |
36 | }
37 |
38 | message LogEntry {
39 | uint64 term=2;
40 | string command=3;
41 | bytes data=4;
42 | }
43 |
44 | message AppendResponse{
45 | int64 term = 1;
46 | bool success = 2;
47 |
48 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/state/StateTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.state
2 |
3 | import edu.ucu.raft.test
4 | import org.assertj.core.api.Assertions.assertThat
5 | import org.junit.Test
6 | import org.junit.runner.RunWith
7 | import org.junit.runners.BlockJUnit4ClassRunner
8 |
9 | @RunWith(BlockJUnit4ClassRunner::class)
10 | class StateTest {
11 |
12 |
13 | val state = State(5)
14 | val sub = state.updates.openSubscription()
15 |
16 | @Test
17 | fun nextTermConvertsFollowerToCandidate() = test {
18 | val prevTerm = state.term
19 | state.nextTerm(prevTerm + 1)
20 | assertThat(state.current).isEqualTo(NodeState.CANDIDATE)
21 | assertThat(state.term).isEqualTo(prevTerm + 1)
22 | }
23 |
24 | @Test
25 | fun stateChangesAreStreamedToChannel() = test {
26 | state.nextTerm(2)
27 | val (prev, next) = sub.receive()
28 | assertThat(state.current).isEqualTo(NodeState.CANDIDATE)
29 | assertThat(prev).isEqualTo(NodeState.FOLLOWER)
30 | assertThat(next).isEqualTo(NodeState.CANDIDATE)
31 | }
32 |
33 |
34 |
35 | }
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | MIT License
2 |
3 | Copyright (c) 2019 Sasha
4 |
5 | Permission is hereby granted, free of charge, to any person obtaining a copy
6 | of this software and associated documentation files (the "Software"), to deal
7 | in the Software without restriction, including without limitation the rights
8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
9 | copies of the Software, and to permit persons to whom the Software is
10 | furnished to do so, subject to the following conditions:
11 |
12 | The above copyright notice and this permission notice shall be included in all
13 | copies or substantial portions of the Software.
14 |
15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
21 | SOFTWARE.
22 |
--------------------------------------------------------------------------------
/docker-compose.yml:
--------------------------------------------------------------------------------
1 | version: "3"
2 |
3 | services:
4 | node_one:
5 | image: kvex:latest
6 | container_name: node_one
7 | environment:
8 | ID: "50"
9 | PORT: "4000"
10 | NODES: "51:172.20.0.6:4000,52:172.20.0.7:4000"
11 | TIMER: "1000"
12 | build:
13 | dockerfile: Dockerfile
14 | context: .
15 | networks:
16 | default:
17 | ipv4_address: 172.20.0.5
18 |
19 | node_two:
20 | image: kvex:latest
21 | container_name: node_two
22 | environment:
23 | ID: "51"
24 | PORT: "4000"
25 | NODES: "50:172.20.0.5:4000,52:172.20.0.7:4000"
26 | TIMER: "950"
27 | build:
28 | dockerfile: Dockerfile
29 | context: .
30 | networks:
31 | default:
32 | ipv4_address: 172.20.0.6
33 |
34 | node_three:
35 | image: kvex:latest
36 | container_name: node_three
37 | environment:
38 | ID: "52"
39 | PORT: "4000"
40 | NODES: "50:172.20.0.5:4000,51:172.20.0.6:4000"
41 | TIMER: "1050"
42 | build:
43 | dockerfile: Dockerfile
44 | context: .
45 | networks:
46 | default:
47 | ipv4_address: 172.20.0.7
48 |
49 |
50 | networks:
51 | default:
52 | driver: bridge
53 | ipam:
54 | config:
55 | - subnet: 172.20.0.0/24
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/RaftNode.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft
2 |
3 | import edu.ucu.raft.log.Command
4 | import edu.ucu.raft.log.LogState
5 | import edu.ucu.raft.grpc.ClusterNodeService
6 | import edu.ucu.raft.grpc.GrpcClusterNode
7 | import edu.ucu.raft.state.NodeState
8 | import io.grpc.ServerBuilder
9 | import java.lang.RuntimeException
10 |
11 | class RaftNode {
12 |
13 | val config = EnvConfiguration()
14 | private val nodes = config.nodes
15 | .map { (id, host, port) -> GrpcClusterNode(host, port) }
16 |
17 | private val controller = RaftController(config, nodes.toMutableList()).apply { start() }
18 |
19 | private val server = ServerBuilder.forPort(config.port)
20 | .addService(ClusterNodeService(controller))
21 | .build().apply { start() }
22 |
23 | fun getState(): LogState {
24 | return controller.state.log.state()
25 | }
26 |
27 | fun isLeader() = controller.state.current == NodeState.LEADER
28 |
29 | fun leaderNode() = config.hosts[controller.state.leaderId] ?: throw RuntimeException("Leader not found")
30 |
31 | suspend fun applyCommand(command: Command): Boolean {
32 | return controller.applyCommand(command)
33 | }
34 |
35 | fun await() {
36 | server.awaitTermination()
37 | }
38 | }
39 |
40 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/actions/CommitAction.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.actions
2 |
3 | import edu.ucu.raft.adapters.ClusterNode
4 | import edu.ucu.raft.state.NodeState
5 | import edu.ucu.raft.state.State
6 | import mu.KotlinLogging
7 |
8 | class CommitAction(val state: State, val cluster: List) {
9 |
10 | private val log = state.log
11 | private val logger = KotlinLogging.logger {}
12 |
13 |
14 | fun perform() {
15 | if (state.current == NodeState.LEADER) {
16 | val newCommit = ((state.log.commitIndex + 1)..Int.MAX_VALUE)
17 | .takeWhile { newCommit ->
18 | val clusterApprove = matchIndexMatches(newCommit)
19 | val logLastTermMatch = log[newCommit]?.term == state.term
20 | clusterApprove && logLastTermMatch
21 | }
22 | .lastOrNull()
23 | newCommit?.run {
24 | logger.info { "Doing commit at $newCommit" }
25 | log.commit(this)
26 | }
27 | }
28 | }
29 |
30 | private fun matchIndexMatches(newCommit: Int): Boolean {
31 | val majority = Math.floorDiv(cluster.size, 2)
32 | return cluster.filter { it.matchIndex >= newCommit }.count() > majority
33 |
34 | }
35 |
36 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/actions/VotingActionTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.actions
2 |
3 | import edu.ucu.raft.state.State
4 | import edu.ucu.raft.test
5 | import edu.ucu.raft.testNode
6 | import org.assertj.core.api.Assertions.assertThat
7 | import org.junit.Test
8 | import org.junit.runner.RunWith
9 | import org.junit.runners.BlockJUnit4ClassRunner
10 |
11 | @RunWith(BlockJUnit4ClassRunner::class)
12 | class VotingActionTest {
13 |
14 |
15 | @Test
16 | fun candidateCanGetVotes() = test {
17 | val state = State(1)
18 | val stateTwo = State(2)
19 | val stateThree = State(3)
20 |
21 | val voting = VotingAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
22 | state.nextTerm(2)
23 | val votingResult = voting.askVotes()
24 |
25 | assertThat(votingResult).isTrue()
26 | }
27 |
28 |
29 | @Test
30 | fun candidateWithStaleTermCantGetVotes() = test {
31 | val state = State(1, term = 0)
32 | val stateTwo = State(2, term = 2)
33 | val stateThree = State(3, term = 2)
34 |
35 | val voting = VotingAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
36 |
37 | state.nextTerm(1)
38 | val votingResult = voting.askVotes()
39 |
40 | assertThat(votingResult).isFalse()
41 | }
42 | }
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/log/LogState.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.log
2 |
3 | import com.google.protobuf.ByteString
4 | import java.io.ByteArrayInputStream
5 | import java.io.ByteArrayOutputStream
6 | import java.io.ObjectInputStream
7 | import java.io.ObjectOutputStream
8 | import java.io.Serializable
9 | import java.util.*
10 |
11 |
12 | sealed class Command: Serializable{
13 | abstract fun update(logState: LogState): LogState
14 |
15 | fun toBytes(): ByteString{
16 | val baos = ByteArrayOutputStream()
17 | ObjectOutputStream(baos).use { it.writeObject(this) }
18 | val b64 = Base64.getEncoder().encodeToString(baos.toByteArray())
19 | return ByteString.copyFromUtf8(b64)
20 | }
21 |
22 |
23 | companion object {
24 | fun fromBytes(byteArray: ByteArray): Command{
25 | val bytes = Base64.getDecoder().decode(String(byteArray))
26 | val readObject = ObjectInputStream(ByteArrayInputStream(bytes)).readObject()
27 | return readObject as Command
28 | }
29 | }
30 | }
31 |
32 | class Set(private val key: String, private val value: ByteArray) : Command() {
33 | override fun update(logState: LogState): LogState = logState.copy(data = logState.data.toMutableMap()
34 | .apply {
35 | put(key, value)
36 | })
37 | }
38 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/actions/VotingAction.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.actions
2 |
3 | import edu.ucu.proto.VoteRequest
4 | import edu.ucu.proto.VoteResponse
5 | import edu.ucu.raft.adapters.ClusterNode
6 | import edu.ucu.raft.state.State
7 | import kotlinx.coroutines.GlobalScope
8 | import kotlinx.coroutines.async
9 | import kotlinx.coroutines.withTimeoutOrNull
10 |
11 | class VotingAction(val state: State, val cluster: List) {
12 |
13 | private fun checkTerm(response: VoteResponse) {
14 | if (response.term > this.state.term) {
15 |
16 | }
17 | }
18 |
19 | suspend fun askVotes(): Boolean {
20 | if (cluster.isEmpty()) return false
21 | val majority = Math.floorDiv(cluster.size, 2)
22 | val request = VoteRequest.newBuilder().setTerm(state.term).setCandidateId(state.id)
23 | .setLastLogIndex(state.log.lastIndex())
24 | .setLastLogTerm(state.log.lastTerm() ?: -1).build()
25 |
26 | val responses = cluster.map { node -> GlobalScope.async { node.requestVote(request) } }
27 | .map { withTimeoutOrNull(200) { it.await() } }
28 | .filterNotNull()
29 |
30 | responses.forEach { checkTerm(it) }
31 | val votes = responses.filter { it.voteGranted }.count()
32 |
33 | return votes >= majority
34 |
35 | }
36 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/clock/TermClockTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.clock
2 |
3 | import edu.ucu.raft.clock.TermClock
4 | import kotlinx.coroutines.runBlocking
5 | import kotlinx.coroutines.withTimeoutOrNull
6 | import org.assertj.core.api.Assertions.assertThat
7 | import org.junit.Test
8 | import org.junit.runner.RunWith
9 | import org.junit.runners.BlockJUnit4ClassRunner
10 |
11 | @RunWith(BlockJUnit4ClassRunner::class)
12 | class TermClockTest {
13 |
14 |
15 | @Test(timeout = 10000)
16 | fun clockTest() {
17 | runBlocking {
18 | val clock = TermClock(1000)
19 | clock.start()
20 | val subscription = clock.channel.openSubscription()
21 |
22 | val term = subscription.receive()
23 | println("Recieved $term")
24 | assertThat(term).isEqualTo(1L)
25 |
26 | clock.freeze()
27 |
28 | val result = withTimeoutOrNull(2000) {
29 | subscription.receive()
30 | }
31 | assertThat(result).isNull()
32 | clock.start()
33 |
34 | val termTwo = subscription.receive()
35 | println("Recieved $termTwo")
36 | assertThat(termTwo).isEqualTo(2L)
37 |
38 |
39 | val termThree = subscription.receive()
40 | println("Recieved $termThree")
41 | assertThat(termThree).isEqualTo(3L)
42 |
43 | }
44 |
45 | }
46 | }
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/grpc/GrpcClusterNode.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.grpc
2 |
3 | import edu.ucu.proto.*
4 | import edu.ucu.raft.adapters.ClusterNode
5 | import io.grpc.ManagedChannelBuilder
6 | import mu.KotlinLogging
7 | import kotlin.coroutines.resume
8 | import kotlin.coroutines.suspendCoroutine
9 |
10 |
11 | class GrpcClusterNode(val host: String, val port: Int) : ClusterNode {
12 |
13 | private val logger = KotlinLogging.logger {}
14 |
15 | @Volatile
16 | override var nextIndex: Int = 0
17 | @Volatile
18 | override var matchIndex: Int = -1
19 |
20 | override val nodeId: String = "$host:$port"
21 |
22 | private val channel = ManagedChannelBuilder.forAddress(host, port).usePlaintext().build()
23 | private val stub = ClusterNodeGrpc.newBlockingStub(channel)
24 |
25 |
26 | override suspend fun requestVote(request: VoteRequest): VoteResponse? {
27 | return suspendCoroutine {
28 | it.resume(kotlin.runCatching { stub.requestVote(request) }.getOrNull())
29 | }
30 | }
31 |
32 |
33 | override suspend fun appendEntries(request: AppendRequest): AppendResponse? {
34 | return suspendCoroutine {
35 | it.resume(kotlin.runCatching { stub.appendEntries(request) }.getOrNull())
36 | }
37 | }
38 |
39 | override fun reinitializeIndex(index: Int) {
40 | logger.info { "Index for node $host:$port reinitialized" }
41 | super.reinitializeIndex(index)
42 | }
43 |
44 | }
45 |
--------------------------------------------------------------------------------
/raft/build.gradle:
--------------------------------------------------------------------------------
1 | plugins {
2 | id 'com.google.protobuf' version '0.8.5'
3 | id 'idea'
4 | }
5 |
6 | repositories {
7 | mavenCentral()
8 | mavenLocal()
9 | }
10 |
11 | def grpcVersion = '1.18.0' // CURRENT_GRPC_VERSION
12 |
13 | dependencies {
14 | implementation "io.grpc:grpc-netty-shaded:${grpcVersion}"
15 | implementation "io.grpc:grpc-protobuf:${grpcVersion}"
16 | implementation "io.grpc:grpc-stub:${grpcVersion}"
17 | implementation 'io.github.microutils:kotlin-logging:1.6.24'
18 | implementation 'org.apache.logging.log4j:log4j-slf4j-impl:2.9.1'
19 | implementation 'org.apache.logging.log4j:log4j-api:2.9.1'
20 | implementation 'org.apache.logging.log4j:log4j-core:2.9.1'
21 | implementation 'org.slf4j:slf4j-api:1.7.25'
22 |
23 | compileOnly "javax.annotation:javax.annotation-api:1.2"
24 |
25 | testImplementation "io.grpc:grpc-testing:${grpcVersion}" // gRCP testing utilities
26 | testImplementation "junit:junit:4.12"
27 | testImplementation "org.mockito:mockito-core:2.23.0"
28 | testImplementation "org.assertj:assertj-core:3.11.1"
29 | }
30 |
31 | idea {
32 | module {
33 | sourceDirs += file("${projectDir}/src/generated/main/java");
34 | sourceDirs += file("${projectDir}/src/generated/main/grpc");
35 | }
36 | }
37 |
38 | protobuf {
39 | protoc { artifact = 'com.google.protobuf:protoc:3.7.0' }
40 | plugins {
41 | grpc { artifact = "io.grpc:protoc-gen-grpc-java:${grpcVersion}" }
42 | }
43 | generatedFilesBaseDir = "$projectDir/src/generated"
44 | generateProtoTasks {
45 | all()*.plugins { grpc {} }
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/RaftConfiguration.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft
2 |
3 | data class RaftNodeData(val id: Int, val host: String, val port: Int)
4 |
5 | interface RaftConfiguration {
6 | val port: Int
7 | val nodes: List
8 | val hosts: Map
9 | val id: Int
10 | val timerInterval: Long
11 | val heartbeatInterval: Long
12 | }
13 |
14 | class StaticConfiguration(override val id: Int, override val port: Int = 4000,
15 | override val timerInterval: Long = 1000,
16 | override val heartbeatInterval: Long = 500) : RaftConfiguration {
17 | override val nodes: List = mutableListOf()
18 | override val hosts: Map = emptyMap()
19 | }
20 |
21 | class EnvConfiguration : RaftConfiguration {
22 |
23 | val env = System.getenv()
24 | override val port = env.getOrDefault("PORT", "4040").toInt()
25 | override val nodes: List = env.getOrDefault("NODES", "50:localhost:4040")
26 | .split(",")
27 | .map {
28 | val uri = it.split(":")
29 | RaftNodeData(uri[0].toInt(), uri[1], uri[2].toInt())
30 | }.toList()
31 |
32 | override val hosts: Map = nodes.groupBy { it.id }.mapValues { (_, v) -> v.first() }
33 |
34 |
35 | override val id = env.getOrDefault("ID", (Math.random() * 100).toInt().toString()).toInt()
36 |
37 | override val timerInterval: Long = env.getOrDefault("TIMER", "500").toLong()
38 |
39 | override val heartbeatInterval: Long = env.getOrDefault("HEARTBEAT_TIMER", "50").toLong()
40 |
41 |
42 | }
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/clock/TermClock.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.clock
2 |
3 | import kotlinx.coroutines.*
4 | import kotlinx.coroutines.channels.BroadcastChannel
5 | import kotlinx.coroutines.channels.ConflatedBroadcastChannel
6 | import kotlinx.coroutines.sync.Mutex
7 | import kotlinx.coroutines.sync.withLock
8 | import mu.KotlinLogging
9 | import java.util.*
10 | import java.util.concurrent.locks.ReentrantLock
11 | import kotlin.concurrent.fixedRateTimer
12 |
13 | class TermClock(private val interval: Long) {
14 | private val logger = KotlinLogging.logger {}
15 |
16 |
17 | val channel = ConflatedBroadcastChannel()
18 | var term: Long = 0
19 | private set
20 |
21 | private val mutex: Mutex = Mutex(false)
22 | private var stopped = true
23 | private lateinit var timer: Timer
24 |
25 | private suspend fun updateTerm(term: Long) {
26 | logger.info { "⏳ Term ${this.term} -> $term" }
27 | this.term = term
28 | channel.send(term)
29 | }
30 |
31 | suspend fun start() {
32 | mutex.withLock {
33 | if (stopped) {
34 | stopped = false
35 | schedule()
36 | }
37 | }
38 | }
39 |
40 | private fun schedule() {
41 | timer = fixedRateTimer(initialDelay = interval, period = interval) {
42 | runBlocking { updateTerm(term + 1) }
43 | }
44 | }
45 |
46 | suspend fun update(newTerm: Long) {
47 | mutex.withLock{
48 | this.term = newTerm
49 | }
50 | }
51 |
52 | suspend fun reset() {
53 | mutex.withLock {
54 | if (stopped) return
55 | timer.cancel()
56 | schedule()
57 | }
58 |
59 | }
60 |
61 | suspend fun freeze() {
62 | mutex.withLock {
63 | if (!stopped) {
64 | stopped = true
65 | timer.cancel()
66 | }
67 | }
68 | }
69 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/utils.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft
2 |
3 | import edu.ucu.proto.AppendRequest
4 | import edu.ucu.proto.AppendResponse
5 | import edu.ucu.proto.VoteRequest
6 | import edu.ucu.proto.VoteResponse
7 | import edu.ucu.raft.adapters.ClusterNode
8 | import edu.ucu.raft.state.State
9 | import kotlinx.coroutines.runBlocking
10 |
11 |
12 | fun test(body: suspend () -> Unit) {
13 | runBlocking {
14 | body()
15 | }
16 | }
17 |
18 | class TestRaftNode(val state: State) : ClusterNode {
19 | override val nodeId: String = "TestNode-${state.id}"
20 |
21 | override var nextIndex: Int = state.log.lastIndex() + 1
22 | override var matchIndex: Int = -1
23 |
24 | override suspend fun requestVote(request: VoteRequest): VoteResponse = state.requestVote(request)
25 | override suspend fun appendEntries(request: AppendRequest): AppendResponse = state.appendEntries(request)
26 |
27 | }
28 |
29 |
30 | class LocalRaftNode(val controller: RaftController) : ClusterNode {
31 |
32 | override var nextIndex: Int = controller.state.log.lastIndex() + 1
33 | override var matchIndex: Int = -1
34 | override val nodeId: String = "LocalTestNode-${controller.state.id}"
35 |
36 | private var isolated = true
37 |
38 | fun isolate() {
39 | if (!isolated) {
40 | isolated = true
41 | controller.stop()
42 | }
43 | }
44 |
45 | fun activate() {
46 | if (isolated) {
47 | isolated = false
48 | controller.start()
49 | }
50 |
51 | }
52 |
53 | override suspend fun requestVote(request: VoteRequest): VoteResponse? =
54 | if (isolated) null else controller.requestVote(request)
55 |
56 | override suspend fun appendEntries(request: AppendRequest): AppendResponse? =
57 | if (isolated) null else controller.appendEntries(request)
58 |
59 | override fun toString(): String {
60 | return "LocalRaftNode(controller=$controller, nextIndex=$nextIndex, matchIndex=$matchIndex, nodeId='$nodeId')"
61 | }
62 |
63 | }
64 |
65 | fun State.testNode(): TestRaftNode {
66 | return TestRaftNode(this)
67 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/log/LogTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.log
2 |
3 | import edu.ucu.proto.LogEntry
4 | import org.assertj.core.api.Assertions.assertThat
5 | import org.junit.Test
6 | import org.junit.runner.RunWith
7 | import org.junit.runners.BlockJUnit4ClassRunner
8 |
9 | @RunWith(BlockJUnit4ClassRunner::class)
10 | class LogTest {
11 |
12 |
13 | @Test
14 | fun logCanAppend() {
15 | val log = Log()
16 | log.append(LogEntry.getDefaultInstance())
17 | log.append(LogEntry.getDefaultInstance())
18 | log.append(LogEntry.getDefaultInstance())
19 |
20 | assertThat(log.lastIndex()).isEqualTo(2)
21 | }
22 |
23 | @Test
24 | fun startingCanFetchLog() {
25 | val log = Log()
26 | log.append(LogEntry.getDefaultInstance())
27 | log.append(LogEntry.getDefaultInstance())
28 | log.append(LogEntry.getDefaultInstance())
29 |
30 | assertThat(log.starting(-1)).hasSize(3)
31 | assertThat(log.starting(0)).hasSize(3)
32 | assertThat(log.starting(1)).hasSize(2)
33 | assertThat(log.starting(2)).hasSize(1)
34 | assertThat(log.starting(3)).hasSize(0)
35 | assertThat(log.starting(4)).hasSize(0)
36 | assertThat(log.starting(5)).hasSize(0)
37 | assertThat(log.starting(6)).hasSize(0)
38 | }
39 |
40 |
41 | @Test
42 | fun commandCanBeSerializedAndDeserialized() {
43 | val set = Set("key", "value".toByteArray())
44 |
45 | val entry = LogEntry.newBuilder().setCommandBytes(set.toBytes()).build()
46 |
47 | val bytes = entry.commandBytes.toByteArray()
48 | val readed = Command.fromBytes(bytes)
49 |
50 | assertThat(readed).isInstanceOf(Set::class.java)
51 | }
52 |
53 |
54 | @Test
55 | fun logStateCanBeConstructed() {
56 | val log = Log()
57 | val set = Set("key", "value".toByteArray())
58 | val entry = LogEntry.newBuilder().setCommandBytes(set.toBytes()).build()
59 | log.append(entry)
60 | log.commit(0)
61 | val state = log.state()
62 | val s = String(state["key"]!!)
63 | assertThat(s).isEqualTo("value")
64 |
65 | }
66 | }
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/log/Log.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.log
2 |
3 | import edu.ucu.proto.LogEntry
4 | import kotlin.collections.ArrayList
5 | import kotlin.math.min
6 |
7 |
8 |
9 | data class LogState(val data: Map = emptyMap()) {
10 | operator fun get(key: String): ByteArray? = data[key]
11 | }
12 |
13 | class Log {
14 |
15 | @Volatile
16 | private var entries: MutableList = ArrayList()
17 |
18 | var commitIndex: Int = -1
19 | private set
20 |
21 | fun lastIndex() = entries.size - 1
22 |
23 | fun lastTerm(): Long? = entries.lastOrNull()?.term
24 |
25 | fun isNotEmpty(): Boolean = entries.isNotEmpty()
26 |
27 |
28 | fun state(): LogState {
29 | return entries.take(commitIndex + 1)
30 | .fold(LogState()) { state: LogState, entry: LogEntry ->
31 | Command.fromBytes(entry.commandBytes.toByteArray()).update(state)
32 | }
33 | }
34 |
35 | fun append(value: LogEntry): Int {
36 | val targetIndex = entries.size
37 | entries.add(value)
38 | return targetIndex
39 | }
40 |
41 | fun commit(index: Int): Boolean {
42 | val idx = min(lastIndex(), index)
43 | commitIndex = idx
44 | return true
45 | }
46 |
47 | fun starting(index: Int): List {
48 | return entries.filterIndexed { i, _ -> i >= index }
49 | }
50 |
51 | operator fun get(prevLogIndex: Int) = entries.getOrNull(prevLogIndex)
52 |
53 | operator fun set(idx: Int, value: LogEntry) {
54 | if (idx == entries.size) {
55 | entries.add(value)
56 | } else {
57 | entries[idx] = value
58 | }
59 | }
60 |
61 | fun prune(startIndex: Int) {
62 |
63 | val drop = entries.size - startIndex
64 | entries = entries.dropLast(drop).toMutableList()
65 |
66 | }
67 |
68 | override fun toString(): String {
69 | val commited = (0..commitIndex).map { "■" }.joinToString(separator = "")
70 | val uncommited = ((commitIndex + 1)..lastIndex()).map { "□" }.joinToString(separator = "")
71 | return "[${commited + uncommited}](${commitIndex + 1}/${entries.size})"
72 | }
73 |
74 |
75 | }
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/actions/HeartbeatAction.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.actions
2 |
3 | import edu.ucu.proto.AppendRequest
4 | import edu.ucu.raft.adapters.ClusterNode
5 | import edu.ucu.raft.state.NodeState
6 | import edu.ucu.raft.state.State
7 | import kotlinx.coroutines.GlobalScope
8 | import kotlinx.coroutines.async
9 | import kotlinx.coroutines.withTimeoutOrNull
10 | import mu.KotlinLogging
11 | import java.lang.RuntimeException
12 |
13 | class HeartbeatAction(val state: State, val cluster: List) {
14 |
15 | private val logger = KotlinLogging.logger {}
16 |
17 | private val log = state.log
18 |
19 | suspend fun send() {
20 | if (state.current == NodeState.LEADER) {
21 | cluster.map {
22 | val prevIndex = it.nextIndex - 1
23 | val prevTerm = if (prevIndex != -1) log[prevIndex]?.term ?: throw RuntimeException("WAT") else -1
24 |
25 | val entries = log.starting(prevIndex + 1)
26 | val request = AppendRequest.newBuilder()
27 | .setTerm(state.term).setLeaderId(state.id).setLeaderCommit(state.log.commitIndex)
28 | .setPrevLogIndex(prevIndex).setPrevLogTerm(prevTerm)
29 | .addAllEntries(entries)
30 | .build()
31 |
32 |
33 | GlobalScope.async {
34 | val response = it.appendEntries(request)
35 | if (response == null) null else it to response
36 | }
37 | }.map { withTimeoutOrNull(200) { it.await() } }
38 | .filterNotNull()
39 | .forEach { (node, response) ->
40 | when {
41 | response.success -> {
42 | node.nextIndex = log.lastIndex() + 1
43 | node.matchIndex = node.nextIndex - 1
44 | }
45 | !response.success -> {
46 | logger.info { "Heartbeat response: ${response.success}-${response.term}" }
47 | node.decreaseIndex()
48 | }
49 | }
50 | }
51 |
52 |
53 | }
54 | }
55 |
56 | }
--------------------------------------------------------------------------------
/key-value-example/src/main/kotlin/edu/ucu/example/KeyValue.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.example
2 |
3 | import edu.ucu.raft.RaftNode
4 | import edu.ucu.raft.log.Set
5 | import io.ktor.application.call
6 | import io.ktor.http.ContentType
7 | import io.ktor.http.HttpStatusCode
8 | import io.ktor.request.receiveText
9 | import io.ktor.response.header
10 | import io.ktor.response.respond
11 | import io.ktor.response.respondText
12 | import io.ktor.routing.get
13 | import io.ktor.routing.post
14 | import io.ktor.routing.routing
15 | import io.ktor.server.engine.embeddedServer
16 | import io.ktor.server.netty.Netty
17 | import kotlinx.io.core.String
18 |
19 | object KeyValue {
20 |
21 | val node = RaftNode()
22 | val serverPortInc = 4000
23 |
24 | @JvmStatic
25 | fun main(args: Array) {
26 | embeddedServer(Netty, serverPortInc + node.config.port) {
27 | routing {
28 | get("/{key}") {
29 | val key = call.parameters["key"]!!
30 | if (node.isLeader()) {
31 | val data = node.getState()[key]?.let { String(it) } ?: ""
32 | call.respondText(data, ContentType.Text.Plain)
33 | } else {
34 | val leader = node.leaderNode()
35 | call.response.header("Location", "http://${leader.host}:${leader.port + serverPortInc}/$key")
36 | call.respond(HttpStatusCode.TemporaryRedirect, "redirected to master")
37 | }
38 | }
39 | post("/{key}") {
40 | val key = call.parameters["key"]!!
41 | if (node.isLeader()) {
42 | val data = call.receiveText()
43 | val result = node.applyCommand(Set(key, data.toByteArray()))
44 | call.respondText("Result: $result")
45 | } else {
46 | val leader = node.leaderNode()
47 | call.response.header("Location", "http://${leader.host}:${leader.port + serverPortInc}/$key")
48 | call.respond(HttpStatusCode.TemporaryRedirect, "redirected to master")
49 | }
50 |
51 | }
52 | }
53 | }.start(true)
54 | node.await()
55 | }
56 | }
--------------------------------------------------------------------------------
/gradlew.bat:
--------------------------------------------------------------------------------
1 | @if "%DEBUG%" == "" @echo off
2 | @rem ##########################################################################
3 | @rem
4 | @rem Gradle startup script for Windows
5 | @rem
6 | @rem ##########################################################################
7 |
8 | @rem Set local scope for the variables with windows NT shell
9 | if "%OS%"=="Windows_NT" setlocal
10 |
11 | set DIRNAME=%~dp0
12 | if "%DIRNAME%" == "" set DIRNAME=.
13 | set APP_BASE_NAME=%~n0
14 | set APP_HOME=%DIRNAME%
15 |
16 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
17 | set DEFAULT_JVM_OPTS=
18 |
19 | @rem Find java.exe
20 | if defined JAVA_HOME goto findJavaFromJavaHome
21 |
22 | set JAVA_EXE=java.exe
23 | %JAVA_EXE% -version >NUL 2>&1
24 | if "%ERRORLEVEL%" == "0" goto init
25 |
26 | echo.
27 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
28 | echo.
29 | echo Please set the JAVA_HOME variable in your environment to match the
30 | echo location of your Java installation.
31 |
32 | goto fail
33 |
34 | :findJavaFromJavaHome
35 | set JAVA_HOME=%JAVA_HOME:"=%
36 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe
37 |
38 | if exist "%JAVA_EXE%" goto init
39 |
40 | echo.
41 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME%
42 | echo.
43 | echo Please set the JAVA_HOME variable in your environment to match the
44 | echo location of your Java installation.
45 |
46 | goto fail
47 |
48 | :init
49 | @rem Get command-line arguments, handling Windows variants
50 |
51 | if not "%OS%" == "Windows_NT" goto win9xME_args
52 |
53 | :win9xME_args
54 | @rem Slurp the command line arguments.
55 | set CMD_LINE_ARGS=
56 | set _SKIP=2
57 |
58 | :win9xME_args_slurp
59 | if "x%~1" == "x" goto execute
60 |
61 | set CMD_LINE_ARGS=%*
62 |
63 | :execute
64 | @rem Setup the command line
65 |
66 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar
67 |
68 | @rem Execute Gradle
69 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS%
70 |
71 | :end
72 | @rem End local scope for the variables with windows NT shell
73 | if "%ERRORLEVEL%"=="0" goto mainEnd
74 |
75 | :fail
76 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of
77 | rem the _cmd.exe /c_ return code!
78 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1
79 | exit /b 1
80 |
81 | :mainEnd
82 | if "%OS%"=="Windows_NT" endlocal
83 |
84 | :omega
85 |
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/actions/CommitTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.actions
2 |
3 | import edu.ucu.raft.log.Log
4 | import edu.ucu.proto.LogEntry
5 | import edu.ucu.raft.state.NodeState
6 | import edu.ucu.raft.state.State
7 | import edu.ucu.raft.testNode
8 | import org.assertj.core.api.Assertions.assertThat
9 | import org.junit.Test
10 | import org.junit.runner.RunWith
11 | import org.junit.runners.BlockJUnit4ClassRunner
12 |
13 |
14 | @RunWith(BlockJUnit4ClassRunner::class)
15 | class CommitTest {
16 |
17 |
18 | @Test
19 | fun logIsCommited() {
20 | val log = Log().apply {
21 | append(LogEntry.newBuilder().setTerm(0).build())
22 | append(LogEntry.newBuilder().setTerm(0).build())
23 | append(LogEntry.newBuilder().setTerm(0).build())
24 |
25 | }
26 | val state = State(id = 1, log = log, startState = NodeState.LEADER)
27 | val stateTwo = State(id = 1, log = log, startState = NodeState.FOLLOWER)
28 | val stateThree = State(id = 1, log = log, startState = NodeState.FOLLOWER)
29 |
30 | val commit = CommitAction(state,
31 | listOf(stateTwo.testNode().also { it.matchIndex = state.log.lastIndex() },
32 | stateThree.testNode().also { it.matchIndex = state.log.lastIndex() }))
33 | assertThat(state.log.commitIndex).isEqualTo(-1)
34 | commit.perform()
35 |
36 | assertThat(state.log.commitIndex).isEqualTo(2)
37 | }
38 |
39 | @Test
40 | fun logCannotBeCommitedIfMajorityLogNotMatched() {
41 | val log = Log().apply {
42 | append(LogEntry.newBuilder().setTerm(0).build())
43 | append(LogEntry.newBuilder().setTerm(0).build())
44 | append(LogEntry.newBuilder().setTerm(0).build())
45 |
46 | }
47 | val state = State(id = 1, log = log, startState = NodeState.LEADER)
48 | val stateTwo = State(id = 1, log = Log(), startState = NodeState.FOLLOWER)
49 | val stateThree = State(id = 1, log = Log(), startState = NodeState.FOLLOWER)
50 |
51 | val commit = CommitAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
52 | assertThat(state.log.commitIndex).isEqualTo(-1)
53 | commit.perform()
54 |
55 | assertThat(state.log.commitIndex).isEqualTo(-1)
56 | }
57 |
58 | @Test
59 | fun logCannotBeCommitedIfTermNotMatched() {
60 | val log = Log().apply {
61 | append(LogEntry.newBuilder().setTerm(0).build())
62 | append(LogEntry.newBuilder().setTerm(0).build())
63 | append(LogEntry.newBuilder().setTerm(0).build())
64 |
65 | }
66 | val state = State(id = 1, log = log, startState = NodeState.LEADER, term = 1)
67 | val stateTwo = State(id = 1, log = Log(), startState = NodeState.FOLLOWER, term = 1)
68 | val stateThree = State(id = 1, log = Log(), startState = NodeState.FOLLOWER, term = 1)
69 |
70 | val commit = CommitAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
71 | assertThat(state.log.commitIndex).isEqualTo(-1)
72 | commit.perform()
73 |
74 | assertThat(state.log.commitIndex).isEqualTo(-1)
75 | }
76 |
77 |
78 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/RaftControllerTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft
2 |
3 | import edu.ucu.proto.VoteRequest
4 | import edu.ucu.raft.clock.TermClock
5 | import edu.ucu.raft.state.NodeState
6 | import kotlinx.coroutines.channels.ConflatedBroadcastChannel
7 | import kotlinx.coroutines.delay
8 | import kotlinx.coroutines.runBlocking
9 | import org.assertj.core.api.Assertions.assertThat
10 | import org.junit.Test
11 | import org.junit.runner.RunWith
12 | import org.junit.runners.BlockJUnit4ClassRunner
13 | import org.mockito.BDDMockito.*
14 | import org.mockito.Mockito.mock
15 |
16 | @RunWith(BlockJUnit4ClassRunner::class)
17 | class RaftControllerTest {
18 |
19 |
20 | val channelMock = ConflatedBroadcastChannel()
21 |
22 |
23 | val clock = mock(TermClock::class.java).also {
24 | given(it.channel).willReturn(channelMock)
25 | }
26 | val consensus = RaftController(StaticConfiguration(id = 5), mutableListOf())
27 |
28 |
29 | @Test
30 | fun controllerIsWorkingConsistently() = test {
31 | val raft = RaftController(StaticConfiguration(id = 1, heartbeatInterval = 50, timerInterval = 400))
32 | assertThat(raft.state.term).isEqualTo(0)
33 | delay(1000)
34 | assertThat(raft.state.term).isEqualTo(0)
35 | raft.start()
36 | delay(1000)
37 | raft.stop()
38 | assertThat(raft.state.term).isEqualTo(2)
39 | delay(1000)
40 | assertThat(raft.state.term).isEqualTo(2)
41 | }
42 |
43 |
44 | @Test
45 | fun nodeCanVoteForCandidate() {
46 | assertThat(consensus.state.current).isEqualTo(NodeState.FOLLOWER)
47 | val request = VoteRequest.newBuilder().setCandidateId(10).setTerm(consensus.state.term).build()
48 |
49 | val result = runBlocking { consensus.requestVote(request) }
50 | assertThat(result.voteGranted).isEqualTo(true)
51 | assertThat(consensus.state.votedFor).isEqualTo(10L)
52 | }
53 |
54 | @Test
55 | fun voteForLowerTermIsNotGranted() {
56 | assertThat(consensus.state.votedFor).isEqualTo(null)
57 | assertThat(consensus.state.current).isEqualTo(NodeState.FOLLOWER)
58 | val request = VoteRequest.newBuilder().setCandidateId(10).setTerm(consensus.state.term - 1).build()
59 | val result = runBlocking { consensus.requestVote(request) }
60 | assertThat(result.voteGranted).isEqualTo(false)
61 | assertThat(consensus.state.votedFor).isEqualTo(null)
62 | }
63 |
64 | @Test
65 | fun onlyOneVotePerTermIsGranted() {
66 | assertThat(consensus.state.votedFor).isEqualTo(null)
67 | assertThat(consensus.state.current).isEqualTo(NodeState.FOLLOWER)
68 |
69 | val request = VoteRequest.newBuilder().setCandidateId(10).setTerm(consensus.state.term).build()
70 |
71 | val result = runBlocking { consensus.requestVote(request) }
72 |
73 | assertThat(result.voteGranted).isEqualTo(true)
74 | assertThat(consensus.state.votedFor).isEqualTo(10L)
75 |
76 | val requestTwo = VoteRequest.newBuilder().setCandidateId(1).setTerm(consensus.state.term).build()
77 |
78 | val resultTwo = runBlocking { consensus.requestVote(requestTwo) }
79 |
80 | assertThat(resultTwo.voteGranted).isEqualTo(false)
81 | assertThat(consensus.state.votedFor).isEqualTo(10)
82 | }
83 |
84 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/actions/HeartbeatActionTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.actions
2 |
3 | import edu.ucu.raft.log.Log
4 | import edu.ucu.proto.LogEntry
5 | import edu.ucu.raft.state.NodeState
6 | import edu.ucu.raft.state.State
7 | import edu.ucu.raft.test
8 | import edu.ucu.raft.testNode
9 | import org.assertj.core.api.Assertions.assertThat
10 | import org.junit.Test
11 | import org.junit.runner.RunWith
12 | import org.junit.runners.BlockJUnit4ClassRunner
13 |
14 | @RunWith(BlockJUnit4ClassRunner::class)
15 | class HeartbeatActionTest {
16 |
17 | @Test
18 | fun heartbeatFromLeaderIsAccepted() = test {
19 | val state = State(1, startState = NodeState.LEADER)
20 | val stateTwo = State(2, startState = NodeState.FOLLOWER)
21 | val stateThree = State(3, startState = NodeState.FOLLOWER)
22 |
23 |
24 | assertThat(stateTwo.leaderId).isNull()
25 | assertThat(stateThree.leaderId).isNull()
26 |
27 | val heartbeat = HeartbeatAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
28 | heartbeat.send()
29 |
30 | assertThat(stateTwo.leaderId).isEqualTo(1)
31 | assertThat(stateThree.leaderId).isEqualTo(1)
32 |
33 | }
34 |
35 | @Test
36 | fun logCanBeReplicatedToEmptyFollowers() = test {
37 | val log = Log().apply {
38 | append(LogEntry.newBuilder().build())
39 | append(LogEntry.newBuilder().build())
40 | append(LogEntry.newBuilder().build())
41 | commit(2)
42 | }
43 |
44 | val state = State(1, startState = NodeState.LEADER, log = log)
45 | val stateTwo = State(2, startState = NodeState.FOLLOWER)
46 | val stateThree = State(3, startState = NodeState.FOLLOWER)
47 |
48 | val heartbeat = HeartbeatAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
49 | heartbeat.send()
50 | assertThat(stateTwo.log.lastIndex()).isEqualTo(log.lastIndex())
51 |
52 | }
53 |
54 | @Test
55 | fun invalidLogOverwritten() = test {
56 |
57 | val state = State(1, startState = NodeState.LEADER, log = Log().apply {
58 | append(LogEntry.newBuilder().setTerm(1).build())
59 | append(LogEntry.newBuilder().setTerm(1).build())
60 | append(LogEntry.newBuilder().setTerm(2).build())
61 | commit(2)
62 | }
63 | )
64 | val stateTwo = State(2, startState = NodeState.FOLLOWER, log = Log().apply {
65 | append(LogEntry.newBuilder().setTerm(1).build())
66 | append(LogEntry.newBuilder().setTerm(1).build())
67 | append(LogEntry.newBuilder().setTerm(1).build())
68 | commit(2)
69 | }
70 | )
71 | val stateThree = State(3, startState = NodeState.FOLLOWER, log = Log().apply {
72 | append(LogEntry.newBuilder().setTerm(1).build())
73 | append(LogEntry.newBuilder().setTerm(1).build())
74 | append(LogEntry.newBuilder().setTerm(1).build())
75 | commit(2)
76 | })
77 |
78 | assertThat(stateTwo.log.lastTerm()).isEqualTo(1)
79 |
80 | val heartbeat = HeartbeatAction(state, listOf(stateTwo.testNode(), stateThree.testNode()))
81 | heartbeat.send()
82 | heartbeat.send()
83 | assertThat(stateTwo.log.lastTerm()).isEqualTo(2)
84 |
85 | }
86 |
87 |
88 |
89 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/state/VoteRequestTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.state
2 |
3 | import edu.ucu.proto.LogEntry
4 | import edu.ucu.proto.VoteRequest
5 | import edu.ucu.raft.test
6 | import org.assertj.core.api.Assertions.*
7 | import org.junit.Test
8 | import org.junit.runner.RunWith
9 | import org.junit.runners.BlockJUnit4ClassRunner
10 |
11 |
12 | @RunWith(BlockJUnit4ClassRunner::class)
13 | class VoteRequestTest {
14 |
15 |
16 | val state = State(5)
17 |
18 | @Test
19 | fun voteForCandidateWithSmallerTermIsNotGranted() = test {
20 | val request = VoteRequest.newBuilder()
21 | .setCandidateId(1).setTerm(-1).setLastLogIndex(-1).setLastLogTerm(-1)
22 | .build()
23 | val result = state.requestVote(request)
24 |
25 | assertThat(result?.voteGranted).isFalse()
26 | assertThat(state.votedFor).isNull()
27 | }
28 |
29 |
30 | @Test
31 | fun voteInSingleTermIsNotChanging() = test {
32 | val request = VoteRequest.newBuilder()
33 | .setCandidateId(1).setTerm(1).setLastLogIndex(-1).setLastLogTerm(-1)
34 | .build()
35 |
36 | val vote = state.requestVote(request)
37 | assertThat(vote?.voteGranted).isTrue()
38 | val requestTwo = VoteRequest.newBuilder()
39 | .setCandidateId(2).setTerm(1).setLastLogIndex(-1).setLastLogTerm(-1)
40 | .build()
41 |
42 | val voteTwo = state.requestVote(requestTwo)
43 | assertThat(voteTwo?.voteGranted).isFalse()
44 | }
45 |
46 | @Test
47 | fun voteForHigherTermIsGrantedEvenIfVotedBefore() = test {
48 | val vote = state.requestVote(VoteRequest.newBuilder()
49 | .setCandidateId(1).setTerm(1).setLastLogIndex(-1).setLastLogTerm(-1)
50 | .build())
51 | assertThat(vote?.voteGranted).isTrue()
52 | val secondVote = state.requestVote(VoteRequest.newBuilder()
53 | .setCandidateId(2).setTerm(2).setLastLogIndex(-1).setLastLogTerm(-1)
54 | .build())
55 | assertThat(secondVote?.voteGranted).isTrue()
56 |
57 | assertThat(state.term).isEqualTo(2)
58 | assertThat(state.votedFor).isEqualTo(2)
59 |
60 | }
61 |
62 | @Test
63 | fun voteForCandidateWithBiggerTermAndShorterLogIsGranted() = test {
64 | state.log[0] = LogEntry.newBuilder().setTerm(0).build()
65 | state.log[1] = LogEntry.newBuilder().setTerm(0).build()
66 | state.log[2] = LogEntry.newBuilder().setTerm(0).build()
67 |
68 | val vote = state.requestVote(VoteRequest.newBuilder()
69 | .setCandidateId(1).setTerm(1).setLastLogIndex(1).setLastLogTerm(1)
70 | .build())
71 |
72 | assertThat(vote?.voteGranted).isTrue()
73 | assertThat(state.votedFor).isEqualTo(1)
74 | }
75 |
76 | @Test
77 | fun voteForCandidateWithSmallerTermNotGranted() = test {
78 | state.log[0] = LogEntry.newBuilder().setTerm(0).build()
79 | state.log[1] = LogEntry.newBuilder().setTerm(0).build()
80 | state.log[2] = LogEntry.newBuilder().setTerm(1).build()
81 |
82 | val vote = state.requestVote(VoteRequest.newBuilder()
83 | .setCandidateId(1).setTerm(1).setLastLogIndex(2).setLastLogTerm(0)
84 | .build())
85 |
86 | assertThat(vote?.voteGranted).isFalse()
87 | assertThat(state.votedFor).isNull()
88 | }
89 |
90 |
91 | }
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # raft-kotlin
2 |
3 | [](https://travis-ci.com/AChepurnoi/raft-kotlin)
4 | [](https://codecov.io/gh/AChepurnoi/raft-kotlin)
5 |
6 |
7 | Kotlin implementation of the raft consensus algorithm
8 |
9 | A raft is a consensus algorithm that is designed to be easy to understand. It's equivalent to Paxos in fault-tolerance and performance
10 |
11 | **Repository provides an example implementation and
12 | show-case usage with in-memory key-value storage**
13 |
14 | ## 👨💻 How to run (key-value example)
15 | #### Building
16 | ```
17 | #Cloning repository
18 | git clone https://github.com/AChepurnoi/raft-kotlin.git
19 |
20 | #Building jar file
21 | ./gradlew jar
22 | ```
23 |
24 |
25 | #### Running docker environment
26 | ```
27 | docker-compose up --build
28 | ```
29 |
30 | To interact with cluster you can use cURL:
31 |
32 | ```
33 | #Set test=hello
34 | docker run --rm --net=raft-kt_default hortonworks/alpine-curl:3.1 curl --request POST --url node_one:8000/test --data 'hello' ; echo
35 | ```
36 |
37 |
38 | ```
39 | #Read test value
40 | docker run --rm --net=raft-kt_default hortonworks/alpine-curl:3.1 curl --request GET --url node_one:8000/test ; echo
41 | ```
42 |
43 | #### Other
44 |
45 | The key-value implementation uses `307 Redirect` to redirect requests from slaves to the master.
46 |
47 | This requires you to be able to resolve the IP from configuration (You should interact with HTTP server only from docker network e.g. you container)
48 |
49 | Another option is to run jar files locally with proper env configuration
50 |
51 | To read node list from env, raft env configuration uses the following notation:
52 | ```
53 | NODES=[ID]:[HOST]:[PORT],[ID]:[HOST]:[PORT]...
54 | ```
55 |
56 |
57 | ## 🔑 Example App (key-value storage) Implementation
58 | Shows the example of how raft module can be used to
59 | implement distributed in-memory key-value storage.
60 |
61 | Current implementation exposes two endpoints:
62 | ```
63 | # Set `key={request_body}
64 | POST HOST/{key}
65 |
66 | #Returns the value of the `key` or `Nil` if the key does not exist
67 | GET HOST/{key}
68 | ```
69 |
70 |
71 | Key-value HTTP server uses `8000` port by default
72 |
73 | ## 🔨 Raft Implementation
74 |
75 | Exposes `RaftNode` class for clients to create a cluster node,
76 | actions to mutate the state of the cluster
77 | and method to fetch the current state.
78 |
79 |
80 | Components:
81 | * State
82 | * Log
83 | * gRPC Client/Server
84 | * Clock
85 | * Actions
86 | * Raft Controller
87 |
88 | ## ⚙️ Testing
89 | * Unit tests - `actions`, `clock`, `log`, `state` and `RaftController` classes are tested
90 | * Integration tests - `RaftClusterTesting` class contains different test cases for living cluster (With `LocalRaftNode` instead of `GrpcClusterNode`)
91 | * Key-Value container testing - kv cluster testing - `Not implemented yet`
92 |
93 |
94 | ## ⛳️ Points to improve
95 | *This is not production ready implementation and very likely there are bugs*
96 |
97 | * Refactoring
98 | * Revisit `@Volatile` and Mutex usages
99 | * Implement persistent log storage
100 | * Implement snapshotting
101 |
102 | ## 🔗 References
103 |
104 | [Instructors-guide-to-raft](https://thesquareplanet.com/blog/instructors-guide-to-raft/)
105 |
106 | [Students-guide-to-raft](https://thesquareplanet.com/blog/students-guide-to-raft/)
107 |
108 | [Raft visualization](http://thesecretlivesofdata.com/raft/)
109 |
110 | [Raft resources](https://raft.github.io/)
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/state/State.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.state
2 |
3 | import edu.ucu.raft.log.Command
4 | import edu.ucu.raft.log.Log
5 | import edu.ucu.proto.*
6 | import kotlinx.coroutines.channels.ConflatedBroadcastChannel
7 | import mu.KotlinLogging
8 |
9 | class State(val id: Int,
10 | val log: Log = Log(),
11 | startState: NodeState = NodeState.FOLLOWER,
12 | term: Long = 0) {
13 |
14 | private val logger = KotlinLogging.logger {}
15 | val updates = ConflatedBroadcastChannel>()
16 |
17 | var term: Long = term
18 | internal set
19 |
20 | @Volatile
21 | var current: NodeState = startState
22 | private set
23 |
24 | @Volatile
25 | var votedFor: Int? = null
26 |
27 | @Volatile
28 | var leaderId: Int? = null
29 |
30 | private suspend fun updateState(next: NodeState) {
31 | if (next == current) return
32 | logger.info { "🔴 State $current -> $next" }
33 | val prev = current
34 | current = next
35 | updates.send(prev to current)
36 | }
37 |
38 | suspend fun nextTerm(newTerm: Long) {
39 | this.term = newTerm
40 | this.votedFor = id
41 | updateState(NodeState.CANDIDATE)
42 | }
43 |
44 | suspend fun promoteToLeader() {
45 | votedFor = null
46 | leaderId = id
47 | updateState(NodeState.LEADER)
48 | }
49 |
50 | suspend fun requestVote(request: VoteRequest): VoteResponse {
51 |
52 | if (request.term > this.term) {
53 | this.term = term
54 | updateState(NodeState.FOLLOWER)
55 | }
56 |
57 | val logValid = validateLog(request)
58 |
59 | val voteResult = when {
60 | this.term > request.term || !logValid -> false
61 |
62 | this.term < request.term -> {
63 | this.term = request.term
64 | this.votedFor = request.candidateId
65 | updateState(NodeState.FOLLOWER)
66 | true
67 | }
68 | this.term == request.term -> {
69 | if (this.votedFor == null && current != NodeState.LEADER) {
70 | this.votedFor = request.candidateId
71 | }
72 | votedFor == request.candidateId
73 | }
74 | else -> {
75 | logger.warn { "🤬 Voting failed: $request" }
76 | false
77 | }
78 | }
79 |
80 | return VoteResponse.newBuilder().setTerm(term).setVoteGranted(voteResult).build()
81 | }
82 |
83 | private fun validateLog(request: VoteRequest): Boolean {
84 |
85 | val leaderLogLastTermIsBigger = log.lastTerm()?.let { it < request.lastLogTerm } ?: false
86 | if (leaderLogLastTermIsBigger) return true
87 |
88 |
89 | val leaderLogLastTermEqual = log.lastTerm()?.let { it == request.lastLogTerm } ?: true
90 |
91 | val leaderLogLongerOrEqual = log.lastIndex() <= request.lastLogIndex
92 |
93 | return leaderLogLastTermEqual && leaderLogLongerOrEqual
94 | }
95 |
96 | suspend fun appendEntries(request: AppendRequest): AppendResponse {
97 | if (request.term < this.term) return AppendResponse.newBuilder().setTerm(term).setSuccess(false).build()
98 | if (request.term >= this.term) {
99 | this.term = request.term
100 | this.votedFor = null
101 | this.leaderId = request.leaderId
102 | updateState(NodeState.FOLLOWER)
103 | }
104 |
105 | if (request.prevLogIndex != -1) {
106 | val logValid = log[request.prevLogIndex]?.run { this.term == request.prevLogTerm } ?: false
107 | if (!logValid) {
108 | logger.warn { "Master log is not cool" }
109 | return AppendResponse.newBuilder().setTerm(term).setSuccess(false).build()
110 | }
111 | }
112 |
113 | var idx = request.prevLogIndex + 1
114 | for (entry in request.entriesList) {
115 |
116 | if (log.isNotEmpty()) {
117 | val logConflict = log[idx]?.run { this.term != entry.term } ?: false
118 | if (logConflict) {
119 | log.prune(idx)
120 | }
121 | }
122 | log[idx] = entry
123 | idx += 1
124 | }
125 |
126 | this.leaderId = request.leaderId
127 | log.commit(request.leaderCommit)
128 | return AppendResponse.newBuilder().setTerm(term).setSuccess(true).build()
129 |
130 | }
131 |
132 | override fun toString(): String {
133 | return "State[id=$id, term=$term, current=$current, leaderId=$leaderId, votedFor=$votedFor, log=$log]"
134 | }
135 |
136 | fun applyCommand(command: Command): Int {
137 | return log.append(LogEntry.newBuilder().setTerm(term).setCommandBytes(command.toBytes()).build())
138 | }
139 |
140 | }
141 |
142 |
--------------------------------------------------------------------------------
/raft/src/main/kotlin/edu/ucu/raft/RaftController.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft
2 |
3 | import edu.ucu.raft.log.Command
4 | import edu.ucu.proto.AppendRequest
5 | import edu.ucu.proto.AppendResponse
6 | import edu.ucu.proto.VoteRequest
7 | import edu.ucu.proto.VoteResponse
8 | import edu.ucu.raft.actions.CommitAction
9 | import edu.ucu.raft.actions.HeartbeatAction
10 | import edu.ucu.raft.actions.VotingAction
11 | import edu.ucu.raft.adapters.ClusterNode
12 | import edu.ucu.raft.adapters.RaftHandler
13 | import edu.ucu.raft.clock.TermClock
14 | import edu.ucu.raft.state.*
15 | import kotlinx.coroutines.*
16 | import kotlinx.coroutines.sync.Mutex
17 | import kotlinx.coroutines.sync.withLock
18 | import mu.KotlinLogging
19 | import java.util.*
20 | import kotlin.concurrent.fixedRateTimer
21 | import kotlin.math.log
22 |
23 |
24 | class RaftController(val config: RaftConfiguration,
25 | val cluster: MutableList = mutableListOf()) : RaftHandler {
26 |
27 | val state = State(id = config.id)
28 |
29 | private val clock = TermClock(config.timerInterval)
30 | private val logger = KotlinLogging.logger {}
31 | private val heartbeat = HeartbeatAction(state, cluster)
32 | private val voting = VotingAction(state, cluster)
33 | private val commit = CommitAction(state, cluster)
34 | private val clockSubscription = clock.channel.openSubscription()
35 | // private val stateLock: Mutex = Mutex(false)
36 | private lateinit var termSubscriber: Job
37 | private lateinit var heartbeatTimer: Timer
38 | private lateinit var stateSubscriber: Job
39 |
40 |
41 | private val stateLogger = fixedRateTimer("logger", initialDelay = 1000, period = 1000) {
42 | logger.info { "⛳️ $state" }
43 | if (state.current == NodeState.LEADER) {
44 | cluster.forEach {
45 | logger.info { "⚙️ Node: ${it.nodeId} - Next: ${it.nextIndex} Match: ${it.matchIndex}" }
46 | }
47 | }
48 | }
49 |
50 | private fun prepareStateSubscriber() {
51 | stateSubscriber = GlobalScope.launch {
52 | for ((prev, current) in state.updates.openSubscription()) {
53 | when {
54 | current == NodeState.LEADER -> {
55 | cluster.forEach { it.reinitializeIndex(state.log.lastIndex() + 1) }
56 | clock.freeze()
57 | }
58 | prev == NodeState.LEADER && current != NodeState.LEADER -> {
59 | clock.start()
60 | }
61 | }
62 | }
63 | }
64 | }
65 |
66 | private fun prepareHeartbeatTimer() {
67 | heartbeatTimer = fixedRateTimer("heartbeat", initialDelay = config.heartbeatInterval, period = config.heartbeatInterval) {
68 | runBlocking {
69 | kotlin.runCatching {
70 | // stateLock.withLock {
71 | heartbeat.send()
72 | commit.perform()
73 | // }
74 | }.onFailure {
75 | logger.error { "Failure: $it" }
76 | }
77 | }
78 | }
79 | }
80 |
81 | private fun prepareTermIncrementSubscriber() {
82 | termSubscriber = GlobalScope.launch {
83 | for (term in clockSubscription) {
84 | logger.info { "Starting term increment" }
85 | state.nextTerm(term)
86 | val result = voting.askVotes()
87 | if (result) {
88 | state.promoteToLeader()
89 | } else {
90 | logger.info { "---> 🤬 Can't promote to leader <---" }
91 | }
92 | }
93 |
94 | }
95 | }
96 |
97 | private suspend fun actualizeTerm(receivedTerm: Long) {
98 | if (clock.term < receivedTerm) {
99 | clock.update(receivedTerm)
100 | }
101 | }
102 |
103 | fun start() {
104 | runBlocking {
105 | clock.start()
106 | prepareTermIncrementSubscriber()
107 | prepareHeartbeatTimer()
108 | prepareStateSubscriber()
109 | }
110 | }
111 |
112 | fun stop() {
113 | runBlocking {
114 | clock.freeze()
115 | termSubscriber.cancelAndJoin()
116 | heartbeatTimer.cancel()
117 | stateSubscriber.cancelAndJoin()
118 | }
119 | }
120 |
121 | override suspend fun requestVote(request: VoteRequest): VoteResponse {
122 | actualizeTerm(request.term)
123 | val vote = state.requestVote(request)
124 |
125 | if (vote.voteGranted) {
126 | clock.reset()
127 | }
128 | logger.info { "🗽Vote request: ${request.candidateId} - term ${request.term} - result: ${vote.voteGranted}" }
129 | return vote
130 |
131 | }
132 |
133 | override suspend fun appendEntries(request: AppendRequest): AppendResponse {
134 | actualizeTerm(request.term)
135 | val result = state.appendEntries(request)
136 |
137 | if (result.success) {
138 | clock.reset()
139 | }
140 | // logger.info { "💎 Validated leader message. Result ${result.success}" }
141 | return result
142 | }
143 |
144 |
145 | suspend fun applyCommand(command: Command): Boolean {
146 | val index = state.applyCommand(command)
147 |
148 | while (index > state.log.commitIndex) {
149 | delay(50)
150 | }
151 | return true
152 | }
153 |
154 |
155 | }
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/RaftClusterTesting.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft
2 |
3 | import edu.ucu.raft.log.Set
4 | import edu.ucu.raft.state.NodeState
5 | import kotlinx.coroutines.delay
6 | import kotlinx.coroutines.withTimeoutOrNull
7 | import org.assertj.core.api.Assertions.assertThat
8 | import org.junit.Test
9 | import org.junit.runner.RunWith
10 | import org.junit.runners.BlockJUnit4ClassRunner
11 |
12 | @RunWith(BlockJUnit4ClassRunner::class)
13 | class RaftClusterTesting {
14 |
15 | private fun clusterOfThree(): List {
16 | val nodeOne = LocalRaftNode(RaftController(StaticConfiguration(1, timerInterval = 500, heartbeatInterval = 50)))
17 | val nodeTwo = LocalRaftNode(RaftController(StaticConfiguration(2, timerInterval = 550, heartbeatInterval = 50)))
18 | val nodeThree = LocalRaftNode(RaftController(StaticConfiguration(3, timerInterval = 650, heartbeatInterval = 50)))
19 |
20 | nodeOne.controller.cluster.addAll(mutableListOf(nodeTwo, nodeThree))
21 | nodeTwo.controller.cluster.addAll(mutableListOf(nodeOne, nodeThree))
22 | nodeThree.controller.cluster.addAll(mutableListOf(nodeTwo, nodeOne))
23 |
24 | return listOf(nodeOne, nodeTwo, nodeThree)
25 | }
26 |
27 | @Test(timeout = 30000)
28 | fun clusterIsStableAndLeaderIsElected() = test {
29 |
30 | val nodes = clusterOfThree()
31 | nodes.forEach { it.activate() }
32 | delay(3000)
33 |
34 | val followers = nodes.filter { it.controller.state.current == NodeState.FOLLOWER }.count()
35 | val leaders = nodes.filter { it.controller.state.current == NodeState.LEADER }.count()
36 |
37 | assertThat(followers).isEqualTo(2)
38 | assertThat(leaders).isEqualTo(1)
39 | }
40 |
41 |
42 | @Test(timeout = 30000)
43 | fun nodeCanBeIsolatedFromClusterAndCatchUpAfter() = test {
44 | val nodes = clusterOfThree()
45 | nodes.forEach { it.activate() }
46 |
47 | delay(3000)
48 | val leader = nodes.find { it.controller.state.current == NodeState.LEADER }!!
49 | val oldTerm = leader.controller.state.term
50 |
51 | leader.isolate()
52 | delay(3000)
53 | leader.activate()
54 | delay(3000)
55 | val newTerm = leader.controller.state.term
56 | assertThat(newTerm).isGreaterThan(oldTerm)
57 | assertThat(leader.controller.state.current).isEqualTo(NodeState.FOLLOWER)
58 | }
59 |
60 | @Test(timeout = 30000)
61 | fun logReplicationIsWorkingCorrectly() = test {
62 | val nodes = clusterOfThree()
63 | nodes.forEach { it.activate() }
64 |
65 | delay(3000)
66 | val leader = nodes.find { it.controller.state.current == NodeState.LEADER }!!
67 | leader.controller.applyCommand(Set("one", "1".toByteArray()))
68 | leader.controller.applyCommand(Set("two", "2".toByteArray()))
69 | leader.controller.applyCommand(Set("three", "3".toByteArray()))
70 |
71 | delay(1000)
72 |
73 | val logs = nodes.map { it.controller.state.log.state() }
74 |
75 | assertThat(logs.all { String(it["one"]!!) == "1" })
76 | assertThat(logs.all { String(it["two"]!!) == "2" })
77 | assertThat(logs.all { String(it["three"]!!) == "3" })
78 | }
79 |
80 | @Test(timeout = 30000)
81 | fun separatingNodeFromClusterAndPuttingBackSynchronizesLogAndLeftUncommitedStaleLeader() = test {
82 | val nodes = clusterOfThree()
83 | nodes.forEach { it.activate() }
84 | delay(3000)
85 | val staleLeader = kotlin.run {
86 | val leader = nodes.find { it.controller.state.current == NodeState.LEADER }!!
87 | leader.controller.applyCommand(Set("one", "1".toByteArray()))
88 | val log = leader.controller.state.log.state()
89 | assertThat(String(log["one"]!!)).isEqualTo("1")
90 | delay(1000)
91 | leader.isolate()
92 | leader
93 | }
94 |
95 |
96 | val newLeader = kotlin.run {
97 | val staleLeaderLog = staleLeader.controller.state.log.state()
98 | val newLeader = nodes.find { it.controller.state.current == NodeState.LEADER }!!
99 | withTimeoutOrNull(500) { newLeader.controller.applyCommand(Set("two", "2".toByteArray())) }
100 | withTimeoutOrNull(500) { newLeader.controller.applyCommand(Set("three", "3".toByteArray())) }
101 | val log = newLeader.controller.state.log
102 | assertThat(log.lastIndex()).isEqualTo(2)
103 | assertThat(log.commitIndex).isEqualTo(0)
104 | delay(1000)
105 | assertThat(staleLeaderLog["two"]).isNull()
106 | assertThat(staleLeaderLog["three"]).isNull()
107 | newLeader
108 | }
109 | staleLeader.activate()
110 | delay(3000)
111 |
112 | val staleLeaderLog = staleLeader.controller.state.log.state()
113 | assertThat(String(staleLeaderLog["one"]!!)).isEqualTo("1")
114 | assertThat(staleLeaderLog["two"]).isNull()
115 | assertThat(staleLeaderLog["three"]).isNull()
116 |
117 | val leader = nodes.find { it.controller.state.current == NodeState.LEADER }!!
118 | leader.controller.applyCommand(Set("two", "2".toByteArray()))
119 | leader.controller.applyCommand(Set("three", "3".toByteArray()))
120 | val leaderLog = leader.controller.state.log.state()
121 | assertThat(String(leaderLog["two"]!!)).isEqualTo("2")
122 | assertThat(String(leaderLog["three"]!!)).isEqualTo("3")
123 |
124 | }
125 |
126 |
127 |
128 |
129 |
130 | }
--------------------------------------------------------------------------------
/gradlew:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env sh
2 |
3 | ##############################################################################
4 | ##
5 | ## Gradle start up script for UN*X
6 | ##
7 | ##############################################################################
8 |
9 | # Attempt to set APP_HOME
10 | # Resolve links: $0 may be a link
11 | PRG="$0"
12 | # Need this for relative symlinks.
13 | while [ -h "$PRG" ] ; do
14 | ls=`ls -ld "$PRG"`
15 | link=`expr "$ls" : '.*-> \(.*\)$'`
16 | if expr "$link" : '/.*' > /dev/null; then
17 | PRG="$link"
18 | else
19 | PRG=`dirname "$PRG"`"/$link"
20 | fi
21 | done
22 | SAVED="`pwd`"
23 | cd "`dirname \"$PRG\"`/" >/dev/null
24 | APP_HOME="`pwd -P`"
25 | cd "$SAVED" >/dev/null
26 |
27 | APP_NAME="Gradle"
28 | APP_BASE_NAME=`basename "$0"`
29 |
30 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
31 | DEFAULT_JVM_OPTS=""
32 |
33 | # Use the maximum available, or set MAX_FD != -1 to use that value.
34 | MAX_FD="maximum"
35 |
36 | warn () {
37 | echo "$*"
38 | }
39 |
40 | die () {
41 | echo
42 | echo "$*"
43 | echo
44 | exit 1
45 | }
46 |
47 | # OS specific support (must be 'true' or 'false').
48 | cygwin=false
49 | msys=false
50 | darwin=false
51 | nonstop=false
52 | case "`uname`" in
53 | CYGWIN* )
54 | cygwin=true
55 | ;;
56 | Darwin* )
57 | darwin=true
58 | ;;
59 | MINGW* )
60 | msys=true
61 | ;;
62 | NONSTOP* )
63 | nonstop=true
64 | ;;
65 | esac
66 |
67 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
68 |
69 | # Determine the Java command to use to start the JVM.
70 | if [ -n "$JAVA_HOME" ] ; then
71 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
72 | # IBM's JDK on AIX uses strange locations for the executables
73 | JAVACMD="$JAVA_HOME/jre/sh/java"
74 | else
75 | JAVACMD="$JAVA_HOME/bin/java"
76 | fi
77 | if [ ! -x "$JAVACMD" ] ; then
78 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME
79 |
80 | Please set the JAVA_HOME variable in your environment to match the
81 | location of your Java installation."
82 | fi
83 | else
84 | JAVACMD="java"
85 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
86 |
87 | Please set the JAVA_HOME variable in your environment to match the
88 | location of your Java installation."
89 | fi
90 |
91 | # Increase the maximum file descriptors if we can.
92 | if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then
93 | MAX_FD_LIMIT=`ulimit -H -n`
94 | if [ $? -eq 0 ] ; then
95 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then
96 | MAX_FD="$MAX_FD_LIMIT"
97 | fi
98 | ulimit -n $MAX_FD
99 | if [ $? -ne 0 ] ; then
100 | warn "Could not set maximum file descriptor limit: $MAX_FD"
101 | fi
102 | else
103 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT"
104 | fi
105 | fi
106 |
107 | # For Darwin, add options to specify how the application appears in the dock
108 | if $darwin; then
109 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\""
110 | fi
111 |
112 | # For Cygwin, switch paths to Windows format before running java
113 | if $cygwin ; then
114 | APP_HOME=`cygpath --path --mixed "$APP_HOME"`
115 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"`
116 | JAVACMD=`cygpath --unix "$JAVACMD"`
117 |
118 | # We build the pattern for arguments to be converted via cygpath
119 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null`
120 | SEP=""
121 | for dir in $ROOTDIRSRAW ; do
122 | ROOTDIRS="$ROOTDIRS$SEP$dir"
123 | SEP="|"
124 | done
125 | OURCYGPATTERN="(^($ROOTDIRS))"
126 | # Add a user-defined pattern to the cygpath arguments
127 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then
128 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)"
129 | fi
130 | # Now convert the arguments - kludge to limit ourselves to /bin/sh
131 | i=0
132 | for arg in "$@" ; do
133 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -`
134 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option
135 |
136 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition
137 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"`
138 | else
139 | eval `echo args$i`="\"$arg\""
140 | fi
141 | i=$((i+1))
142 | done
143 | case $i in
144 | (0) set -- ;;
145 | (1) set -- "$args0" ;;
146 | (2) set -- "$args0" "$args1" ;;
147 | (3) set -- "$args0" "$args1" "$args2" ;;
148 | (4) set -- "$args0" "$args1" "$args2" "$args3" ;;
149 | (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;;
150 | (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;;
151 | (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;;
152 | (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;;
153 | (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;;
154 | esac
155 | fi
156 |
157 | # Escape application args
158 | save () {
159 | for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done
160 | echo " "
161 | }
162 | APP_ARGS=$(save "$@")
163 |
164 | # Collect all arguments for the java command, following the shell quoting and substitution rules
165 | eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS"
166 |
167 | # by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong
168 | if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then
169 | cd "$(dirname "$0")"
170 | fi
171 |
172 | exec "$JAVACMD" "$@"
173 |
--------------------------------------------------------------------------------
/raft/src/test/kotlin/edu/ucu/raft/state/AppendRequestTest.kt:
--------------------------------------------------------------------------------
1 | package edu.ucu.raft.state
2 |
3 | import edu.ucu.proto.AppendRequest
4 | import edu.ucu.proto.LogEntry
5 | import edu.ucu.raft.test
6 | import org.assertj.core.api.Assertions.*
7 | import org.junit.Test
8 | import org.junit.runner.RunWith
9 | import org.junit.runners.BlockJUnit4ClassRunner
10 |
11 | @RunWith(BlockJUnit4ClassRunner::class)
12 | class AppendRequestTest {
13 |
14 | val state = State(5)
15 |
16 | @Test
17 | fun logEntryCouldBeAppendedToEmptyLog() = test {
18 | val request = buildAppendRequest()
19 | assertThat(state.log.lastIndex()).isEqualTo(-1)
20 |
21 | val result = state.appendEntries(request)
22 | assertThat(state.log.lastIndex()).isEqualTo(0)
23 | assertThat(result?.success).isTrue()
24 | }
25 |
26 |
27 | @Test
28 | fun logEntryCouldBeAppliedSequentially() = test {
29 | val request = buildAppendRequest()
30 | assertThat(state.log.lastIndex()).isEqualTo(-1)
31 |
32 | val result = state.appendEntries(request)
33 | assertThat(state.log.lastIndex()).isEqualTo(0)
34 |
35 | val rTwo = buildAppendRequest(prevLogIndex = 0, prevLogTerm = 0)
36 | val resultTwo = state.appendEntries(rTwo)
37 |
38 | assertThat(state.log.lastIndex()).isEqualTo(1)
39 | assertThat(resultTwo?.success).isTrue()
40 | }
41 |
42 | @Test
43 | fun prevTermValidationIsWorking() = test {
44 | val request = buildAppendRequest()
45 | val result = state.appendEntries(request)
46 | assertThat(state.log.lastIndex()).isEqualTo(0)
47 |
48 | val rTwo = buildAppendRequest(prevLogIndex = 0, prevLogTerm = 1)
49 | val resultTwo = state.appendEntries(rTwo)
50 |
51 | assertThat(state.log.lastIndex()).isEqualTo(0)
52 | assertThat(resultTwo?.success).isFalse()
53 | }
54 |
55 | @Test
56 | fun prevLogIndexValidationIsWorking() = test {
57 | val request = buildAppendRequest()
58 | val result = state.appendEntries(request)
59 | assertThat(state.log.lastIndex()).isEqualTo(0)
60 |
61 | val rTwo = buildAppendRequest(prevLogIndex = 1, prevLogTerm = 0)
62 | val resultTwo = state.appendEntries(rTwo)
63 | assertThat(state.log.lastIndex()).isEqualTo(0)
64 | assertThat(resultTwo?.success).isFalse()
65 | }
66 |
67 |
68 | @Test
69 | fun expectSameEntryApplicationOverrided() = test {
70 | val request = buildAppendRequest()
71 | assertThat(state.log.lastIndex()).isEqualTo(-1)
72 |
73 | val result = state.appendEntries(request)
74 | assertThat(state.log.lastIndex()).isEqualTo(0)
75 |
76 | val resultTwo = state.appendEntries(request)
77 |
78 | assertThat(state.log.lastIndex()).isEqualTo(0)
79 | assertThat(resultTwo?.success).isTrue()
80 | }
81 |
82 |
83 | @Test
84 | fun expectOutdatedRequestTermIsDetected() = test {
85 | val request = buildAppendRequest(currentTerm = -1)
86 | assertThat(state.log.lastIndex()).isEqualTo(-1)
87 |
88 | val result = state.appendEntries(request)
89 | assertThat(state.log.lastIndex()).isEqualTo(-1)
90 | assertThat(result?.success).isFalse()
91 | }
92 |
93 |
94 | @Test
95 | fun expectConflictingLogEntryWillOverwriteExisting() = test {
96 | state.appendEntries(buildAppendRequest())
97 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 0))
98 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 1))
99 |
100 | assertThat(state.log.lastIndex()).isEqualTo(2)
101 |
102 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 0, entryTerm = 1))
103 |
104 | assertThat(state.log.lastIndex()).isEqualTo(1)
105 | }
106 |
107 | @Test
108 | fun expectNonConflictingLogEntryWillNotOverwriteExisting() = test {
109 | state.appendEntries(buildAppendRequest())
110 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 0))
111 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 1))
112 |
113 | assertThat(state.log.lastIndex()).isEqualTo(2)
114 |
115 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 0))
116 |
117 | assertThat(state.log.lastIndex()).isEqualTo(2)
118 | }
119 |
120 |
121 | @Test
122 | fun expectCommitIndexUpdatesToLeaders() = test {
123 | state.appendEntries(buildAppendRequest())
124 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 0))
125 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 1, leaderCommit = 1))
126 |
127 |
128 | assertThat(state.log.lastIndex()).isEqualTo(2)
129 | assertThat(state.log.commitIndex).isEqualTo(1)
130 | }
131 |
132 |
133 | @Test
134 | fun expectCommitIndexUpdatesAccordingToTheRulesOfMinBetweenLeaderAndLog() = test {
135 | state.appendEntries(buildAppendRequest())
136 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 0))
137 | state.appendEntries(buildAppendRequest(prevLogTerm = 0, prevLogIndex = 1, leaderCommit = 4))
138 |
139 |
140 | assertThat(state.log.lastIndex()).isEqualTo(2)
141 | assertThat(state.log.commitIndex).isEqualTo(2)
142 | }
143 |
144 |
145 | @Test
146 | fun expectHeartbeatCanBeProcessed() = test {
147 | state.appendEntries(heartbeatRequest(leaderId = 1))
148 |
149 | assertThat(state.leaderId).isEqualTo(1)
150 | assertThat(state.current).isEqualTo(NodeState.FOLLOWER)
151 | }
152 |
153 |
154 | @Test
155 | fun expectAppendMessageWithHigherTermUpdatesTerm() = test {
156 | state.appendEntries(heartbeatRequest(leaderId = 1, currentTerm = 5))
157 |
158 | assertThat(state.term).isEqualTo(5)
159 | assertThat(state.leaderId).isEqualTo(1)
160 |
161 | }
162 |
163 | @Test
164 | fun expectAppendMessageWithHigherTermConvertsToFollower() = test {
165 | state.nextTerm(1)
166 | assertThat(state.current).isEqualTo(NodeState.CANDIDATE)
167 |
168 | state.appendEntries(heartbeatRequest(leaderId = 1, currentTerm = 2))
169 |
170 | assertThat(state.term).isEqualTo(2)
171 | assertThat(state.leaderId).isEqualTo(1)
172 | assertThat(state.current).isEqualTo(NodeState.FOLLOWER)
173 |
174 | }
175 |
176 | fun buildAppendRequest(prevLogTerm: Long = -1,
177 | prevLogIndex: Int = -1,
178 | entryTerm: Long = 0,
179 | currentTerm: Long = 0,
180 | leaderId: Int = 1,
181 | leaderCommit: Int = 0): AppendRequest {
182 | val entry = LogEntry.newBuilder().setTerm(entryTerm).build()
183 | val request = AppendRequest.newBuilder()
184 | .setTerm(currentTerm)
185 | .setLeaderId(leaderId)
186 | .setLeaderCommit(leaderCommit)
187 | .setPrevLogIndex(prevLogIndex)
188 | .setPrevLogTerm(prevLogTerm)
189 | .addEntries(entry)
190 | .build()
191 | return request
192 | }
193 |
194 | fun heartbeatRequest(prevLogTerm: Long = -1,
195 | prevLogIndex: Int = -1,
196 | currentTerm: Long = 0,
197 | leaderId: Int = 1,
198 | leaderCommit: Int = 0): AppendRequest {
199 |
200 | return AppendRequest.newBuilder()
201 | .setTerm(currentTerm)
202 | .setLeaderId(leaderId)
203 | .setLeaderCommit(leaderCommit)
204 | .setPrevLogIndex(prevLogIndex)
205 | .setPrevLogTerm(prevLogTerm)
206 | .build()
207 | }
208 |
209 | }
--------------------------------------------------------------------------------