├── .gitignore
├── .sbtopts
├── .scalafmt.conf
├── .travis.yml
├── LICENSE
├── README.md
├── build.sbt
├── build.sc
├── dockerBuild.sh
├── makeDoc.sh
├── package.sh
├── project
├── Dependencies.scala
├── build.properties
└── plugins.sbt
├── riff-akka
└── src
│ └── main
│ └── scala
│ └── riff
│ └── akka
│ ├── http
│ └── WebServer.scala
│ └── streams
│ └── RiffStream.scala
├── riff-core
├── js
│ └── src
│ │ ├── main
│ │ ├── resources
│ │ │ └── index-dev.html
│ │ └── scala
│ │ │ ├── org
│ │ │ └── reactivestreams
│ │ │ │ ├── Processor.scala
│ │ │ │ ├── Publisher.scala
│ │ │ │ ├── Subscriber.scala
│ │ │ │ └── Subscription.scala
│ │ │ └── riff
│ │ │ └── raft
│ │ │ ├── log
│ │ │ ├── RaftLog.scala
│ │ │ ├── Repo.scala
│ │ │ └── StringFormat.scala
│ │ │ └── timer
│ │ │ └── DefaultClock.scala
│ │ └── test
│ │ └── scala
│ │ └── riff
│ │ ├── RiffSpec.scala
│ │ └── raft
│ │ ├── log
│ │ └── JSRaftLogTest.scala
│ │ └── timer
│ │ └── RaftTimerTest.scala_ignored
├── jvm
│ └── src
│ │ ├── main
│ │ └── scala
│ │ │ └── riff
│ │ │ ├── RaftPipe.scala
│ │ │ ├── raft
│ │ │ ├── log
│ │ │ │ ├── FileBasedLog.scala
│ │ │ │ └── RaftLog.scala
│ │ │ ├── node
│ │ │ │ ├── NIOPersistentState.scala
│ │ │ │ └── RecordingMessageHandler.scala
│ │ │ ├── reactive
│ │ │ │ └── ReactiveClient.scala
│ │ │ └── timer
│ │ │ │ └── DefaultClock.scala
│ │ │ └── reactive
│ │ │ ├── AsyncPublisher.scala
│ │ │ ├── AsyncSubscription.scala
│ │ │ ├── AsyncSubscriptionState.scala
│ │ │ ├── MultiSubscriberProcessor.scala
│ │ │ ├── ReactivePipe.scala
│ │ │ ├── ReactiveTimerCallback.scala
│ │ │ ├── ReplayPublisher.scala
│ │ │ └── SingleSubscriber.scala
│ │ └── test
│ │ ├── resources
│ │ ├── input.sh
│ │ └── logback-test.xml
│ │ └── scala
│ │ └── riff
│ │ ├── RaftPipeTest.scala
│ │ ├── RiffSpec.scala
│ │ ├── RiffThreadedSpec.scala
│ │ ├── raft
│ │ ├── log
│ │ │ └── FileBasedRaftLogTest.scala
│ │ ├── node
│ │ │ └── NIOPersistentStateTest.scala
│ │ └── timer
│ │ │ └── RaftClockTest.scala
│ │ └── reactive
│ │ ├── AsPublisherTest.scala
│ │ ├── AsyncPublisherTest.scala
│ │ ├── AsyncPublisherVerificationTest.scala
│ │ ├── AsyncSubscriptionStateTest.scala
│ │ ├── AsyncSubscriptionTest.scala
│ │ ├── CollectPublisherTest.scala
│ │ ├── FixedPublisher.scala
│ │ ├── FixedPublisherTest.scala
│ │ ├── MultiSubscriberProcessorTest.scala
│ │ ├── PublishersTest.scala
│ │ ├── ReactivePipeTest.scala
│ │ ├── ReplayPublisherTest.scala
│ │ ├── TestListener.scala
│ │ └── reactive.scala
└── shared
│ └── src
│ ├── main
│ └── scala
│ │ └── riff
│ │ ├── raft
│ │ ├── AppendStatus.scala
│ │ ├── RaftClient.scala
│ │ ├── client
│ │ │ ├── SingleAppendFSM.scala
│ │ │ └── StateUpdateMsg.scala
│ │ ├── exceptions.scala
│ │ ├── log
│ │ │ ├── BaseLog.scala
│ │ │ ├── CachingLog.scala
│ │ │ ├── DelegateLog.scala
│ │ │ ├── InMemory.scala
│ │ │ ├── LogAppendResult.scala
│ │ │ ├── LogCoords.scala
│ │ │ ├── LogEntry.scala
│ │ │ ├── LogState.scala
│ │ │ ├── RaftLogOps.scala
│ │ │ └── StateMachineLog.scala
│ │ ├── messages
│ │ │ └── RaftMessage.scala
│ │ ├── node
│ │ │ ├── CandidateState.scala
│ │ │ ├── LeadersClusterView.scala
│ │ │ ├── NodeRole.scala
│ │ │ ├── NodeState.scala
│ │ │ ├── Peer.scala
│ │ │ ├── PersistentState.scala
│ │ │ ├── RaftCluster.scala
│ │ │ ├── RaftMessageHandler.scala
│ │ │ ├── RaftNode.scala
│ │ │ ├── RaftNodeResult.scala
│ │ │ └── RoleCallback.scala
│ │ ├── package.scala
│ │ └── timer
│ │ │ ├── RaftClock.scala
│ │ │ ├── RandomTimer.scala
│ │ │ ├── TimerCallback.scala
│ │ │ └── Timers.scala
│ │ └── reactive
│ │ ├── AsPublisher.scala
│ │ ├── AsSubscriber.scala
│ │ ├── CollectPublisher.scala
│ │ ├── MapPublisher.scala
│ │ ├── Publishers.scala
│ │ └── Subscribers.scala
│ └── test
│ └── scala
│ └── riff
│ ├── BaseSpec.scala
│ ├── raft
│ ├── client
│ │ └── SingleAppendFSMTest.scala
│ ├── integration
│ │ ├── IntegrationTest.scala
│ │ ├── package.scala
│ │ └── simulator
│ │ │ ├── AdvanceResult.scala
│ │ │ ├── HasTimeline.scala
│ │ │ ├── MergeSorted.scala
│ │ │ ├── MergeSortedTest.scala
│ │ │ ├── RaftSimulator.scala
│ │ │ ├── SimulatedClock.scala
│ │ │ ├── Timeline.scala
│ │ │ ├── TimelineTest.scala
│ │ │ ├── TimelineType.scala
│ │ │ └── snapshots.scala
│ ├── log
│ │ ├── LogAppendSuccessTest.scala
│ │ ├── RaftLogTCK.scala
│ │ └── RaftLogTest.scala
│ ├── messages
│ │ ├── AppendDataTest.scala
│ │ ├── AppendEntriesResponseTest.scala
│ │ └── AppendEntriesTest.scala
│ ├── node
│ │ ├── ElectionTest.scala
│ │ ├── Handlers.scala
│ │ ├── LeaderNodeTest.scala
│ │ ├── LeadersClusterViewTest.scala
│ │ ├── PeerTest.scala
│ │ ├── PersistentStateTCK.scala
│ │ ├── PersistentStateTest.scala
│ │ ├── RaftClusterTest.scala
│ │ ├── RaftNodeTest.scala
│ │ └── RichNodeState.scala
│ ├── packageTest.scala
│ └── timer
│ │ ├── LoggedInvocationClock.scala
│ │ ├── RaftClockTCK.scala
│ │ └── RandomTimerTest.scala
│ └── reactive
│ └── SubscribersTest.scala
├── riff-fs2
└── src
│ └── main
│ └── scala
│ └── riff
│ └── fs2
│ └── Fs2Clock.scala
├── riff-json
├── jvm
│ └── src
│ │ └── test
│ │ └── scala
│ │ └── riff
│ │ └── json
│ │ └── RaftMessageFormatJVMTest.scala
└── shared
│ └── src
│ ├── main
│ └── scala
│ │ └── riff
│ │ └── json
│ │ ├── LowPriorityRiffJsonImplicits.scala
│ │ ├── RaftMessageFormat.scala
│ │ └── implicits.scala
│ └── test
│ └── scala
│ └── riff
│ └── json
│ └── RaftMessageFormatTest.scala
├── riff-monix
├── TODO.md
└── src
│ ├── main
│ └── scala
│ │ └── riff
│ │ └── monix
│ │ ├── EventSource.scala
│ │ ├── InfiniteConcurrentSubject.scala
│ │ ├── LowPriorityRiffMonixImplicits.scala
│ │ ├── MonixClient.scala
│ │ ├── MonixClock.scala
│ │ ├── ObservableCache.scala
│ │ ├── ObservableRaftEndpoint.scala
│ │ ├── ObservableState.scala
│ │ ├── ObservableTimerCallback.scala
│ │ ├── RaftPipeMonix.scala
│ │ ├── RiffSchedulers.scala
│ │ ├── client
│ │ └── AppendStatusObservable.scala
│ │ └── log
│ │ ├── AppendOps.scala
│ │ ├── CommittedOps.scala
│ │ ├── ObservableLog.scala
│ │ └── package.scala
│ └── test
│ └── scala
│ └── riff
│ └── monix
│ ├── EventSourceTest.scala
│ ├── LowPriorityRiffMonixImplicitsTest.scala
│ ├── MonixClientTest.scala
│ ├── ObservableTimerCallbackTest.scala
│ ├── RaftPipeMonixTest.scala
│ ├── RiffMonixSpec.scala
│ ├── client
│ └── AppendStatusObservableTest.scala
│ └── log
│ └── ObservableLogTest.scala
├── riff-runtime
└── src
│ └── main
│ └── scala
│ └── riff
│ └── runtime
│ └── RiffApp.scala
├── riff-vertx
└── src
│ ├── main
│ ├── resources
│ │ └── logback.xml
│ └── scala
│ │ ├── riff
│ │ └── web
│ │ │ └── vertx
│ │ │ ├── WebFrameEndpoint.scala
│ │ │ ├── WebSocketObserver.scala
│ │ │ ├── client
│ │ │ ├── RestClient.scala
│ │ │ └── SocketClient.scala
│ │ │ └── server
│ │ │ ├── Main.scala
│ │ │ ├── RestHandler.scala
│ │ │ ├── RoutingSocketHandler.scala
│ │ │ ├── Server.scala
│ │ │ ├── ServerEndpoint.scala
│ │ │ ├── ServerWebSocketHandler.scala
│ │ │ ├── Startup.scala
│ │ │ └── WebSocketFrameAsWebFrame.scala
│ │ └── streaming
│ │ ├── api
│ │ ├── Endpoint.scala
│ │ ├── HostPort.scala
│ │ ├── implicits
│ │ │ ├── LowPriorityStreamingImplicits.scala
│ │ │ └── package.scala
│ │ └── sockets
│ │ │ └── WebFrame.scala
│ │ └── rest
│ │ ├── EndpointCoords.scala
│ │ ├── HttpMethod.scala
│ │ ├── RestInput.scala
│ │ ├── RestRequest.scala
│ │ ├── StreamingConfig.scala
│ │ └── WebURI.scala
│ └── test
│ └── scala
│ └── riff
│ └── web
│ └── vertx
│ ├── SocketClientServerIntegrationTest.scala
│ ├── client
│ └── RestClientTest.scala
│ └── server
│ └── StartupTest.scala
├── src
└── main
│ └── paradox
│ ├── about
│ ├── api.md
│ ├── gettingStarted.md
│ ├── index.md
│ └── usingIntelliJ.md
│ ├── blog
│ ├── 01_raftSimulatorForTheWin.md
│ ├── 02_eventSourceAGoGo.md
│ ├── 03_overParameterization.md
│ ├── 04_goingReactive.md
│ ├── 05_moreNonConcurrentTesting.md
│ └── index.md
│ └── index.md
├── test.sh
├── travis
├── README.MD
└── jvmopts
└── version.sbt
/.gitignore:
--------------------------------------------------------------------------------
1 | .DS_Store
2 | *.log
3 | target
4 | .idea
5 | .idea_modules
6 | out
7 | shared-src
8 |
--------------------------------------------------------------------------------
/.sbtopts:
--------------------------------------------------------------------------------
1 | -J-Xms2g
2 | -J-Xmx2g
3 |
--------------------------------------------------------------------------------
/.scalafmt.conf:
--------------------------------------------------------------------------------
1 | # http://scalameta.org/scalafmt
2 | maxColumn = 180
3 |
4 | align = more
5 |
6 |
7 | # If true, the margin character | is aligned with the opening triple quote string literals
8 | assumeStandardLibraryStripMargin = true
9 |
10 | newlines {
11 | alwaysBeforeTopLevelStatements = true
12 | sometimesBeforeColonInMethodReturnType = true
13 | penalizeSingleSelectMultiArgList = false
14 | alwaysBeforeElseAfterCurlyIf = false
15 | neverInResultType = false
16 | }
17 |
18 | spaces {
19 | afterKeywordBeforeParen = true
20 | }
21 |
22 | binPack {
23 | parentConstructors = true
24 | literalArgumentLists = true
25 | }
26 |
27 | runner {
28 | optimizer {
29 | # Set to -1 to disable. Number of characters needed to trigger "config-style" formatting
30 | # see: http://scalameta.org/scalafmt/#runner.optimizer.forceConfigStyleOnOffset
31 | forceConfigStyleOnOffset = 150
32 |
33 | # minimum number of func arguments before config-style (look at top of file) is enabled
34 | forceConfigStyleMinArgCount = 2
35 | }
36 | }
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language: scala
2 | scala:
3 | - 2.12.7
4 | script:
5 | - sbt ++$TRAVIS_SCALA_VERSION -jvm-opts travis/jvmopts clean test
6 | jdk:
7 | - oraclejdk8
8 |
9 | sudo: false
10 |
11 | cache:
12 | directories:
13 | - $HOME/.sbt
14 | - $HOME/.ivy2
15 | - $HOME/.coursier
16 |
17 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Riff
2 | [](https://travis-ci.org/aaronp/riff)
3 |
4 | A Raft implementation written in scala which cleanly separates the Raft logic from any particular
5 | transport/framework.
6 |
7 | Check out the project documentation [here](https://aaronp.github.io/riff/index.html)
8 |
9 | ## API docs
10 | * [Core](https://aaronp.github.io/riff/api/riffCoreCrossProject/riff/raft/index.html)
11 | * [Monix](https://aaronp.github.io/riff/api/riffMonix/riff/monix/index.html)
12 | * [FS2](https://aaronp.github.io/riff/api/riffFs2/riff/fs2/)
13 | * [Akka](https://aaronp.github.io/riff/api/riffAkka/riff/akka/http/index.html)
14 |
15 | ## Reasoning
16 |
17 | The core project only has a dependency on eie (a tiny IO library), and then offers support for akka,monix,fs2,http4s,etc
18 | in separate sub-projects.
19 |
20 | The outer-most representation clearly takes its inputs (requests, responses and timer messages) and produces addressed
21 | messages which can then be used by the sub-projects to send REST requests, put messages on queues, pipe them through a
22 | stream, etc.
23 |
24 | From the outside, the Raft node takes requests, responses and timer messages as inputs and produces requests/responses as
25 | outputs.
26 |
27 | Internally, the Raft log or ephemeral node state can also easily implement a reactive streams Publisher so that clients
28 | of this library can e.g. observe the log, state transitions, etc.
29 |
30 | ## Building
31 |
32 | It is currently built using sbt, though I hope to add Mill (and fury?) support shortly
33 | Mill certainly looks really promising!
34 |
35 | ```
36 | sbt clean coverage test coverageReport
37 | ```
--------------------------------------------------------------------------------
/dockerBuild.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | docker run --rm --mount type=bind,source="$(pwd)",target=/opt nightscape/scala-mill /bin/sh -c 'cd /opt; chmod +x test.sh; /opt/test.sh'
3 |
--------------------------------------------------------------------------------
/makeDoc.sh:
--------------------------------------------------------------------------------
1 | sbt paradox makeSite ghpagesPushSite
2 |
--------------------------------------------------------------------------------
/package.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 | # This builds (assembles) a jar and runs it
4 | #
5 | # see http://www.lihaoyi.com/mill/index.html
6 | mill core[2.12.6].assembly
7 |
--------------------------------------------------------------------------------
/project/Dependencies.scala:
--------------------------------------------------------------------------------
1 | import sbt._
2 |
3 | object Dependencies {
4 |
5 | val config: ModuleID = "com.typesafe" % "config" % "1.3.0"
6 |
7 | //https://github.com/typesafehub/scala-logging
8 | val logging = List(
9 | "com.typesafe.scala-logging" %% "scala-logging" % "3.7.2",
10 | "ch.qos.logback" % "logback-classic" % "1.1.11")
11 |
12 | val testDependencies = List(
13 | "org.scalactic" %% "scalactic" % "3.0.5" % "test",
14 | "org.scalatest" %% "scalatest" % "3.0.5" % "test",
15 | "org.pegdown" % "pegdown" % "1.6.0" % "test",
16 | "junit" % "junit" % "4.12" % "test"
17 | )
18 |
19 | val monix = List("monix", "monix-execution", "monix-eval", "monix-reactive", "monix-tail").map { art =>
20 | "io.monix" %% art % "3.0.0-RC1"
21 | }
22 |
23 | val simulacrum: ModuleID = "com.github.mpilquist" %% "simulacrum" % "0.13.0"
24 |
25 | val http4s = List("http4s-blaze-server", "http4s-circe", "http4s-dsl").map { art => "org.http4s" %% art % "0.18.12"
26 | }
27 |
28 | val vertx = List(
29 | "io.vertx" %% "vertx-lang-scala" % "3.5.2",
30 | "io.vertx" %% "vertx-web-scala" % "3.5.2"
31 | )
32 |
33 | val fs2 = List("co.fs2" %% "fs2-core" % "0.10.4")
34 |
35 | val RiffMonix: List[ModuleID] = monix ::: logging ::: testDependencies
36 | val RiffFs2: List[ModuleID] = fs2 ::: logging ::: testDependencies
37 |
38 | val RiffAkka: List[ModuleID] = {
39 | val streams = "com.typesafe.akka" %% "akka-stream" % "2.5.17"
40 | val akkaCirce = "de.heikoseeberger" %% "akka-http-circe" % "1.19.0"
41 | val cors = "ch.megard" %% "akka-http-cors" % "0.2.2"
42 | val akkaHttp = List("", "-core").map { suffix => "com.typesafe.akka" %% s"akka-http$suffix" % "10.1.5"
43 | } :+ cors :+ akkaCirce :+ ("com.typesafe.akka" %% "akka-http-testkit" % "10.1.5" % "test")
44 |
45 | val akka = List(
46 | "com.typesafe.akka" %% "akka-actor" % "2.5.14",
47 | "com.typesafe.akka" %% "akka-testkit" % "2.5.14" % Test
48 | )
49 | streams +: akkaHttp ::: akka ::: logging ::: testDependencies
50 | }
51 | val RiffHttp4s: List[ModuleID] = http4s ::: logging ::: testDependencies
52 | val RiffVertx: List[ModuleID] = config :: monix ::: vertx ::: logging ::: testDependencies
53 | }
54 |
--------------------------------------------------------------------------------
/project/build.properties:
--------------------------------------------------------------------------------
1 | sbt.version=1.2.0
2 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.5")
2 | addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.7.0")
3 | addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.6.2")
4 | addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.9.3")
5 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.1.0")
6 | addSbtPlugin("com.lucidchart" % "sbt-scalafmt" % "1.14")
7 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.5.1")
8 | addSbtPlugin("com.typesafe.sbt" % "sbt-site" % "1.3.2")
9 | addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.0")
10 | addSbtPlugin("com.timushev.sbt" % "sbt-updates" % "0.3.1")
11 | addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.4.1")
12 | addSbtPlugin("org.portable-scala" % "sbt-scalajs-crossproject" % "0.6.0")
13 | addSbtPlugin("org.scala-js" % "sbt-scalajs" % "0.6.25")
14 | addSbtPlugin("com.lightbend.paradox" % "sbt-paradox" % "0.4.2")
15 | addSbtPlugin("com.lightbend.paradox" % "sbt-paradox-theme" % "0.4.2")
16 | addSbtPlugin("de.johoop" % "sbt-testng-plugin" % "3.1.1")
--------------------------------------------------------------------------------
/riff-akka/src/main/scala/riff/akka/http/WebServer.scala:
--------------------------------------------------------------------------------
1 | package riff.akka.http
2 |
3 | import akka.actor.ActorSystem
4 | import akka.http.scaladsl.Http
5 | import akka.http.scaladsl.model._
6 | import akka.http.scaladsl.server.Directives._
7 | import akka.stream.ActorMaterializer
8 | import scala.io.StdIn
9 |
10 | object WebServer {
11 |
12 | def main(args: Array[String]): Unit = {
13 |
14 | implicit val system = ActorSystem("my-system")
15 | implicit val materializer = ActorMaterializer()
16 | // needed for the future flatMap/onComplete in the end
17 | implicit val executionContext = system.dispatcher
18 |
19 | val route =
20 | path("hello") {
21 | get {
22 | complete(HttpEntity(ContentTypes.`text/html(UTF-8)`, "
Say hello to akka-http
"))
23 | }
24 | }
25 |
26 | val bindingFuture = Http().bindAndHandle(route, "localhost", 8080)
27 |
28 | println(s"Server online at http://localhost:8080/\nPress RETURN to stop...")
29 | StdIn.readLine() // let it run until user presses return
30 | bindingFuture
31 | .flatMap(_.unbind()) // trigger unbinding from the port
32 | .onComplete(_ => system.terminate()) // and shutdown when done
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/riff-akka/src/main/scala/riff/akka/streams/RiffStream.scala:
--------------------------------------------------------------------------------
1 | package riff.akka.streams
2 |
3 | object RiffStream {
4 |
5 | import akka.stream._
6 | import akka.stream.scaladsl._
7 |
8 | }
9 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/resources/index-dev.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | Example Scala.js application
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
17 |
18 |
19 |
20 |
21 |
24 |
25 |
28 |
29 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/org/reactivestreams/Processor.scala:
--------------------------------------------------------------------------------
1 | package org.reactivestreams
2 |
3 | /** A Processor represents a processing stage—which is both a [[Subscriber]]
4 | * and a [[Publisher]] and obeys the contracts of both.
5 | *
6 | * @tparam T the type of element signaled to the [[Subscriber]]
7 | * @tparam R the type of element signaled by the [[Publisher]]
8 | */
9 | trait Processor[T, R] extends Subscriber[T] with Publisher[R]
10 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/org/reactivestreams/Publisher.scala:
--------------------------------------------------------------------------------
1 | package org.reactivestreams
2 |
3 | /** A [[Publisher]] is a provider of a potentially unbounded number of
4 | * sequenced elements, publishing them according to the demand received
5 | * from its [[Subscriber]].
6 | *
7 | * A [[Publisher]] can serve multiple [[Subscriber Subscribers]] are
8 | * [[Publisher!.subscribe subscribed]] dynamically at various points in time.
9 | *
10 | * @tparam T the type of element signaled.
11 | */
12 | trait Publisher[T] {
13 |
14 | /**
15 | * Request the publisher to start streaming data.
16 | *
17 | * This is a factory method and can be called multiple times, each time
18 | * starting a new [[Subscription]].
19 | *
20 | * Each [[Subscription]] will work for only a single [[Subscriber]].
21 | * A [[Subscriber]] should only subscribe once to a single [[Publisher]].
22 | *
23 | * If the [[Publisher]] rejects the subscription attempt or otherwise fails
24 | * it will signal the error via [[Subscriber.onError]].
25 | *
26 | * @param subscriber the [[Subscriber]] that will consume signals
27 | * from this [[Publisher]]
28 | */
29 | def subscribe(subscriber: Subscriber[_ >: T]): Unit
30 | }
31 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/org/reactivestreams/Subscriber.scala:
--------------------------------------------------------------------------------
1 | package org.reactivestreams
2 |
3 | /** Will receive call to [[Subscriber!.onSubscribe onSubscribe]]
4 | * once after passing an instance of [[Subscriber]] to [[Publisher.subscribe]].
5 | *
6 | * No further notifications will be received until [[Subscription.request]] is called.
7 | *
8 | * After signaling demand:
9 | *
10 | * - One or more invocations of [[Subscriber!.onNext onNext]] up to the
11 | * maximum number defined by [[Subscription.request]]
12 | * - Single invocations of [[Subscriber!.onError onError]] or
13 | * [[Subscriber!.onComplete onComplete]] which signals a terminal state after
14 | * which no furter events will be sent.
15 | *
16 | * Demand can be signaled via [[Subscription.request]] whenever the
17 | * [[Subscriber]] instance is capable of handling more.
18 | *
19 | * @tparam T the type of element signaled.
20 | */
21 | trait Subscriber[T] {
22 |
23 | /** Invoked after calling [[Publisher.subscribe]].
24 | *
25 | * No data will start flowing until [[Subscription.request]] is invoked.
26 | *
27 | * It is the responsibility of this [[Subscriber]] instance
28 | * to call [[Subscription.request]] whenever more data is wanted.
29 | *
30 | * The [[Publisher]] will send notifications only
31 | * in response to [[Subscription.request]].
32 | *
33 | * @param s [[Subscription]] that allows requesting data via [[Subscription.request]]
34 | */
35 | def onSubscribe(s: Subscription): Unit
36 |
37 | /** Data notification sent by the [[Publisher]] in response to
38 | * requests to [[Subscription.request]].
39 | *
40 | * @param t the element signaled
41 | */
42 | def onNext(t: T): Unit
43 |
44 | /** Failed terminal state.
45 | *
46 | * No further events will be sent even
47 | * if [[Subscription.request]] is invoked again.
48 | *
49 | * @param t the throwable signaled
50 | */
51 | def onError(t: Throwable): Unit
52 |
53 | /** Successful terminal state.
54 | *
55 | * No further events will be sent even
56 | * if [[Subscription.request]] is invoked again.
57 | */
58 | def onComplete(): Unit
59 | }
60 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/org/reactivestreams/Subscription.scala:
--------------------------------------------------------------------------------
1 | package org.reactivestreams
2 |
3 | /** A [[Subscription]] represents a one-to-one lifecycle of a [[Subscriber]]
4 | * subscribing to a [[Publisher]].
5 | *
6 | * It can be used only once by a single [[Subscriber]]. It is used
7 | * for both signaling demand for data and for canceling demand (and allow
8 | * resource cleanup).
9 | */
10 | trait Subscription {
11 |
12 | /** No events will be sent by a [[Publisher]] until demand is signaled via this method.
13 | *
14 | * It can be called however often and whenever needed—but the outstanding cumulative
15 | * demand must never exceed Long.MAX_VALUE. An outstanding cumulative demand
16 | * of Long.MAX_VALUE may be treated by the [[Publisher]] as "effectively unbounded".
17 | *
18 | * Whatever has been requested can be sent by the [[Publisher]] so
19 | * only signal demand for what can be safely handled.
20 | *
21 | * A [[Publisher]] can send less than is requested if the stream ends but
22 | * then must emit either [[Subscriber.onError]] or [[Subscriber.onComplete]].
23 | *
24 | * @param n the strictly positive number of elements to requests to
25 | * the upstream [[Publisher]].
26 | */
27 | def request(n: Long): Unit
28 |
29 | /** Request the [[Publisher]] to stop sending data and clean up resources.
30 | *
31 | * Data may still be sent to meet previously signalled demand after
32 | * calling cancel as this request is asynchronous.
33 | */
34 | def cancel(): Unit
35 | }
36 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/riff/raft/log/Repo.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import org.scalajs.dom.raw.Storage
3 |
4 | /**
5 | * Abstraction from Storage for testing -- you probably don't have to do this,
6 | * but you can't just 'new Storage'
7 | *
8 | * @tparam A
9 | */
10 | trait Repo {
11 | def setItem(key: String, value: String)
12 | def getItem(key: String): Option[String]
13 | def removeItem(key: String): Unit
14 | }
15 |
16 | object Repo {
17 |
18 | class Instance[A: StringFormat](storage: Storage) extends Repo {
19 | override def setItem(key: String, value: String): Unit = {
20 | storage.setItem(key, value)
21 | }
22 | override def getItem(key: String): Option[String] = {
23 | Option(storage.getItem(key))
24 | }
25 | override def removeItem(key: String): Unit = storage.removeItem(key)
26 | }
27 | def apply[A: StringFormat](storage: Storage) = new Instance(storage)
28 |
29 | def apply[A: StringFormat](): Repo = new Repo {
30 | val fmt = StringFormat[A]
31 | private var map = Map[String, String]()
32 | override def setItem(key: String, value: String): Unit = {
33 | map = map.updated(key, value)
34 | }
35 | override def getItem(key: String): Option[String] = {
36 | map.get(key)
37 | }
38 | override def removeItem(key: String): Unit = map = map - key
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/riff/raft/log/StringFormat.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import scala.util.{Success, Try}
3 |
4 | trait StringFormat[A] {
5 | def asString(value: A): String
6 | def fromString(value: String): Try[A]
7 | }
8 |
9 | object StringFormat {
10 | def apply[A](implicit inst: StringFormat[A]) = inst
11 |
12 | implicit object Identity extends StringFormat[String] {
13 | override def asString(value: String): String = value
14 | override def fromString(value: String): Try[String] = Success(value)
15 | }
16 | }
17 |
--------------------------------------------------------------------------------
/riff-core/js/src/main/scala/riff/raft/timer/DefaultClock.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | import org.scalajs.dom.window
4 |
5 | import scala.concurrent.duration.FiniteDuration
6 |
7 | class DefaultClock(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeatTimeout: RandomTimer) extends RaftClock {
8 |
9 | type CancelT = Int
10 |
11 | def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): CancelT = {
12 | window.setTimeout(() => callback.onReceiveHeartbeatTimeout(), sendHeartbeatTimeout.toMillis)
13 | }
14 |
15 | def resetSendHeartbeatTimeout(callback: TimerCallback[_]): CancelT = {
16 | window.setTimeout(() => callback.onSendHeartbeatTimeout(), sendHeartbeatTimeout.toMillis)
17 | }
18 |
19 | def cancelTimeout(c: CancelT): Unit = {
20 | window.clearTimeout(c)
21 | }
22 | }
23 |
24 | object DefaultClock {
25 |
26 | def apply(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeatTimeout: RandomTimer) = {
27 | new DefaultClock(sendHeartbeatTimeout, receiveHeartbeatTimeout)
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/riff-core/js/src/test/scala/riff/RiffSpec.scala:
--------------------------------------------------------------------------------
1 | package riff
2 | import scala.concurrent.duration.FiniteDuration
3 |
4 | class RiffSpec extends BaseSpec
5 |
--------------------------------------------------------------------------------
/riff-core/js/src/test/scala/riff/raft/log/JSRaftLogTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import scala.scalajs.js
3 |
4 | class JSRaftLogTest extends RaftLogTCK {
5 |
6 | override protected def withLog(test: RaftLog[String] => Unit): Unit = {
7 |
8 | RaftLog[String](Repo[String]())
9 | }
10 |
11 | }
12 |
--------------------------------------------------------------------------------
/riff-core/js/src/test/scala/riff/raft/timer/RaftTimerTest.scala_ignored:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 | import scala.concurrent.duration._
3 | import org.scalajs.dom.window
4 |
5 | import scala.concurrent.{Await, Future}
6 | import scalajs.concurrent.JSExecutionContext.Implicits.queue
7 | import scala.scalajs.js._
8 |
9 | /**
10 | * If you have a custom timer (e.g. in monix, cats, fs2, akka, etc), you should just be able to extend the TCK
11 | * and plug in your implementation like here
12 | */
13 | class RaftTimerTest extends RaftTimerTCK {
14 | override def newTimer[A](sendHeartbeatTimeout: FiniteDuration, receiveHeartbeatTimeout: FiniteDuration)(implicit callback: TimerCallback[A]) =
15 | RaftTimer[A](sendHeartbeatTimeout, receiveHeartbeatTimeout)
16 | //
17 | //
18 | override def assertAfter[T](time: FiniteDuration)(f: => T) = {
19 | // val fut = Future {
20 | //
21 | // }
22 | // Await.result(fut, time)
23 | window.setTimeout(() => f, time.toMillis)
24 | // Thread.sleep()
25 |
26 | }
27 | }
28 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/raft/log/RaftLog.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import java.nio.file.Path
3 |
4 | import eie.io.{FromBytes, ToBytes}
5 |
6 | /**
7 | * The persistent log
8 | *
9 | * @tparam T the type stored in the log
10 | */
11 | trait RaftLog[A] extends RaftLogOps[A]
12 |
13 | object RaftLog {
14 |
15 | def apply[A: ToBytes: FromBytes](path: Path, createIfNotExists: Boolean = false): FileBasedLog[A] = {
16 | FileBasedLog[A](path, createIfNotExists)
17 | }
18 |
19 | def inMemory[A]() = new InMemory[A]
20 |
21 | }
22 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/raft/node/NIOPersistentState.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import java.nio.file.Path
3 |
4 | import eie.io._
5 | import riff.raft.Term
6 |
7 | object NIOPersistentState {
8 |
9 | def apply(dir: Path, createDirIfNotExists: Boolean): PersistentState = {
10 | val dirOk = dir.isDir || (createDirIfNotExists && dir.mkDirs().isDir)
11 | require(dirOk, s"${dir} is not a directory")
12 | new NIOPersistentState(dir).cached()
13 | }
14 | }
15 |
16 | /**
17 | * Some [[PersistentState]] written to the file system using java.nio
18 | *
19 | * @param dir
20 | * @param ev$1
21 | * @param ev$2
22 | * @tparam NodeKey
23 | */
24 | class NIOPersistentState(dir: Path) extends PersistentState {
25 |
26 | private val currentTermFile = {
27 | val path = dir.resolve(".currentTerm")
28 | if (!path.exists()) {
29 | path.text = 0.toString
30 | }
31 | path
32 | }
33 |
34 | override def currentTerm: Term = currentTermFile.text.toInt
35 |
36 | override def currentTerm_=(term: Term) = {
37 | val current = currentTerm
38 | require(term > current, s"attempt to decrement term from $current to $term")
39 | currentTermFile.text = term.toString
40 | this
41 | }
42 |
43 | private def votedForFile(term: Term) = dir.resolve(s"${term}.votedFor")
44 |
45 | override def votedFor(term: Term): Option[String] = {
46 | val path = votedForFile(term)
47 | if (path.exists()) {
48 | FromBytes[String].read(path.bytes).toOption
49 | } else {
50 | None
51 | }
52 | }
53 |
54 | override def castVote(term: Term, node: String) = {
55 | val alreadyVoted = votedFor(term)
56 | require(alreadyVoted.isEmpty, s"Already voted in term $term for $alreadyVoted")
57 | votedForFile(term).bytes = ToBytes[String].bytes(node)
58 | }
59 |
60 | }
61 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/raft/node/RecordingMessageHandler.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import com.typesafe.scalalogging.StrictLogging
3 | import riff.raft.NodeId
4 | import riff.raft.messages.RaftMessage
5 |
6 | import scala.util.control.NonFatal
7 |
8 | case class RecordingMessageHandler[A](underlying: RaftMessageHandler[A]) extends RaftMessageHandler[A] with StrictLogging with AutoCloseable {
9 | private var requests: List[RaftMessage[A]] = Nil
10 |
11 | private var responses: List[RaftNodeResult[A]] = Nil
12 |
13 | private def getStack(): String = {
14 | val t = Thread.currentThread()
15 |
16 | t.getStackTrace.tail.tail.mkString(s"${t.getName} call stack:\n\t\t* ", "\n\t\t* ", "\n")
17 | }
18 |
19 | @volatile private var handling = false
20 | @volatile private var callStack = getStack()
21 |
22 | override def nodeId: NodeId = underlying.nodeId
23 |
24 | override def onMessage(input: RaftMessage[A]): Result = {
25 | require(!handling, "not single-threaded! : " + callStack)
26 |
27 | callStack = getStack()
28 |
29 | handling = true
30 | requests = input :: requests
31 | val response = try {
32 | underlying.onMessage(input)
33 | } catch {
34 | case NonFatal(e) =>
35 | logger.error(s"Error on message $input: $e")
36 |
37 | (requests.tail zip responses).foreach {
38 | case tuple @ (in, out) =>
39 | logger.error(s"""${nodeId} >>
40 | |$in
41 | |--> $out
42 | |""".stripMargin)
43 | }
44 |
45 | throw e
46 | }
47 | responses = response :: responses
48 |
49 | handling = false
50 | response
51 | }
52 |
53 | override def close(): Unit = {
54 | underlying match {
55 | case closable: AutoCloseable => closable.close()
56 | case _ =>
57 | }
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/raft/reactive/ReactiveClient.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.reactive
2 | import org.reactivestreams.{Publisher, Subscriber}
3 | import riff.raft.messages.{AppendData, RaftMessage}
4 | import riff.raft.{AppendStatus, RaftClient}
5 | import riff.reactive.ReactivePipe
6 |
7 | import scala.concurrent.ExecutionContext
8 | import scala.reflect.ClassTag
9 |
10 | case class ReactiveClient[A: ClassTag](inputSubscriber: Subscriber[RaftMessage[A]])(implicit executionContext: ExecutionContext) extends RaftClient[Publisher, A] {
11 | override def append(data: Array[A]): Publisher[AppendStatus] = {
12 | val newPipe = ReactivePipe.single[AppendStatus](1000, 10, 100)
13 | inputSubscriber.onNext(AppendData(newPipe.input, data))
14 | newPipe.output
15 | }
16 | }
17 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/raft/timer/DefaultClock.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | import java.util.concurrent.Executors.newScheduledThreadPool
4 | import java.util.concurrent.{ScheduledExecutorService, ScheduledFuture, TimeUnit}
5 |
6 | import scala.concurrent.duration.FiniteDuration
7 |
8 | class DefaultClock(sendHeartbeatTimeout: FiniteDuration,
9 | receiveHeartbeat: RandomTimer,
10 | schedulerService: ScheduledExecutorService = newScheduledThreadPool(1),
11 | cancelMayInterruptIfRunning: Boolean = true)
12 | extends RaftClock with AutoCloseable {
13 | override type CancelT = ScheduledFuture[Unit]
14 |
15 | override def cancelTimeout(c: ScheduledFuture[Unit]): Unit = c.cancel(cancelMayInterruptIfRunning)
16 |
17 | override def resetSendHeartbeatTimeout(callback: TimerCallback[_]): ScheduledFuture[Unit] = {
18 |
19 | val next = schedulerService
20 | .schedule(new Runnable() {
21 | override def run(): Unit = {
22 | callback.onSendHeartbeatTimeout()
23 | ()
24 |
25 | }
26 | }, sendHeartbeatTimeout.toMillis, TimeUnit.MILLISECONDS)
27 | .asInstanceOf[ScheduledFuture[Unit]]
28 |
29 | next
30 | }
31 |
32 | override def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): ScheduledFuture[Unit] = {
33 | val next = schedulerService
34 | .schedule(new Runnable() {
35 | override def run(): Unit = {
36 | callback.onReceiveHeartbeatTimeout()
37 | ()
38 |
39 | }
40 | }, receiveHeartbeat.next().toMillis, TimeUnit.MILLISECONDS)
41 | .asInstanceOf[ScheduledFuture[Unit]]
42 |
43 | next
44 | }
45 |
46 | override def close(): Unit = {
47 | schedulerService.shutdown()
48 | }
49 | }
50 |
51 | object DefaultClock {
52 |
53 | def apply(sendHeartbeatTimeout: FiniteDuration,
54 | receiveHeartbeat: RandomTimer,
55 | schedulerService: ScheduledExecutorService = java.util.concurrent.Executors.newScheduledThreadPool(2),
56 | cancelMayInterruptIfRunning: Boolean = true): RaftClock = {
57 | new DefaultClock(sendHeartbeatTimeout, receiveHeartbeat, schedulerService, cancelMayInterruptIfRunning)
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/reactive/MultiSubscriberProcessor.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import com.typesafe.scalalogging.StrictLogging
3 | import org.reactivestreams.{Subscriber, Subscription}
4 |
5 | import scala.concurrent.ExecutionContext
6 |
7 | object MultiSubscriberProcessor {
8 |
9 | def apply[A](queueSize: Int, delayErrorsFromSubscribers: Boolean)(implicit execContext: ExecutionContext): MultiSubscriberProcessor[A] = {
10 | new MultiSubscriberProcessor[A] {
11 | override def maxQueueSize: Int = queueSize
12 | override def delayErrors: Boolean = delayErrorsFromSubscribers
13 | override implicit def ctxt: ExecutionContext = execContext
14 | }
15 | }
16 | }
17 |
18 | /**
19 | * A 'Processor' (both a subscriber and publisher) which can be subscribed to multiple publishers in order to collate and publish their values.
20 | *
21 | * {{{
22 | * val sub : MultiSubscriberProcessor[A] = ...
23 | * pub1.subscribe(sub)
24 | * pub2.subscribe(sub)
25 | * pub3.subscribe(sub)
26 | *
27 | * // the subscribers to 'sub' will receive the values published from pub1, pub2 and pub3
28 | * sub.subscribe(...)
29 | * }}}
30 | *
31 | * @param delayErrors if false, errors from any publishers will be propagated to subscribers to this MultiSubscriberProcessor
32 | */
33 | trait MultiSubscriberProcessor[A] extends AsyncPublisher[A] with Subscriber[A] with StrictLogging with AutoCloseable {
34 |
35 | /** @return true if we should suppress errors from any one subscription
36 | */
37 | def delayErrors: Boolean
38 |
39 | private var errorOpt: Option[Throwable] = None
40 |
41 | override def onSubscribe(s: Subscription): Unit = {
42 | logger.debug(s"onSubscribe($s)")
43 | s.request(Long.MaxValue)
44 | }
45 | override final def onNext(t: A): Unit = {
46 | logger.debug(s"onNext($t)")
47 | enqueueMessage(t)
48 | }
49 | override def onError(t: Throwable): Unit = {
50 | logger.debug(s"onError($t)")
51 | if (!delayErrors) {
52 | enqueueError(t)
53 | }
54 | errorOpt = errorOpt.orElse(Option(t))
55 | }
56 | override def onComplete(): Unit = {
57 | logger.debug(s"onComplete() - ignored")
58 | // ignore, as we may subscribe to something else
59 | }
60 |
61 | /**
62 | * Exposes a means to force complete immediately this subscriber.
63 | *
64 | * This is necessary as a multi-subscriber, by definition, can be reused and
65 | * handed to a publisher at any time, so this will explicitly complete and downstream
66 | * subscribers and thus release any resources used.
67 | */
68 | def complete(propagateErrors: Boolean = true) = {
69 | errorOpt match {
70 | case Some(err) if propagateErrors =>
71 | enqueueError(err)
72 | case _ =>
73 | enqueueComplete()
74 | }
75 | }
76 |
77 | /**
78 | * Marks this subscription in error. Similar to 'complete', this is required as this subscription
79 | * will be subscribed to ma
80 | *
81 | * @param exp
82 | */
83 | def inError(exp: Throwable) = {
84 | enqueueError(exp)
85 | }
86 |
87 | override def close() = {
88 | enqueueComplete()
89 | }
90 |
91 | }
92 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/reactive/ReactivePipe.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.{Publisher, Subscriber}
3 |
4 | import scala.concurrent.ExecutionContext
5 |
6 | /**
7 | * Represents an input/output pipe
8 | *
9 | * @param input
10 | * @param output
11 | * @param ev$1
12 | * @param ev$2
13 | * @tparam Sub
14 | * @tparam Pub
15 | */
16 | case class ReactivePipe[In, Out, Sub[_]: AsSubscriber, Pub[_]: AsPublisher](input: Sub[In], output: Pub[Out]) extends AutoCloseable {
17 |
18 | /**
19 | * Convenience method for 'subscribe', but which returns the subscriber instance:
20 | * {{{
21 | * val mySub = publisher.subscribeWith(new FooSubscriber) // mySub will be a FooSubscriber
22 | * }}}
23 | *
24 | * @param s the subscriber to return
25 | * @tparam S the subscriber type
26 | * @return the same subscriber
27 | */
28 | final def subscribeWith[S <: Subscriber[_ >: Out]](s: S): S = {
29 | implicitly[AsPublisher[Pub]].subscribeWith(output, s)
30 | }
31 |
32 | def outputPublisher: Publisher[Out] = AsPublisher[Pub].asPublisher(output)
33 | def inputSubscriber: Subscriber[In] = AsSubscriber[Sub].asSubscriber(input)
34 | override def close(): Unit = {
35 | input match {
36 | case closable: AutoCloseable => closable.close()
37 | case _ =>
38 | }
39 | output match {
40 | case closable: AutoCloseable => closable.close()
41 | case _ =>
42 | }
43 | }
44 | }
45 |
46 | object ReactivePipe {
47 |
48 | def multi[A](queueSize: Int, delayErrors: Boolean, minimumRequestedThreshold: Int = 10, subscriptionBatchSize: Int = 100)(
49 | implicit executionContext: ExecutionContext): ReactivePipe[A, A, Subscriber, Publisher] = {
50 | val feedAndSink: ReactivePipe[A, A, Subscriber, Publisher] =
51 | single(queueSize, minimumRequestedThreshold, subscriptionBatchSize)
52 | val multiSubscriber = MultiSubscriberProcessor[A](queueSize, delayErrors)
53 |
54 | // here the feedAndSink.input will be fed from multiple feeds
55 | multiSubscriber.subscribe(feedAndSink.input)
56 | new ReactivePipe[A, A, Subscriber, Publisher](multiSubscriber, feedAndSink.output)
57 | }
58 |
59 | def single[A](maxQueueSize: Int, minimumRequestedThreshold: Int, subscriptionBatchSize: Int)(
60 | implicit executionContext: ExecutionContext): ReactivePipe[A, A, Subscriber, Publisher] = {
61 | val feedAndSink = new Instance[A](maxQueueSize, minimumRequestedThreshold, subscriptionBatchSize)
62 | new ReactivePipe[A, A, Subscriber, Publisher](feedAndSink, feedAndSink)
63 | }
64 |
65 | class Instance[A](override val maxQueueSize: Int, override protected val minimumRequestedThreshold: Int, override protected val subscriptionBatchSize: Int)(
66 | override implicit val ctxt: ExecutionContext)
67 | extends AsyncPublisher[A] with SingleSubscriber[A] with AutoCloseable {
68 | require(minimumRequestedThreshold <= subscriptionBatchSize)
69 | require(minimumRequestedThreshold >= 0)
70 | override protected def doOnNext(message: A): Unit = enqueueMessage(message)
71 | override protected def doOnError(err: Throwable): Unit = enqueueError(err)
72 | override def doOnComplete(): Unit = enqueueComplete()
73 | override def close() = onComplete()
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/reactive/ReactiveTimerCallback.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import riff.raft.messages.{ReceiveHeartbeatTimeout, SendHeartbeatTimeout, TimerMessage}
3 | import riff.raft.timer.TimerCallback
4 |
5 | import scala.concurrent.ExecutionContext
6 |
7 | /**
8 | * An implementation of a [[TimerCallback]] which publishes TimerMessages
9 | *
10 | * @param ctxt the execution context used for each subscriber to consumer its messages
11 | */
12 | class ReactiveTimerCallback private (override val maxQueueSize: Int = 1000)(override implicit val ctxt: ExecutionContext)
13 | extends TimerCallback[TimerMessage] with AsyncPublisher[TimerMessage] {
14 |
15 | def onTimeout(msg: TimerMessage): TimerMessage = {
16 | enqueueMessage(msg)
17 | msg
18 | }
19 |
20 | override def onSendHeartbeatTimeout(): TimerMessage = {
21 | onTimeout(SendHeartbeatTimeout)
22 | }
23 | override def onReceiveHeartbeatTimeout(): TimerMessage = {
24 | onTimeout(ReceiveHeartbeatTimeout)
25 | }
26 |
27 | }
28 |
29 | object ReactiveTimerCallback {
30 |
31 | def apply()(implicit ctxt: ExecutionContext): ReactiveTimerCallback = {
32 | new ReactiveTimerCallback
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/reactive/ReplayPublisher.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 |
3 | /**
4 | * A publisher which will replay all elements from before any subscription begins
5 | *
6 | * @tparam A the type of data which is appended to the log (could just be a byte array, some union type, etc)
7 | */
8 | trait ReplayPublisher[A] extends AsyncPublisher[A] {
9 |
10 | private var values = List[A]()
11 |
12 | override final protected def enqueueMessages(messages: Iterable[A]): Unit = {
13 | values = messages.foldRight(values)(_ +: _)
14 | super.enqueueMessages(messages)
15 | }
16 |
17 | override final protected def enqueueMessage(message: A): Unit = {
18 | values = message +: values
19 | super.enqueueMessage(message)
20 | }
21 |
22 | /**
23 | * This is called before 'onSubscribe' is called
24 | * @param sub the subscription to add
25 | * @return the subscription
26 | */
27 | override protected def add(sub: AsyncSubscription[A]): AsyncSubscription[A] = {
28 | val retVal = super.add(sub)
29 |
30 | // at this point we can charge this subscription w/ all the values already received
31 | sub.enqueueAll(values.reverse)
32 |
33 | retVal
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/main/scala/riff/reactive/SingleSubscriber.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 |
3 | import org.reactivestreams.{Subscriber, Subscription}
4 |
5 | /**
6 | * Adds a means to accessing the Subscription when subscribed.
7 | *
8 | * The 'onError' will remove the subscription, so overriding classes should implement 'doOnError'
9 | * for error-handling behvior.
10 | *
11 | *
12 | */
13 | trait SingleSubscriber[A] extends Subscriber[A] {
14 |
15 | /** @return the number of elements requested from its subscription
16 | */
17 | protected def subscriptionBatchSize: Int
18 |
19 | /** @return the number of elements received of the 'subscriptionBatchSize', below which another 'request()' request is made
20 | */
21 | protected def minimumRequestedThreshold: Int
22 |
23 | private var receivedCounter = subscriptionBatchSize - minimumRequestedThreshold
24 |
25 | protected def doOnNext(message: A): Unit
26 |
27 | protected def doOnError(err: Throwable): Unit // = {}
28 |
29 | protected def doOnComplete(): Unit
30 |
31 | private var subscriptionOpt: Option[Subscription] = None
32 | override def onSubscribe(s: Subscription): Unit = {
33 | subscriptionOpt match {
34 | case Some(_) => s.cancel()
35 | case None =>
36 | if (subscriptionBatchSize > 0) {
37 | s.request(subscriptionBatchSize)
38 | }
39 | subscriptionOpt = Option(s)
40 | }
41 | }
42 |
43 | /** @param n the number of elements to request
44 | * @return true if there was a subscription from which to request, false otherwise
45 | */
46 | def request(n: Long) = {
47 | subscriptionOpt.fold(false) { s =>
48 | s.request(n)
49 | true
50 | }
51 | }
52 |
53 | def cancel() = subscriptionOpt.fold(false) { s =>
54 | s.cancel()
55 | subscriptionOpt = None
56 | true
57 | }
58 | override final def onNext(nextMsg: A): Unit = {
59 | if (nextMsg == null) {
60 | throw new NullPointerException
61 | }
62 |
63 | subscriptionOpt.foreach { s =>
64 | receivedCounter = receivedCounter - 1
65 | if (receivedCounter == 0) {
66 | receivedCounter = subscriptionBatchSize - minimumRequestedThreshold
67 | s.request(subscriptionBatchSize)
68 | }
69 | }
70 | doOnNext(nextMsg)
71 | }
72 | final override def onError(t: Throwable): Unit = {
73 | if (t == null) {
74 | throw new NullPointerException
75 | }
76 | doOnError(t)
77 | subscriptionOpt = None
78 | }
79 | override final def onComplete(): Unit = {
80 | subscriptionOpt = None
81 | doOnComplete()
82 | }
83 | }
84 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/resources/input.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 | echo "user:"
4 | read varname
5 | echo "Hello $varname"
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 | RIFF: %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
6 |
7 |
8 |
9 | riff.log
10 |
11 | RIFF: L%d{yyyy-MM-dd HH:mm:ss} [%thread] %-5level %logger{36} - %msg%n
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/RiffSpec.scala:
--------------------------------------------------------------------------------
1 | package riff
2 |
3 | import java.nio.file.Path
4 | import java.util.UUID
5 |
6 | import eie.io.LowPriorityIOImplicits
7 |
8 | class RiffSpec extends BaseSpec with LowPriorityIOImplicits {
9 |
10 | def nextTestDir(name: String) = {
11 | s"target/test/${name}-${UUID.randomUUID()}".asPath
12 | }
13 |
14 | def withDir[T](f: Path => T): T = {
15 |
16 | val name: String = getClass.getSimpleName
17 | val dir: Path = nextTestDir(name)
18 | if (dir.exists()) {
19 | dir.delete()
20 | }
21 | dir.mkDirs()
22 | try {
23 | f(dir)
24 | } finally {
25 | dir.delete()
26 | }
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/RiffThreadedSpec.scala:
--------------------------------------------------------------------------------
1 | package riff
2 | import java.util.concurrent.TimeUnit
3 |
4 | import org.scalatest.BeforeAndAfterAll
5 | import org.scalatest.concurrent.Eventually
6 | import riff.raft.timer.{RaftClock, RandomTimer}
7 | import riff.reactive.newContextWithThreadPrefix
8 |
9 | import scala.concurrent.ExecutionContextExecutorService
10 | import scala.concurrent.duration._
11 |
12 | class RiffThreadedSpec extends RiffSpec with BeforeAndAfterAll with Eventually {
13 |
14 | override implicit def testTimeout: FiniteDuration = 5.seconds
15 |
16 | protected def newExecCtxt(): ExecutionContextExecutorService = {
17 | newContextWithThreadPrefix(getClass.getSimpleName)
18 | }
19 |
20 | def withExecCtxt[T](f: ExecutionContextExecutorService => T): T = {
21 | val ctxt = newExecCtxt()
22 | try {
23 | f(ctxt)
24 | } finally {
25 | ctxt.shutdown()
26 | ctxt.awaitTermination(testTimeout.toMillis, TimeUnit.MILLISECONDS) shouldBe true
27 | }
28 | }
29 |
30 | def withClock[T](hbTimeout: FiniteDuration)(thunk: RaftClock => T): T = {
31 | val clock = RaftClock(hbTimeout, RandomTimer(hbTimeout + 200.millis, hbTimeout + 300.millis))
32 | try {
33 | thunk(clock)
34 | } finally {
35 | clock.close()
36 | }
37 | }
38 |
39 | }
40 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/raft/log/FileBasedRaftLogTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 |
3 | class FileBasedRaftLogTest extends RaftLogTCK {
4 |
5 | "RaftLog(file)" should {
6 | "error if given a file as a path" in {
7 | withDir { dir =>
8 | val file = dir.resolve("imAFile.txt").text = "hi"
9 | val exp = intercept[Exception] {
10 | RaftLog[String](file)
11 | }
12 | exp.getMessage should include("imAFile.txt is not a directory")
13 | }
14 | }
15 | "error if the directory doesn't exist" in {
16 | withDir { dir =>
17 | val doesntExist = dir.resolve("data").resolve("instance-1")
18 | val exp = intercept[Exception] {
19 | RaftLog[String](doesntExist)
20 | }
21 | exp.getMessage should include("is not a directory")
22 | }
23 | }
24 | "create the directory if it doesn't exist and 'createIfNotExists' is specified" in {
25 | withDir { dir =>
26 | val doesntExist = dir.resolve("data").resolve("instance-1")
27 | doesntExist.exists() shouldBe false
28 | val log = RaftLog[String](doesntExist, createIfNotExists = true)
29 | doesntExist.exists() shouldBe true
30 | log.append(LogCoords(4, 5), "hi")
31 | doesntExist.children should not be empty
32 | }
33 | }
34 | }
35 | "RaftLog.ForDir append" should {
36 | "remove old appended entries if asked to append an earlier entry with a greater term" in {
37 | withDir { dir =>
38 | val log = RaftLog[String](dir)
39 | log.logState shouldBe LogState.Empty
40 |
41 | log.append(LogCoords(4, 1), "a")
42 | log.append(LogCoords(4, 2), "b")
43 | log.append(LogCoords(4, 3), "c")
44 |
45 | log.termForIndex(1) shouldBe Some(4)
46 | log.termForIndex(2) shouldBe Some(4)
47 | log.termForIndex(3) shouldBe Some(4)
48 | log.dir.children should contain allElementsOf ((1 to 3).map(i => log.dir.resolve(s"$i.entry")))
49 | log.latestAppended() shouldBe LogCoords(4, 3)
50 |
51 | // call the method under test -- appending term 5 at index 2 should remove our previous entries
52 | log.append(LogCoords(5, 2), "replacing entry") shouldBe LogAppendResult(
53 | firstIndex = LogCoords(5, 2),
54 | lastIndex = LogCoords(5, 2),
55 | replacedIndices = Seq(LogCoords(4, 2), LogCoords(4, 3))
56 | )
57 | log.latestAppended() shouldBe LogCoords(5, 2)
58 | log.termForIndex(1) shouldBe Some(4)
59 | log.termForIndex(2) shouldBe Some(5)
60 | log.termForIndex(3) shouldBe None
61 | }
62 | }
63 | }
64 |
65 | override protected def withLog(testLogic: RaftLog[String] => Unit): Unit = {
66 | withDir { dir =>
67 | withClue("file based") {
68 | testLogic(RaftLog[String](dir))
69 | }
70 | }
71 |
72 | withDir { dir =>
73 | withClue("file based cached") {
74 | testLogic(RaftLog[String](dir).cached())
75 | }
76 | }
77 | }
78 | }
79 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/raft/node/NIOPersistentStateTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | class NIOPersistentStateTest extends PersistentStateTCK {
4 |
5 | def withPersistentState(test: PersistentState => Unit) = {
6 | withDir { dir =>
7 | val st8 = NIOPersistentState(dir, true)
8 | test(st8)
9 | }
10 | }
11 | }
12 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/raft/timer/RaftClockTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 | import scala.concurrent.duration._
3 |
4 | /**
5 | * If you have a custom timer (e.g. in monix, cats, fs2, akka, etc), you should just be able to extend the TCK
6 | * and plug in your implementation like here
7 | */
8 | class RaftClockTest extends RaftClockTCK {
9 | override def newTimer(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeat: RandomTimer): RaftClock = {
10 | RaftClock(sendHeartbeatTimeout, receiveHeartbeat)
11 | }
12 |
13 | override def assertAfter[T](time: FiniteDuration)(f: => T) = {
14 | Thread.sleep(time.toMillis)
15 | f
16 | }
17 | }
18 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/AsPublisherTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.Publisher
3 | import riff.RiffThreadedSpec
4 |
5 | class AsPublisherTest extends RiffThreadedSpec {
6 |
7 | import AsPublisher._
8 | import AsPublisher.syntax._
9 | "AsPublisher.takeWhile" should {
10 | "complete when the predicate completes" in {
11 | val integers: Publisher[Int] = FixedPublisher((0 to 10), false)
12 | val pub = integers.takeWhile(_ != 4)
13 |
14 | val listener = pub.subscribeWith(new TestListener[Int](100, 100))
15 | listener.received.toList shouldBe List(0, 1, 2, 3)
16 | listener.completed shouldBe true
17 | }
18 | }
19 | "AsPublisher.takeWhileIncludeLast" should {
20 | "complete when the predicate completes but include the first result which returned false" in {
21 | val integers: Publisher[Int] = FixedPublisher((0 to 10), false)
22 | val pub = integers.takeWhileIncludeLast(_ != 4)
23 |
24 | val listener = pub.subscribeWith(new TestListener[Int](100, 100))
25 | listener.received.toList shouldBe List(0, 1, 2, 3, 4)
26 | listener.completed shouldBe true
27 | }
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/AsyncPublisherVerificationTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import eie.io.Lazy
3 | import org.reactivestreams.Publisher
4 | import org.reactivestreams.tck.PublisherVerification
5 | import org.testng.annotations.AfterTest
6 |
7 | import scala.concurrent.ExecutionContext
8 |
9 | class AsyncPublisherVerificationTest extends PublisherVerification[Long](testEnv) {
10 |
11 | private val lazyCtxt = Lazy(newContextWithThreadPrefix(getClass.getSimpleName))
12 |
13 | implicit def execContext = lazyCtxt.value
14 |
15 | @AfterTest
16 | def afterAll(): Unit = {
17 | lazyCtxt.foreach(_.shutdown())
18 | }
19 | override def createPublisher(elements: Long): Publisher[Long] = {
20 | new AsyncPublisher[Long] {
21 |
22 | // use a larger queue as we enqueue chunks
23 | override def maxQueueSize: Int =
24 | if (elements > 10000) {
25 | 100000
26 | } else {
27 | 100
28 | }
29 | override implicit def ctxt: ExecutionContext = execContext
30 | override protected def add(sub: AsyncSubscription[Long]): AsyncSubscription[Long] = {
31 |
32 | val r = super.add(sub)
33 | if (elements > 10000) {
34 | execContext.execute(new Runnable {
35 | override def run() = {
36 |
37 | var i = elements
38 | while (i > 0) {
39 | val next = i.min(500).toInt
40 | enqueueMessages((0 until next).map(_.toLong))
41 | i = i - next
42 | Thread.`yield`()
43 | }
44 | enqueueComplete()
45 | }
46 | })
47 |
48 | } else {
49 | var i = elements
50 | while (i > 0) {
51 | val next = i.min(Int.MaxValue).toInt
52 | enqueueMessages((0 until next).map(_.toLong))
53 | i = i - next
54 | }
55 | enqueueComplete()
56 | }
57 | r
58 | }
59 | }
60 | }
61 | override def createFailedPublisher(): Publisher[Long] = {
62 | new AsyncPublisher[Long] {
63 | override protected def maxQueueSize: Int = 200
64 | override implicit def ctxt: ExecutionContext = execContext
65 | enqueueError(new Exception("kapow"))
66 | }
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/AsyncSubscriptionStateTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.{Subscriber, Subscription}
3 | import riff.RiffSpec
4 | import riff.reactive.AsyncSubscriptionState.Request
5 |
6 | class AsyncSubscriptionStateTest extends RiffSpec {
7 |
8 | "AsyncSubscriptionState.update" should {
9 | "process all the elements when the total queued is greater than the batch size" in {
10 | val batchSize = 4
11 | val lots: Vector[Long] = (0L to batchSize * 3).toVector
12 |
13 | var receivedCount = 0L
14 | val subscriber = new Subscriber[Long] {
15 | override def onSubscribe(s: Subscription): Unit = ???
16 | override def onNext(t: Long): Unit = {
17 | receivedCount = receivedCount + 1
18 | }
19 | override def onError(t: Throwable): Unit = ???
20 | override def onComplete(): Unit = ???
21 | }
22 |
23 | Given(s"A state where ${batchSize} is already requested and there are ${lots.size} elements")
24 | val state = AsyncSubscriptionState[Long](subscriber, batchSize, lots, false, false, batchSize = batchSize)
25 |
26 | val newState = state.update(Request(3))
27 | // we should only have 10 - 3 (or 7) elements left
28 | newState.totalRequested shouldBe 0
29 | receivedCount shouldBe 4 + 3
30 | newState.valueQueue.size shouldBe lots.size - 7
31 | }
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/CollectPublisherTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import riff.RiffThreadedSpec
3 |
4 | import scala.concurrent.ExecutionContext
5 |
6 | class CollectPublisherTest extends RiffThreadedSpec {
7 |
8 | "CollectPublisher" should {
9 | "filter based on the predicate while still honoring the total number requested" in {
10 | withExecCtxt { execCtxt =>
11 | Given("A filtered publisher on even integers")
12 | object Integers extends AsyncPublisher[Int] {
13 | override implicit def ctxt: ExecutionContext = execCtxt
14 | def push(values: Iterable[Int]) = {
15 | values.foreach(enqueueMessage)
16 | enqueueComplete()
17 | }
18 | override protected def maxQueueSize: Int = 200
19 | }
20 |
21 | val filtered = CollectPublisher(Integers) {
22 | case i if i % 2 == 0 => i + 100
23 | }
24 |
25 | When("a subscriber requests 2 elements")
26 | val listener = new TestListener[Int](0, 0)
27 | filtered.subscribe(listener)
28 |
29 | Integers.push(0 until 10)
30 |
31 | listener.received shouldBe (empty)
32 | listener.request(2)
33 |
34 | Then("the listener should receive those two elements")
35 | eventually {
36 | listener.received.toList should contain only (100, 102)
37 | }
38 |
39 | When("the listener requests the remaining 3 elements")
40 | listener.request(3)
41 |
42 | When("the listener should receive the remaining 3 positive ints")
43 | eventually {
44 | listener.received.toList should contain only (100, 102, 104, 106, 108)
45 | }
46 |
47 | When("the listener requests the next element")
48 | listener.request(1)
49 | Then("it should be told the elements are complete")
50 | eventually {
51 | listener.completed shouldBe true
52 | }
53 | }
54 | }
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/FixedPublisher.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
3 |
4 | import org.reactivestreams.{Publisher, Subscriber, Subscription}
5 |
6 | object FixedPublisher {
7 |
8 | def apply[T](first: T, theRest: T*): FixedPublisher[T] = {
9 | new FixedPublisher(first +: theRest.toList, true)
10 | }
11 | }
12 | case class FixedPublisher[T](values: Iterable[T], allowOnComplete: Boolean) extends Publisher[T] {
13 | override def subscribe(s: Subscriber[_ >: T]): Unit = {
14 | s.onSubscribe(new Subscription {
15 | var remaining = values.toList
16 | var pending = new AtomicLong(0)
17 | var inProcess = new AtomicBoolean(false)
18 |
19 | // if we're not careful we can form an infinite loop here if the subscriber calls 'request' from its onNext
20 | override def request(n: Long): Unit = {
21 | if (inProcess.compareAndSet(false, true)) {
22 | feed(n)
23 | inProcess.compareAndSet(true, false)
24 | if (pending.get > 0 && remaining.nonEmpty) {
25 | request(pending.get)
26 | }
27 | } else {
28 | pending.addAndGet(n)
29 | }
30 | }
31 |
32 | def feed(n: Long): Unit = {
33 | val (taken, left) = if (n >= Int.MaxValue) {
34 | (remaining, Nil)
35 | } else {
36 | remaining.splitAt(n.toInt)
37 | }
38 | remaining = left
39 | taken.foreach(s.onNext)
40 | if (allowOnComplete && left.isEmpty) {
41 | s.onComplete()
42 | }
43 | }
44 | override def cancel(): Unit = {
45 | remaining = Nil
46 | }
47 | })
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/FixedPublisherTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import riff.BaseSpec
3 |
4 | class FixedPublisherTest extends BaseSpec {
5 |
6 | "FixedPublisher" should {
7 | "publish the elements" in {
8 | val listener = new TestListener[Int]()
9 | FixedPublisher(10 to 13, true).subscribe(listener)
10 | listener.request(1)
11 | listener.received.size shouldBe 1
12 | listener.completed shouldBe false
13 |
14 | listener.request(10)
15 | listener.received.size shouldBe 4
16 | listener.completed shouldBe true
17 |
18 | }
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/PublishersTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import riff.RiffSpec
3 |
4 | class PublishersTest extends RiffSpec {
5 |
6 | "Publishers.NoOpSubscription" should {
7 | "not error" in {
8 | Publishers.NoOpSubscription.request(123)
9 | Publishers.NoOpSubscription.cancel()
10 | }
11 | }
12 | "Publishers.Completed" should {
13 | "notify subscribers that there is no data upon subscription" in {
14 | val sub = new TestListener[String]
15 | Publishers.Completed.subscribe(sub)
16 | sub.completed shouldBe true
17 | }
18 | }
19 | "Publishers.InError" should {
20 | "notify subscribers that there is an error" in {
21 | object Bang extends Exception
22 | val sub = new TestListener[String]
23 | Publishers.InError(Bang).subscribe(sub)
24 | sub.errors.toList shouldBe List(Bang)
25 | }
26 | }
27 | "Publishers.Fixed" should {
28 | "notify subscribers of the inputs" in {
29 |
30 | val sub = new TestListener[Int](0, 0)
31 | Publishers.Fixed(List(1, 2, 3, 4)).subscribe(sub)
32 | sub.received should be(empty)
33 | sub.request(2)
34 | sub.received should contain only (1, 2)
35 | sub.completed shouldBe false
36 | sub.request(1)
37 | sub.received should contain only (1, 2, 3)
38 | sub.completed shouldBe false
39 |
40 | sub.request(1)
41 | sub.received should contain only (1, 2, 3, 4)
42 | sub.completed shouldBe true
43 | }
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/ReplayPublisherTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.scalatest.BeforeAndAfterAll
3 | import org.scalatest.concurrent.Eventually
4 | import riff.RiffSpec
5 |
6 | import scala.concurrent.ExecutionContext
7 |
8 | class ReplayPublisherTest extends RiffSpec with BeforeAndAfterAll with Eventually {
9 |
10 | private val execContext = newContext()
11 |
12 | override def afterAll(): Unit = {
13 | execContext.shutdown()
14 | }
15 | "ReplayPublisher" should {
16 | "send elements pushed before a subscription was received" in {
17 |
18 | object TestPublisher extends ReplayPublisher[String] {
19 | override protected def maxQueueSize: Int = 200
20 | override implicit def ctxt: ExecutionContext = execContext
21 | def push(value: String) = enqueueMessage(value)
22 | def complete() = enqueueComplete()
23 | }
24 |
25 | TestPublisher.push("first")
26 | TestPublisher.push("second")
27 | TestPublisher.push("third")
28 |
29 | val listener = TestPublisher.subscribeWith(new TestListener[String](0, 0))
30 | listener.completed shouldBe false
31 | listener.received shouldBe empty
32 |
33 | listener.request(2)
34 | eventually {
35 | listener.received.toList shouldBe List("first", "second")
36 | }
37 |
38 | TestPublisher.push("last")
39 | TestPublisher.complete()
40 |
41 | listener.request(6)
42 | eventually {
43 | listener.received.toList shouldBe List("first", "second", "third", "last")
44 | }
45 | listener.completed shouldBe true
46 | }
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/TestListener.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 |
3 | import scala.collection.mutable.ListBuffer
4 |
5 | class TestListener[A](override val minimumRequestedThreshold: Int = 0, override val subscriptionBatchSize: Int = 0) extends SingleSubscriber[A] {
6 | val received = ListBuffer[A]()
7 | val errors = ListBuffer[Throwable]()
8 | var completed = false
9 | override def doOnNext(t: A): Unit = {
10 | received += t
11 | }
12 | override def doOnError(t: Throwable): Unit = {
13 | errors += t
14 | }
15 | override def doOnComplete(): Unit = {
16 | completed = true
17 | }
18 | }
19 |
--------------------------------------------------------------------------------
/riff-core/jvm/src/test/scala/riff/reactive/reactive.scala:
--------------------------------------------------------------------------------
1 | package riff
2 | import java.util.concurrent.atomic.AtomicInteger
3 | import java.util.concurrent.{ExecutorService, Executors, ThreadFactory}
4 |
5 | import org.reactivestreams.tck.TestEnvironment
6 |
7 | import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
8 |
9 | package object reactive {
10 |
11 | val DEFAULT_TIMEOUT_MILLIS = 50L
12 | val DEFAULT_NO_SIGNALS_TIMEOUT_MILLIS: Long = DEFAULT_TIMEOUT_MILLIS
13 | val PUBLISHER_REFERENCE_CLEANUP_TIMEOUT_MILLIS = 250L
14 |
15 | val testEnv = new TestEnvironment(DEFAULT_TIMEOUT_MILLIS, DEFAULT_TIMEOUT_MILLIS)
16 |
17 | val ids = new AtomicInteger(0)
18 |
19 | private def setName(prefix: String)(t: Thread) = {
20 | t.setName(s"$prefix${ids.incrementAndGet()}")
21 | t
22 | }
23 |
24 | def newContextWithThreadPrefix(threadPrefix: String): ExecutionContextExecutorService = {
25 | newContext(setName(threadPrefix))
26 | }
27 |
28 | def newContext(init: Thread => Thread = setName(getClass.getSimpleName)): ExecutionContextExecutorService = {
29 | val factory = new ThreadFactory {
30 |
31 | override def newThread(r: Runnable): Thread = {
32 | init(new Thread(r))
33 | }
34 | }
35 | val es: ExecutorService = Executors.newCachedThreadPool(factory)
36 | ExecutionContext.fromExecutorService(es)
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/AppendStatus.scala:
--------------------------------------------------------------------------------
1 | package riff.raft
2 | import riff.raft.log.{LogAppendSuccess, LogCoords}
3 | import riff.raft.messages.AppendEntriesResponse
4 |
5 | /**
6 | * Represents the current state of the cluster following an append
7 | *
8 | * @param logCoords the coords of the entry appended
9 | * @param appended a map of the cluster ids to a flag indicating whether or not the node has appended the entry
10 | * @param appendedCoords the log coords which have been appended on the leader
11 | * @param clusterSize the size of the cluster
12 | * @param committed the coordinates of committed entries on the leader after quarum has ack'd
13 | * @param errorAfterAppend A means of letting observers know that a log append error has occurred, perhaps due to a leader change
14 | */
15 | final case class AppendStatus(leaderAppendResult: LogAppendSuccess,
16 | appended: Map[NodeId, AppendEntriesResponse],
17 | appendedCoords: Set[LogCoords],
18 | clusterSize: Int,
19 | committed: Set[LogCoords],
20 | errorAfterAppend: Option[Exception] = None) {
21 |
22 | def allCommitted: Boolean = committed == appendedCoords
23 |
24 | override def toString: String = {
25 |
26 | val responses = appended.map {
27 | case (name, resp) => s"\t$name : $resp"
28 | }
29 |
30 | s"""AppendStatus(
31 | | leaderAppendResult = $leaderAppendResult,
32 | | clusterSize = $clusterSize,
33 | | ${appendedCoords.size} appendedCoords = $appendedCoords,
34 | | ${appended.size} appended: ${responses.mkString("(\n", ",\n", "),")}
35 | | ${committed.size} committed = ${committed.mkString("[", ",", "]")})
36 | | errorAfterAppend = $errorAfterAppend""".stripMargin
37 | }
38 |
39 | /** Meant to signal that we have all that we expect to -- acks from EVERY node in the cluster and the entries are committed.
40 | *
41 | * If a node is down, OR A LEADER CHANGE HAPPENS, CAUSING NOT ALL NODES TO ACK AN APPEND, then this may never be true.
42 | *
43 | * In the case, however, that not all nodes respond, we will notice the 'errorAfterAppend' field become set
44 | *
45 | * @return true if we've received all the messages expected
46 | */
47 | def isComplete: Boolean = {
48 | numberOfPeersContainingCommittedIndex == clusterSize && allCommitted
49 | }
50 |
51 | def numberOfPeersContainingCommittedIndex: Int = appended.values.count { resp => //
52 | resp.success && appendedCoords.contains(resp.coords)
53 | }
54 |
55 | def withResult(from: NodeId, response: AppendEntriesResponse): AppendStatus = {
56 | copy(appended = appended.updated(from, response))
57 | }
58 |
59 | def withCommit(coords: LogCoords): AppendStatus = {
60 | require(appendedCoords.contains(coords), "Attempt to add committed coors for entries we've not appended")
61 | copy(committed = committed + coords)
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/RaftClient.scala:
--------------------------------------------------------------------------------
1 | package riff.raft
2 | import scala.reflect.ClassTag
3 |
4 | /**
5 | * The external use-cases for a node in the Raft protocol are:
6 | *
7 | * 1) adding/removing nodes
8 | * 2) appending data to the cluster, which should return either a new (pending) commit coords, a redirect, or a 'no known leader'
9 | * 3) subscribing to lifecycle events, which should return a Publisher which lets subscribers know about cluster and leadership changes
10 | * 4) subscribing to a commit log, which should return a Publisher which lets subscribers know when logs are appended or committed
11 | *
12 | * All of these algebras don't necessarily have to be in the same trait
13 | */
14 | // https://softwaremill.com/free-tagless-compared-how-not-to-commit-to-monad-too-early/
15 | trait RaftClient[F[_], A] {
16 |
17 | /**
18 | * Convenience method for appending a var-arg set of data
19 | *
20 | * @param data the first element to append
21 | * @param theRest the remaining elements
22 | * @param classTag
23 | * @return an F[AppendStatus] representing the append results
24 | */
25 | final def append(data: A, theRest: A*)(implicit classTag: ClassTag[A]): F[AppendStatus] =
26 | append(data +: theRest.toArray)
27 |
28 | /**
29 | * This is a client's view of a raft cluster, which simply wants to write some data.
30 | *
31 | * It should return some kind of observable result F[AppendStatus] which can represent an updated status as each member
32 | * of the cluster replies to the append. F[AppendStatus] should also be able to represent a failure if the node represented
33 | * by this client is not the leader or otherwise can't support the append.
34 | *
35 | * This call may also trigger a [[AppendOccurredOnDisconnectedLeader]] if the append occurs on a leader which can't
36 | * replicate the data and is subsequently replaced by another leader in another election.
37 | *
38 | * @param data the data to write
39 | * @return an observable of the append results as they are appended/co
40 | */
41 | def append(data: Array[A]): F[AppendStatus]
42 | }
43 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/client/StateUpdateMsg.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.client
2 |
3 | import riff.raft.NodeId
4 | import riff.raft.log.{LogAppendResult, LogCoords}
5 | import riff.raft.messages.AppendEntriesResponse
6 |
7 | /**
8 | * Represents the fixed inputs to the [[SingleAppendFSM]] in order to support a feed of [[riff.raft.AppendStatus]] messages
9 | */
10 | sealed trait StateUpdateMsg
11 |
12 | object StateUpdateMsg {
13 |
14 | /** The log append which we know to be a result of inserting the data for which we want to observe
15 | *
16 | * @param appendResult the log append result of appending some new data
17 | * @return a StateUpdateMsg
18 | */
19 | def initialAppend(appendResult: LogAppendResult): InitialLogAppend = InitialLogAppend(appendResult)
20 |
21 | /** Create a message from a log commit result (any log commit result).
22 | *
23 | * We need this not onto to update our [[riff.raft.AppendStatus]] messages, but also to know if we should complete w/o having
24 | * had all the responses.
25 | *
26 | * We need to consider this scenario:
27 | *
28 | * 1) we're the leader of a 5 node cluster
29 | * 2) one of the followers goes down
30 | * 3) we accept an append and eventually commit it when ack'd by the other nodes (but obviously not by the down node)
31 | * 4) another node becomes leader
32 | * 5) the down node comes up
33 | * !!) the "client" observing our updates will never get an 'onComplete', because the down node will never response to
34 | * us, the old leader
35 | *
36 | * In this situation, we can notice when our log is committed, and thereby end/complete the Observable[AppendStatus]
37 | *
38 | * @param committed the committed log result
39 | * @return a StateUpdateMsg
40 | */
41 | def logCommit(committed: LogCoords): EntryCommitted = EntryCommitted(committed)
42 |
43 | /** Create a message from a log appendresult (any log append result) - used to determine if our data is overwritten
44 | * by a new leader
45 | *
46 | * @param committed the append log result
47 | * @return a StateUpdateMsg
48 | */
49 | def logAppend(appendResult: LogAppendResult): LogAppend = LogAppend(appendResult)
50 |
51 | /** Create a message from a node's response message to this node
52 | *
53 | * @param responseFrom the sending node ID
54 | * @param appendResponse the response
55 | * @return a StateUpdateMsg
56 | */
57 | def responseFromNode(responseFrom: NodeId, appendResponse: AppendEntriesResponse) = NodeResponded(responseFrom, appendResponse)
58 | }
59 |
60 | case class EntryCommitted(committed: LogCoords) extends StateUpdateMsg
61 | case class NodeResponded(responseFrom: NodeId, appendResponse: AppendEntriesResponse) extends StateUpdateMsg
62 | case class InitialLogAppend(appendResult: LogAppendResult) extends StateUpdateMsg
63 | case class LogAppend(appendResult: LogAppendResult) extends StateUpdateMsg
64 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/exceptions.scala:
--------------------------------------------------------------------------------
1 | package riff.raft
2 |
3 | import riff.raft.log.{LogAppendSuccess, LogCoords}
4 |
5 | trait LogError extends Exception
6 |
7 | final case class LogAppendException[T](coords: LogCoords, data: T, err: Throwable)
8 | extends Exception(s"Error appending ${coords} : $data", err) with LogError
9 | final case class LogCommitException(coords: LogCoords, err: Throwable)
10 | extends Exception(s"Error committing ${coords}", err) with LogError
11 | final case class AttemptToCommitMissingIndex(attemptedIndex: LogIndex)
12 | extends Exception(s"couldn't find the term for $attemptedIndex") with LogError
13 | final case class AttemptToOverwriteACommittedIndex(attemptedLogIndex: LogIndex, latestCommittedIndex: LogIndex)
14 | extends Exception(s"Attempt to append $attemptedLogIndex when the latest committed index is $latestCommittedIndex") with LogError
15 |
16 |
17 | trait ClientError extends Exception
18 | final case class AppendOccurredOnDisconnectedLeader(originalAppend : LogAppendSuccess, newAppend : LogAppendSuccess) extends Exception(
19 | s"$originalAppend was appended but not committed by a disconnected leader and has later been replaced by $newAppend") with ClientError
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/BaseLog.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft._
3 |
4 | import scala.collection.immutable
5 |
6 | private[log] abstract class BaseLog[T] extends RaftLog[T] { // with LazyLogging {
7 |
8 | /**
9 | * we can get in this state if we've been leader and accepted some append commands from a client,
10 | * only to then discover there was a leader election and we were voted out, in which case we may
11 | * have extra, invalid uncommitted entries
12 | *
13 | * @param coordsOfTheFirstNewEntryToAppend the latest append coords
14 | * @return the indices to remove
15 | */
16 | protected def checkForOverwrite(firstIndex : LogIndex, firstTerm: Term): Either[LogAppendResult, immutable.Seq[LogCoords]] = {
17 | val latest: LogCoords = latestAppended()
18 |
19 | // if our latest index is the same or after the index to append, that implies
20 | // we were the leader who accepted an append while another node who didn't have that entry
21 | // won an election, and is now appending
22 | if (latest.index >= firstIndex) {
23 |
24 | // ... and if that's the case, then the term of that log entry should be > than the term for our entry
25 | if (firstTerm <= latest.term) {
26 | Left(AttemptToAppendLogEntryAtEarlierTerm(LogCoords(firstTerm, firstIndex), latest))
27 | } else {
28 | Right((firstIndex to latest.index).flatMap(coordsForIndex))
29 | }
30 | } else {
31 | // the coords are after our term
32 | if (firstTerm < latest.term) {
33 | Left(AttemptToAppendLogEntryAtEarlierTerm(LogCoords(firstTerm, firstIndex), latest))
34 | } else if (firstIndex != latest.index + 1) {
35 | Left(AttemptToSkipLogEntry(LogCoords(firstTerm, firstIndex), latest.index + 1))
36 | } else {
37 | Right(Nil)
38 | }
39 | }
40 | }
41 |
42 | /**
43 | * hook for subclasses after having determined which entries need to be committed.
44 | *
45 | * @param index the index to commit
46 | * @param entriesToCommit the entries determined which need to be committed
47 | */
48 | protected def doCommit(index: LogIndex, entriesToCommit: immutable.IndexedSeq[LogCoords]): Unit
49 |
50 | override final def commit(index: LogIndex): Seq[LogCoords] = {
51 | val previous = latestCommit()
52 | if (previous < index) {
53 | val committed: immutable.IndexedSeq[LogCoords] = ((previous + 1) to index).map { i =>
54 | val term = termForIndex(i).getOrElse(throw AttemptToCommitMissingIndex(i))
55 | LogCoords(term, i)
56 | }
57 |
58 | doCommit(index, committed)
59 |
60 | committed
61 | } else {
62 | Nil
63 | }
64 | }
65 |
66 | protected def assertCommit(logIndex: LogIndex) = {
67 | val kermit = latestCommit
68 | if (kermit >= logIndex) {
69 | throw AttemptToOverwriteACommittedIndex(logIndex, kermit)
70 | }
71 | kermit
72 | }
73 | }
74 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/CachingLog.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.LogIndex
3 |
4 | /**
5 | * Wraps a log by keeping in memory the latest commit/append index
6 | *
7 | * @param underlying the underlying log
8 | * @tparam A
9 | */
10 | case class CachingLog[A](override val underlying: RaftLog[A]) extends DelegateLog[A] {
11 | private var latestAppendCache: Option[LogCoords] = None
12 | private var latestCommitCache: Option[LogIndex] = None
13 | override def appendAll(firstIndex: LogIndex, data: Array[LogEntry[A]]): LogAppendResult = {
14 | latestAppendCache = None
15 | underlying.appendAll(firstIndex, data)
16 | }
17 | override def latestCommit(): LogIndex = {
18 | latestCommitCache.getOrElse {
19 | val value = underlying.latestCommit()
20 | latestCommitCache = Option(value)
21 | value
22 | }
23 | }
24 | override def latestAppended(): LogCoords = {
25 | latestAppendCache.getOrElse {
26 | val value = underlying.latestAppended
27 | latestAppendCache = Option(value)
28 | value
29 | }
30 | }
31 | override def commit(index: LogIndex): Seq[LogCoords] = {
32 | latestCommitCache = None
33 | underlying.commit(index)
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/DelegateLog.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 |
3 | import riff.raft.{LogIndex, Term}
4 |
5 | trait DelegateLog[A] extends RaftLog[A] {
6 | def underlying: RaftLog[A]
7 | override def appendAll(fromIndex: LogIndex, data: Array[LogEntry[A]]): LogAppendResult = underlying.appendAll(fromIndex, data)
8 | override def latestCommit(): LogIndex = underlying.latestCommit()
9 | override def termForIndex(index: LogIndex): Option[Term] = underlying.termForIndex(index)
10 | override def latestAppended(): LogCoords = underlying.latestAppended()
11 | override def commit(index: LogIndex): Seq[LogCoords] = underlying.commit(index)
12 | override def entryForIndex(index: LogIndex): Option[LogEntry[A]] = underlying.entryForIndex(index)
13 | }
14 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/InMemory.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.{LogIndex, Term}
3 |
4 | import scala.collection.immutable
5 |
6 | /**
7 | * An in-memory persistent log. Not a great idea for production, but nifty for testing
8 | *
9 | * @tparam T
10 | */
11 | class InMemory[T]() extends BaseLog[T] {
12 | private var mostRecentFirstEntries = List[(LogCoords, T)]()
13 | private var lastCommitted = 0
14 |
15 | override protected def doCommit(index: LogIndex, entriesToCommit: immutable.IndexedSeq[LogCoords]): Unit = {
16 | require(lastCommitted < index, s"doCommit called w/ $index when lastCommitted is $lastCommitted")
17 | lastCommitted = index
18 | }
19 |
20 | override def appendAll(logIndex: LogIndex, data: Array[LogEntry[T]]): LogAppendResult = {
21 | require(logIndex > 0, s"log indices should begin at 1: $logIndex")
22 | if (data.isEmpty) {
23 | LogAppendResult(LogCoords.Empty, LogCoords.Empty)
24 | } else {
25 | doAppendAll(logIndex, data.head.term, data)
26 | }
27 | }
28 |
29 | private def doAppendAll(logIndex: LogIndex, firstTerm : Term, data: Array[LogEntry[T]]): LogAppendResult = {
30 | assertCommit(logIndex)
31 |
32 | checkForOverwrite(logIndex, firstTerm) match {
33 | case Left(err) => err
34 | case Right(indicesToDelete) =>
35 | mostRecentFirstEntries = indicesToDelete.foldLeft(mostRecentFirstEntries) {
36 | case (list, i) => list.dropWhile(_._1.index >= i.index)
37 | }
38 | val newEntries: Array[(LogCoords, T)] = data.zipWithIndex.map {
39 | case (LogEntry(term, e), i) => LogCoords(term = term, index = logIndex + i) -> e
40 | }.reverse
41 |
42 | mostRecentFirstEntries match {
43 | case (head, _) :: _ =>
44 | require(logIndex == head.index + 1)
45 | mostRecentFirstEntries = newEntries.toList ++ mostRecentFirstEntries
46 | case tail =>
47 | require(logIndex == 1)
48 | mostRecentFirstEntries = newEntries.toList ++ tail
49 | }
50 | LogAppendResult(newEntries.last._1, newEntries.head._1, indicesToDelete)
51 | }
52 | }
53 |
54 | override def latestCommit(): LogIndex = {
55 | lastCommitted
56 | }
57 |
58 | override def entryForIndex(index: LogIndex) = {
59 | termForIndex(index).flatMap { term =>
60 | mostRecentFirstEntries.collectFirst {
61 | case (coords, value) if coords.term == term && index == coords.index => LogEntry(term, value)
62 | }
63 | }
64 | }
65 | override def termForIndex(index: LogIndex): Option[Term] = {
66 | mostRecentFirstEntries.collectFirst {
67 | case (LogCoords(term, `index`), _) => term
68 | }
69 | }
70 | override def latestAppended(): LogCoords = mostRecentFirstEntries.headOption.map(_._1).getOrElse(LogCoords.Empty)
71 |
72 | override def contains(entry: LogCoords): Boolean = mostRecentFirstEntries.exists(_._1 == entry)
73 | }
74 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/LogAppendResult.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.{LogIndex, NodeId, Term}
3 | import riff.raft.messages.AppendEntriesResponse
4 |
5 | import scala.util.control.NoStackTrace
6 |
7 | sealed trait LogAppendResult
8 |
9 | object LogAppendResult {
10 |
11 | def apply(firstIndex: LogCoords, lastIndex: LogCoords, replacedIndices: Seq[LogCoords] = Nil): LogAppendSuccess = {
12 | LogAppendSuccess(firstIndex, lastIndex, replacedIndices)
13 | }
14 | }
15 |
16 | /**
17 | * Represents the return type of a file-based raft log
18 | *
19 | * @param firstIndex the first index written
20 | * @param lastIndex the last index written
21 | * @param replacedLogCoords in the case where a disconnected leader had accepted commits, these are the coordinates of replaced entries which had been appended (but not committed) from an out-of-date leader
22 | */
23 | final case class LogAppendSuccess(firstIndex: LogCoords, lastIndex: LogCoords, replacedLogCoords: Seq[LogCoords] = Nil)
24 | extends LogAppendResult {
25 | require(firstIndex.term == lastIndex.term, s"appended result w/ ${firstIndex} to ${lastIndex}")
26 | def numIndices = lastIndex.index - firstIndex.index + 1
27 |
28 | def appendedCoords: Set[LogCoords] = {
29 | val term = firstIndex.term
30 | (firstIndex.index to lastIndex.index).map { logIndex =>
31 | LogCoords(term, logIndex)
32 | }.toSet
33 | }
34 |
35 | def contains(response: AppendEntriesResponse): Boolean = {
36 | response.term == firstIndex.term && (response.matchIndex >= firstIndex.index && response.matchIndex <= lastIndex.index)
37 | }
38 | }
39 |
40 | final case class AttemptToSkipLogEntry(attemptedLogEntry: LogCoords, expectedNextIndex: LogIndex)
41 | extends Exception(
42 | s"Attempt to skip a log entry by appending ${attemptedLogEntry.index} w/ term ${attemptedLogEntry.term} when the next expected entry should've been $expectedNextIndex")
43 | with LogAppendResult with NoStackTrace
44 | final case class AttemptToAppendLogEntryAtEarlierTerm(attemptedEntry: LogCoords, latestLogEntryAppended: LogCoords)
45 | extends Exception(
46 | s"An attempt to append ${attemptedEntry.index} w/ term ${attemptedEntry.term} when our latest entry was $latestLogEntryAppended. If an election took place after we were the leader, the term should've been incremented")
47 | with LogAppendResult with NoStackTrace
48 |
49 | final case class NotTheLeaderException(attemptedNodeId: NodeId, term: Term, leadIdOpt: Option[NodeId])
50 | extends Exception(
51 | s"Attempt to append to node '${attemptedNodeId}' in term ${term}${leadIdOpt.fold("")(name =>
52 | s". The leader is ${name}")}"
53 | ) with LogAppendResult with NoStackTrace
54 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/LogCoords.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.{LogIndex, Term}
3 |
4 | /**
5 | * Represents the coords of a log entry
6 | *
7 | * @param term
8 | * @param index
9 | */
10 | final case class LogCoords(term: Term, index: LogIndex) {
11 | require(term >= 0)
12 | require(index >= 0)
13 | override def toString = s"LogCoords(term=$term, index=$index)"
14 | def asKey = s"${term}:${index}"
15 | }
16 |
17 | object LogCoords {
18 | val Empty = LogCoords(0, 0)
19 |
20 | private val LatestAppended = """([0-9]+):([0-9]+)""".r
21 |
22 | object FromKey {
23 | def unapply(key: String) = {
24 | key match {
25 | case LatestAppended(t, i) => Some(LogCoords(t.toInt, i.toInt))
26 | case _ => None
27 | }
28 | }
29 | }
30 | }
31 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/LogEntry.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.Term
3 |
4 | final case class LogEntry[T](term: Term, data: T)
5 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/LogState.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.{LogIndex, Term}
3 |
4 | final case class LogState(commitIndex: LogIndex, latestTerm: Term, latestIndex: LogIndex)
5 |
6 | object LogState {
7 |
8 | val Empty = LogState(0, 0, 0)
9 | }
10 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/log/StateMachineLog.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.raft.LogIndex
3 |
4 | /**
5 | * Invokes a function when log entries are committed
6 | *
7 | * @param underlying the wrapped RaftLog
8 | * @param onCommitted the side-effectful state-machine function
9 | * @tparam A
10 | */
11 | class StateMachineLog[A](override val underlying: RaftLog[A], onCommitted: LogEntry[A] => Unit) extends DelegateLog[A] {
12 |
13 | override def commit(index: LogIndex): Seq[LogCoords] = {
14 | val committed = super.commit(index)
15 |
16 | if (committed.nonEmpty) {
17 | committed.foreach {
18 | case LogCoords(_, index) =>
19 | entryForIndex(index).foreach(onCommitted)
20 | }
21 | }
22 |
23 | committed
24 | }
25 | }
26 |
27 | object StateMachineLog {
28 | def apply[A](underlying: RaftLog[A])(applyToStateMachine: LogEntry[A] => Unit): StateMachineLog[A] = new StateMachineLog[A](underlying, applyToStateMachine)
29 | }
30 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/CandidateState.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | import riff.raft.{NodeId, Term, isMajority}
4 | import riff.raft.messages.RequestVoteResponse
5 |
6 | /**
7 | * Contains the functions for when a node becomes a candidate due to a heartbeat timeout
8 | */
9 | case class CandidateState(term: Term, votesFor: Set[NodeId], votesAgainst: Set[NodeId], clusterSize: Int) {
10 |
11 | def canBecomeLeader = isMajority(votesFor.size, clusterSize)
12 |
13 | def update(from: NodeId, reply: RequestVoteResponse): CandidateState = {
14 | if (reply.term == term) {
15 | if (reply.granted) {
16 | copy(votesFor = votesFor + from)
17 | } else {
18 | copy(votesAgainst = votesAgainst + from)
19 | }
20 | } else {
21 | this
22 | }
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/LeadersClusterView.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import riff.raft.{LogIndex, NodeId}
3 | import riff.raft.log.LogCoords
4 | import riff.raft.messages.AppendEntriesResponse
5 |
6 | import scala.collection.immutable
7 |
8 | /**
9 | * Keeps track of the leader's ephemeral view of the cluster
10 | *
11 | * @param initialPeersByKey the known cluster state
12 | * @tparam NodeId the type representing this peer node. Typically just a String identifier,
13 | * though could be something more meaningful/useful, like a websocket. Just so long as it provides a meaningful hashCode/equals
14 | */
15 | private[node] class LeadersClusterView(cluster: RaftCluster) {
16 | private var peersByKey = Map[NodeId, Peer]()
17 |
18 | def eligibleNodesForPreviousEntry(previous: LogCoords): immutable.Iterable[NodeId] = {
19 | toMap.collect {
20 | case (key, peer) if peer.matchIndex == previous.index => key
21 | }
22 | }
23 |
24 | /** @param index
25 | * @return the number of nodes which have AT LEAST the same match index
26 | */
27 | def matchIndexCount(index: LogIndex): Int = {
28 | cluster.peers.count { id => //
29 | peersByKey.get(id).map(_.matchIndex).exists(_ >= index)
30 | }
31 | }
32 |
33 | def toMap(): Map[NodeId, Peer] = cluster.peers.foldLeft(Map[NodeId, Peer]()) {
34 | case (map, id) => map.updated(id, peersByKey.getOrElse(id, Peer.Empty))
35 | }
36 |
37 | def stateForPeer(peer: NodeId): Option[Peer] =
38 | if (cluster.contains(peer)) {
39 | peersByKey.get(peer).orElse(Option(Peer.Empty))
40 | } else {
41 | None
42 | }
43 |
44 | def update(node: NodeId, response: AppendEntriesResponse): Option[Peer] = {
45 | if (!cluster.contains(node)) {
46 | peersByKey = peersByKey - node
47 | None
48 | } else {
49 | val oldPeer = peersByKey.getOrElse(node, Peer.Empty)
50 | val newPeer = if (response.success) {
51 | oldPeer.setMatchIndex(response.matchIndex)
52 | } else {
53 | val newNextIndex = oldPeer.nextIndex - 1
54 | if (newNextIndex > 0) {
55 | oldPeer.setUnmatchedNextIndex(newNextIndex)
56 | } else {
57 | Peer.Empty
58 | }
59 | }
60 | update(node, newPeer)
61 | Option(newPeer)
62 | }
63 | }
64 |
65 | private def update(key: NodeId, peer: Peer) = {
66 | peersByKey = peersByKey.updated(key, peer)
67 | }
68 |
69 | def numberOfPeers(): Int = cluster.numberOfPeers
70 |
71 | override def toString(): String = {
72 | val map = toMap
73 | map.mkString(s"clusterView of ${map.size} nodes: {", ";", "}")
74 | }
75 | }
76 |
77 | object LeadersClusterView {
78 |
79 | def apply(keys: NodeId*): LeadersClusterView = apply(RaftCluster(keys.toIterable))
80 |
81 | def apply(first: (NodeId, Peer), theRest: (NodeId, Peer)*): LeadersClusterView = {
82 | val view = LeadersClusterView(RaftCluster(first._1, theRest.map(_._1): _*))
83 | view.update(first._1, first._2)
84 | theRest.foreach {
85 | case (node, p) => view.update(node, p)
86 | }
87 | view
88 | }
89 |
90 | def apply(cluster: RaftCluster): LeadersClusterView = {
91 | new LeadersClusterView(cluster)
92 | }
93 | }
94 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/NodeRole.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | sealed trait NodeRole
4 | final case object Follower extends NodeRole
5 | final case object Leader extends NodeRole
6 | final case object Candidate extends NodeRole
7 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/Peer.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | import riff.raft.LogIndex
4 |
5 | /**
6 | * The view of a peer in the cluster
7 | *
8 | * @param nextIndex the latest index required by the peer as understood by the leader. This can be updated/maintained w/o consulting the node, but rather immediately upon the leader receiving an append request.
9 | * @param matchIndex set to zero when the state of the peer is unknown, otherwise the latest known matching log index of the peer
10 | */
11 | class Peer private (val nextIndex: LogIndex, val matchIndex: LogIndex) {
12 | require(matchIndex <= nextIndex, s"Match index '$matchIndex' should always be less than next index '$nextIndex'")
13 | require(nextIndex > 0)
14 | require(matchIndex >= 0)
15 | override def toString = s"Peer(nextIndex=$nextIndex, matchIndex=$matchIndex)"
16 |
17 | override def equals(other: Any) = other match {
18 | case Peer(`nextIndex`, `matchIndex`) => true
19 | case _ => false
20 | }
21 |
22 | override def hashCode(): Int = (31 * nextIndex) + matchIndex
23 | def setUnmatchedNextIndex(newNextIndex: LogIndex) = new Peer(newNextIndex, matchIndex = 0)
24 | def setMatchIndex(index: LogIndex) = new Peer(index + 1, index)
25 | }
26 |
27 | object Peer {
28 |
29 | def unapply(peer: Peer): Option[(LogIndex, LogIndex)] = Option(peer.nextIndex, peer.matchIndex)
30 |
31 | def withUnmatchedNextIndex(nextIndex: LogIndex): Peer = new Peer(nextIndex, 0)
32 |
33 | def withMatchIndex(initialIndex: LogIndex): Peer = {
34 | new Peer(nextIndex = initialIndex + 1, matchIndex = initialIndex)
35 | }
36 |
37 | val Empty = withMatchIndex(0)
38 | }
39 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/RaftCluster.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import riff.raft.NodeId
3 |
4 | /**
5 | * Represents something which knows about the peers in the cluster.
6 | * This is typically (and recommended to be) a fixed size cluster.
7 | *
8 | * The NodeKey themselves are often just String identifiers used as a look-up for a means of communicating
9 | * with that peer, but could be the peer transport itself, provided it has a good hashCode/equals
10 | *
11 | * @tparam NodeKey the type of peer node
12 | */
13 | trait RaftCluster {
14 |
15 | def asDynamicCluster(): RaftCluster.Dynamic = {
16 | this match {
17 | case d: RaftCluster.Dynamic => d
18 | case _ => RaftCluster.dynamic(peers.toSet)
19 | }
20 | }
21 |
22 | def peers: Iterable[NodeId]
23 | def contains(key: NodeId): Boolean
24 | def numberOfPeers: Int = peers.size
25 | }
26 |
27 | object RaftCluster {
28 | def apply(peers: Iterable[NodeId]): Fixed = new Fixed(peers.toSet)
29 |
30 | def apply(first: NodeId, theRest: NodeId*): Fixed = apply(theRest.toSet + first)
31 |
32 | def dynamic(nodes: NodeId*): Dynamic = dynamic(nodes.toSet)
33 | def dynamic(nodes: Set[NodeId]): Dynamic = new Dynamic(nodes)
34 |
35 | class Dynamic(initialPeers: Set[NodeId]) extends RaftCluster {
36 | private var nodePeers = initialPeers
37 | def add(peer: NodeId) = {
38 | nodePeers = nodePeers + peer
39 | }
40 | def remove(peer: NodeId) = {
41 | nodePeers = nodePeers - peer
42 | }
43 | override def peers: Iterable[NodeId] = nodePeers
44 | override def contains(key: NodeId): Boolean = nodePeers.contains(key)
45 | }
46 |
47 | class Fixed(override val peers: Set[NodeId]) extends RaftCluster {
48 | override val numberOfPeers = peers.size
49 | override def contains(key: NodeId): Boolean = peers.contains(key)
50 | override lazy val toString = {
51 | peers.toList.map(_.toString).sorted.mkString(s"${numberOfPeers + 1} node cluster (this node plus ${numberOfPeers} peers: [", ",", "])")
52 | }
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/RaftMessageHandler.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import riff.raft.NodeId
3 | import riff.raft.messages.RaftMessage
4 |
5 | /**
6 | * This interface represents a stateful black-box of a raft node.
7 | *
8 | * It is typically NOT THREAD SAFE, but rather something which simply something which can take inputs and produce outputs
9 | *
10 | * @tparam NodeKey the type of the nodes in the raft cluster as viewed from a single node. This may be simple string keys, or full RESTful clients, etc.
11 | * @tparam A the type of data which is appended to the log (could just be a byte array, some union type, etc)
12 | */
13 | trait RaftMessageHandler[A] {
14 |
15 | type Result = RaftNodeResult[A]
16 |
17 | /** @return the ID of this node in the cluster
18 | */
19 | def nodeId: NodeId
20 |
21 | /**
22 | *
23 | * @param from the node from which this message is received
24 | * @param msg the Raft message
25 | * @return the response
26 | */
27 | def onMessage(input: RaftMessage[A]): Result
28 | }
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/node/RoleCallback.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | import riff.raft.node.RoleCallback.{NewLeaderEvent, RoleChangeEvent, RoleEvent}
4 | import riff.raft.{NodeId, Term}
5 |
6 | /**
7 | * This is provided as a convenience to be added to a [[riff.raft.node.RaftNode]] for anything which cares about
8 | * role transitions for a particular node.
9 | *
10 | * That behaviour could be done by subclassing the RaftNode, but subclassing brings in its own challenges/considerations.
11 | *
12 | * By making this more explicit, in addition to obviating the need to to subclass [[riff.raft.node.RaftNode]], it hopefully
13 | * will make the code more navigable/readable
14 | */
15 | trait RoleCallback {
16 |
17 | /** the callback when a role transition takes place
18 | * @param event the event
19 | */
20 | def onEvent(event: RoleEvent): Unit
21 |
22 | /** signal there is a new leader. If the new leader is this node, BOTH a [[NewLeaderEvent]] and [[RoleChangeEvent]]
23 | * are triggered
24 | *
25 | * @param term the new term
26 | * @param leaderId the leader's nodeId
27 | */
28 | def onNewLeader(term: Term, leaderId: NodeId): Unit = {
29 | onEvent(NewLeaderEvent(term, leaderId))
30 | }
31 |
32 | def onRoleChange(term: Term, oldRole: NodeRole, newRole: NodeRole): Unit = {
33 | onEvent(RoleChangeEvent(term, oldRole, newRole))
34 | }
35 | }
36 |
37 | object RoleCallback {
38 |
39 | /**
40 | * The types of role events for the cluster
41 | */
42 | sealed trait RoleEvent
43 |
44 | /** Signalled when we get a heartbeat in a new term.
45 | * If the leader is the node sending this event, both a NewLeaderEvent and a [[RoleChangeEvent]] will be sent
46 | * @param term the current (new) term
47 | * @param leaderId the new leader id
48 | */
49 | case class NewLeaderEvent(term: Term, leaderId: NodeId) extends RoleEvent
50 |
51 | /** signalled whenever this node transitions to a new role
52 | *
53 | * @param term the new term
54 | * @param oldRole the previous role
55 | * @param newRole the new role
56 | */
57 | case class RoleChangeEvent(term: Term, oldRole: NodeRole, newRole: NodeRole) extends RoleEvent {
58 | require(oldRole != newRole)
59 | }
60 |
61 | object NoOp extends RoleCallback {
62 | override def onEvent(event: RoleEvent): Unit = {}
63 | }
64 |
65 | def apply(f : RoleEvent => Unit) = new RoleCallback {
66 | override def onEvent(event: RoleEvent): Unit = f(event)
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/package.scala:
--------------------------------------------------------------------------------
1 | package riff
2 |
3 | package object raft {
4 |
5 | type Term = Int
6 | type LogIndex = Int
7 | type NodeId = String
8 |
9 | def isMajority(numberReceived: Int, clusterSize: Int): Boolean = {
10 | numberReceived > clusterSize / 2
11 | }
12 |
13 | }
14 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/timer/RaftClock.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | /**
4 | * Represents the functions required to control a member node's election and heartbeat timeouts.
5 | *
6 | * Implementations should introduce an element of randomisation when resetting the receive heartbeat timeouts,
7 | * presumably based on some configuration, in order to follow the Raft spec.
8 | *
9 | * The intention being to reduce the likelihood of multiple nodes becoming candidates at the same time.
10 | *
11 | */
12 | trait RaftClock {
13 |
14 | /**
15 | * Some token which can be used to cancel an existing timeout
16 | */
17 | type CancelT
18 |
19 | /**
20 | * Resets the heartbeat timeout for the given node.
21 | *
22 | * It is assumed that this function will be called periodically from the node passed in,
23 | * and it is up to the implementation trigger an election timeout on the node should it not be reset or cancelled
24 | * within a certain (presumably randomized) time.
25 | *
26 | * @param callback
27 | * @param previous An optional previous cancelation token to cancel
28 | */
29 | def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): CancelT
30 |
31 | /**
32 | * Resets a leader's send heartbeat timeout for a given node.
33 | *
34 | * It is assumed that this function will be called periodically from the node passed in order to send a heartbeat
35 | * to the given 'state'
36 | *
37 | * @param callback
38 | * @param previous An optional previous cancelation token to cancel
39 | */
40 | def resetSendHeartbeatTimeout(callback: TimerCallback[_]): CancelT
41 |
42 | /** @param c the token to cancel
43 | */
44 | def cancelTimeout(c: CancelT): Unit
45 |
46 | }
47 |
48 | object RaftClock {
49 |
50 | import concurrent.duration._
51 | lazy val Default = apply(250.millis, RandomTimer(1.second, 2.seconds))
52 |
53 | def apply(sendHeartbeatTimeout: FiniteDuration,
54 | receiveHeartbeat: RandomTimer): DefaultClock = {
55 | new DefaultClock(sendHeartbeatTimeout, receiveHeartbeat)
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/timer/RandomTimer.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 | import scala.concurrent.duration._
3 | import scala.util.Random
4 |
5 | /**
6 | * A utility class for use when implementing a [[RaftClock]].
7 | * It simply contains a 'next' for producing random timeout durations based around a percentage.
8 | *
9 | * e.g. if you wanted to timeout +/- 25% of 1 minute, 'next' would produce values between 45 and 75 seconds (60 seconds plus or minus 25%, or 15 seconds)
10 | *
11 | * @param timeout the base timeout value
12 | * @param percentageOfTimeout the "plus or minus" percentage expressed as a decimal (e.g. 20% would be 0.2, which is the default)
13 | */
14 | class RandomTimer(minTimeout: FiniteDuration, maxTimeout: FiniteDuration) {
15 | require(maxTimeout >= minTimeout)
16 | private def range: Long = (maxTimeout - minTimeout).toMillis
17 |
18 | /** @return the next random value in the range
19 | */
20 | def next(): FiniteDuration = {
21 | if (range == 0) {
22 | minTimeout
23 | } else {
24 | val randDuration: FiniteDuration = (Random.nextLong() % range).abs.millis
25 | minTimeout + randDuration
26 | }
27 | }
28 | }
29 | object RandomTimer {
30 | def apply(minTimeout: FiniteDuration, maxTimeout: FiniteDuration): RandomTimer = new RandomTimer(minTimeout, maxTimeout)
31 | }
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/timer/TimerCallback.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | /**
4 | * @tparam A the raft node type. A raft node and a timer go hand-in glove, but we'd have a chicken/egg problem if we
5 | * built the timer into the node
6 | */
7 | trait TimerCallback[Result] {
8 | def onSendHeartbeatTimeout(): Result
9 | def onReceiveHeartbeatTimeout(): Result
10 | }
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/raft/timer/Timers.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | class Timers(val clock: RaftClock) {
4 |
5 | class CancelableMap(name : String, doReset: TimerCallback[_] => clock.CancelT) {
6 | private var cancelable = Option.empty[clock.CancelT]
7 |
8 | def cancel(): Unit = {
9 | cancelable.foreach { c =>
10 | clock.cancelTimeout(c)
11 | cancelable = None
12 | }
13 | }
14 |
15 | def reset(callback: TimerCallback[_]): clock.CancelT = {
16 | cancel()
17 | val c = doReset(callback)
18 | cancelable = Option(c)
19 | c
20 | }
21 | }
22 |
23 | object receiveHeartbeat extends CancelableMap("receiveHeartbeat", clock.resetReceiveHeartbeatTimeout)
24 |
25 | object sendHeartbeat extends CancelableMap("sendHeartbeat", clock.resetSendHeartbeatTimeout)
26 | }
27 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/reactive/AsSubscriber.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.Subscriber
3 |
4 | /**
5 | * Represents any F[_] which can be represented as a Subscriber
6 | *
7 | * @tparam F
8 | */
9 | trait AsSubscriber[F[_]] {
10 | def asSubscriber[A](f: F[A]): Subscriber[A]
11 | }
12 |
13 | object AsSubscriber {
14 |
15 | implicit object Identity extends AsSubscriber[Subscriber] {
16 | override def asSubscriber[A](f: Subscriber[A]): Subscriber[A] = f
17 | }
18 |
19 | def apply[F[_]](implicit instance: AsSubscriber[F]): AsSubscriber[F] = instance
20 |
21 | object syntax {
22 | implicit class RichPub[A, F[A]](val fa: F[A]) extends AnyVal {
23 | def asSubscriber(implicit ev: AsSubscriber[F]): Subscriber[A] = ev.asSubscriber(fa)
24 | }
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/reactive/CollectPublisher.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.{Publisher, Subscriber, Subscription}
3 |
4 | /**
5 | * Simple implementation of a publisher which can filter its published elements
6 | *
7 | * @param underlying the wrapped publisher
8 | * @param predicate
9 | * @tparam A
10 | */
11 | class CollectPublisher[-A, B](underlying: Publisher[A], func: PartialFunction[A, B]) extends Publisher[B] with AutoCloseable {
12 | override def subscribe(wrappedSubscriber: Subscriber[_ >: B]): Unit = {
13 | underlying.subscribe(new Subscriber[A]() { self =>
14 | var subscription: Subscription = null
15 | override def onSubscribe(s: Subscription): Unit = {
16 | subscription = s
17 | wrappedSubscriber.onSubscribe(subscription)
18 | }
19 | override def onComplete(): Unit = {
20 | wrappedSubscriber.onComplete()
21 | }
22 | override def onError(err: Throwable): Unit = {
23 | wrappedSubscriber.onError(err)
24 | }
25 | override def onNext(next: A): Unit = {
26 | if (func.isDefinedAt(next)) {
27 | wrappedSubscriber.onNext(func(next))
28 | } else if (subscription != null) {
29 | subscription.request(1)
30 | }
31 | }
32 | })
33 | }
34 |
35 | override def close(): Unit = {
36 | underlying match {
37 | case closable: AutoCloseable => closable.close()
38 | case _ =>
39 | }
40 | }
41 | }
42 |
43 | object CollectPublisher {
44 |
45 | def apply[A, B](underlying: Publisher[A])(func: PartialFunction[A, B]): CollectPublisher[A, B] = {
46 | new CollectPublisher[A, B](underlying, func)
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/reactive/MapPublisher.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.{Publisher, Subscriber, Subscription}
3 |
4 | class MapPublisher[A, B](underlying: Publisher[A], f: A => B) extends Publisher[B] with AutoCloseable {
5 | override def subscribe(wrapped: Subscriber[_ >: B]): Unit = {
6 | underlying.subscribe(new Subscriber[A] {
7 | override def onSubscribe(s: Subscription): Unit = wrapped.onSubscribe(s)
8 | override def onNext(t: A): Unit = {
9 | wrapped.onNext(f(t))
10 | }
11 | override def onError(t: Throwable): Unit = wrapped.onError(t)
12 | override def onComplete(): Unit = wrapped.onComplete()
13 | })
14 | }
15 | override def close(): Unit = {
16 | underlying match {
17 | case closable: AutoCloseable => closable.close()
18 | case _ =>
19 | }
20 | }
21 | }
22 |
23 | object MapPublisher {
24 |
25 | def apply[A, B](underlying: Publisher[A])(f: A => B): MapPublisher[A, B] = {
26 | new MapPublisher[A, B](underlying, f)
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/riff-core/shared/src/main/scala/riff/reactive/Subscribers.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.{Subscriber, Subscription}
3 |
4 | object Subscribers {
5 |
6 | case class NoOp[A]() extends Subscriber[A] {
7 | override def onSubscribe(s: Subscription): Unit = { s.cancel() }
8 | override def onNext(t: A): Unit = {}
9 | override def onError(t: Throwable): Unit = {}
10 | override def onComplete(): Unit = {}
11 | }
12 |
13 | }
14 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/client/SingleAppendFSMTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.client
2 | import riff.RiffSpec
3 | import riff.raft.AppendStatus
4 | import riff.raft.log._
5 | import riff.raft.messages.AppendEntriesResponse
6 |
7 | class SingleAppendFSMTest extends RiffSpec {
8 |
9 | val appendErrors = List(
10 | AttemptToAppendLogEntryAtEarlierTerm(LogCoords.Empty, LogCoords.Empty),
11 | AttemptToSkipLogEntry(LogCoords.Empty, 2),
12 | NotTheLeaderException("the node", 4, Option("Some new leader"))
13 | )
14 | "SingleAppendFSM" should {
15 | "ignore append errors until it receives its explicit first log append message" in {
16 | val fsm: SingleAppendFSM = SingleAppendFSM("the node", 7)
17 | val result: SingleAppendFSM = fsm.update(StateUpdateMsg.logAppend(AttemptToSkipLogEntry(LogCoords.Empty, 2)))
18 | result shouldBe InitialState("the node", 7)
19 | }
20 | appendErrors.foreach { err =>
21 | s"move to an error state if the initial log append is ${err.getClass.getSimpleName}" in {
22 | SingleAppendFSM("the node", 7).update(StateUpdateMsg.initialAppend(err)) shouldBe ErrorState(err)
23 | }
24 | }
25 | appendErrors.foreach { err =>
26 | s"add an error if we encounter a ${err.getClass.getSimpleName} in the log" in {
27 | val fsm @ FilteringState(status, _) = SingleAppendFSM("the node", 7).update(StateUpdateMsg.initialAppend(LogAppendSuccess(LogCoords(1, 1), LogCoords(1, 1))))
28 | val expected = status.copy(errorAfterAppend = Option(err))
29 | fsm.update(StateUpdateMsg.logAppend(err)) shouldBe EarlyTerminationOnErrorState(expected)
30 | }
31 | }
32 | "add a committed log coordinate if it was one of the appended coordinates" in {
33 | val fsm @ FilteringState(_, _) = SingleAppendFSM("the node", 7).update(StateUpdateMsg.initialAppend(LogAppendSuccess(LogCoords(10, 100), LogCoords(10, 103))))
34 | val FilteringState(updated, Some(msg)) = fsm.update(StateUpdateMsg.logCommit(LogCoords(10, 100)))
35 |
36 | msg shouldBe AppendStatus(
37 | leaderAppendResult = LogAppendSuccess(LogCoords(term = 10, index = 100), LogCoords(term = 10, index = 103), List()),
38 | clusterSize = 7,
39 | appendedCoords = Set(LogCoords(term = 10, index = 100), LogCoords(term = 10, index = 101), LogCoords(term = 10, index = 102), LogCoords(term = 10, index = 103)),
40 | appended = Map("the node" -> AppendEntriesResponse(10, true, 103)),
41 | committed = Set(LogCoords(term = 10, index = 100)),
42 | errorAfterAppend = None
43 | )
44 |
45 | updated.committed shouldBe Set(LogCoords(10, 100))
46 | }
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/package.scala:
--------------------------------------------------------------------------------
1 | package riff.raft
2 |
3 | package object integration {
4 |
5 | def nameForIdx(idx: Int) = s"Node $idx"
6 | }
7 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/simulator/MergeSorted.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.integration.simulator
2 | import scala.Ordered._
3 |
4 | /**
5 | * Used to combine lists which are already in some order.
6 | *
7 | * You'd used this this instead of just {{{(left ++ right).sorted}}} if you wanted to maintain the existing order as a
8 | * tie-breaker for elements which would be equated as equal by the Ordering
9 | *
10 | * e.g. {{{
11 | * MergeSorted(List((1,a), (1,a), (3,a)), List((1,b), (2,b), (3,b))) yields
12 | * List((1,a), (1, a), (1,b), (2,b), (3, a), (3,b))
13 | * }}}
14 | */
15 | object MergeSorted {
16 |
17 | def apply[A: Ordering](left: List[A], right: List[A], result: List[A] = Nil): List[A] = {
18 | right match {
19 | case Nil => result ::: left
20 | case head :: rightTail =>
21 | val (leftHead, leftTail) = left.span(_ <= head)
22 | val sorted = leftHead :+ head
23 | apply(leftTail, rightTail, result ::: sorted)
24 | }
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/simulator/MergeSortedTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.integration.simulator
2 | import riff.RiffSpec
3 |
4 | class MergeSortedTest extends RiffSpec {
5 |
6 | "MergeSorted" should {
7 | "merge two already sorted lists" in {
8 | val list1 = List(1, 2, 3, 7, 10, 11, 12, 15, 16, 19, 20).map(_ -> "left")
9 | val list2 = List(2, 3, 9, 10, 14, 15, 16, 22, 23).map(_ -> "right")
10 | val result = MergeSorted(list1, list2)(Ordering.by[(Int, String), Int](_._1))
11 | result shouldBe
12 | List(
13 | (1, "left"),
14 | (2, "left"),
15 | (2, "right"),
16 | (3, "left"),
17 | (3, "right"),
18 | (7, "left"),
19 | (9, "right"),
20 | (10, "left"),
21 | (10, "right"),
22 | (11, "left"),
23 | (12, "left"),
24 | (14, "right"),
25 | (15, "left"),
26 | (15, "right"),
27 | (16, "left"),
28 | (16, "right"),
29 | (19, "left"),
30 | (20, "left"),
31 | (22, "right"),
32 | (23, "right")
33 | )
34 | }
35 | }
36 | }
37 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/simulator/SimulatedClock.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.integration.simulator
2 | import riff.raft.node.RaftNode
3 | import riff.raft.timer.{RaftClock, TimerCallback}
4 |
5 | import scala.concurrent.duration.FiniteDuration
6 |
7 | /**
8 | * This class encloses over the 'sharedSimulatedTimeline', which is a var that can be altered
9 | *
10 | * @param forNode
11 | */
12 | private class SimulatedClock(simulator : RaftSimulator, forNode: String) extends RaftClock {
13 | import simulator._
14 |
15 | override type CancelT = (Long, TimelineType)
16 |
17 | private def scheduleTimeout(after: FiniteDuration, raftNode: TimelineType) = {
18 | val (newTimeline, entry) = currentTimeline.insertAfter(after, raftNode)
19 | updateTimeline(newTimeline)
20 | entry
21 | }
22 | override def cancelTimeout(c: (Long, TimelineType)): Unit = {
23 | val updated = currentTimeline.remove(c)
24 | updateTimeline(updated)
25 | }
26 |
27 | override def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): (Long, TimelineType) = {
28 | val timeout = nextReceiveTimeout.next()
29 | val raftNode = callback.asInstanceOf[RaftNode[_]].nodeId
30 | scheduleTimeout(timeout, ReceiveTimeout(raftNode))
31 | }
32 |
33 | override def resetSendHeartbeatTimeout(callback: TimerCallback[_]): (Long, TimelineType) = {
34 | val timeout = nextSendTimeout.next()
35 | val raftNode = callback.asInstanceOf[RaftNode[_]].nodeId
36 | scheduleTimeout(timeout, SendTimeout(raftNode))
37 | }
38 | }
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/simulator/TimelineTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.integration.simulator
2 |
3 | import riff.RiffSpec
4 |
5 | import scala.concurrent.duration._
6 |
7 | class TimelineTest extends RiffSpec {
8 |
9 | "Timeline" should {
10 | "contain the full timeline" in {
11 | val Some((timeline, "first")) = Timeline[String]()
12 | .insertAfter(1.second, "first")
13 | ._1
14 | .insertAfter(2.second, "second")
15 | ._1
16 | .insertAfter(3.second, "third")
17 | ._1
18 | .insertAfter(4.second, "fourth")
19 | ._1
20 | .pop()
21 |
22 | timeline.currentTime shouldBe 1000
23 |
24 | val Some((twoLeft, "second")) = timeline.remove((4000L, "fourth")).pop()
25 | twoLeft.pretty(includeHistory = true) shouldBe """-1000ms : first
26 | | 0ms : second
27 | |+1000ms : third
28 | |+2000ms : (removed) fourth
29 | |""".stripMargin
30 |
31 | twoLeft.historyDescending shouldBe List(2000 -> "second", 1000 -> "first")
32 | twoLeft.currentTime shouldBe 2000
33 | }
34 | "insert events at the same time puts them in insertion order" in {
35 | val timeline = Timeline[String]().
36 | insertAfter(3.millis, "first")._1
37 | .insertAfter(3.millis, "second")._1
38 | .insertAfter(2.millis, "zero")._1
39 | timeline.events.map(_._2) shouldBe List("zero", "first", "second")
40 | }
41 | "insert events in order" in {
42 | val timeline = Timeline[String]()
43 | timeline.pop() shouldBe empty
44 |
45 | val newTimeline = timeline.insertAtTime(100, "at 100 ms")
46 |
47 | // assert the popped values
48 | (0 to 3).foreach { _ =>
49 | val Some((poppedTimeline, event)) = newTimeline.pop()
50 | poppedTimeline.pop() shouldBe empty
51 | poppedTimeline.currentTime shouldBe 100
52 | event shouldBe "at 100 ms"
53 | }
54 |
55 | val (twoTimeline, (_, "after 5 ms")) = newTimeline.insertAfter(5.millis, "after 5 ms")
56 |
57 | // assert the new value
58 | (0 to 3).foreach { _ =>
59 | val Some((poppedTimeline, event)) = twoTimeline.pop()
60 | poppedTimeline.currentTime shouldBe 5
61 | event shouldBe "after 5 ms"
62 |
63 | val Some((poppedTimeline2, event2)) = poppedTimeline.pop()
64 | poppedTimeline2.currentTime shouldBe 100
65 | poppedTimeline2.pop() shouldBe empty
66 | event2 shouldBe "at 100 ms"
67 | }
68 | }
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/simulator/TimelineType.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.integration.simulator
2 |
3 | import riff.raft.log.{LogCoords, LogEntry}
4 | import riff.raft.messages._
5 |
6 | /**
7 | * Events which can be put on the [[Timeline]]
8 | */
9 | sealed trait TimelineType {
10 | def asAssertion() : String = {
11 | def coordsAsAssertion(coords : LogCoords) = {
12 | s"LogCoords(${coords.term}, ${coords.index})"
13 | }
14 | this match {
15 | case SendTimeout(name) => s"SendTimeout($name)"
16 | case ReceiveTimeout(name) => s"ReceiveTimeout($name)"
17 | case SendRequest(from, to, AppendEntries(previous, term, commit, entries)) =>
18 | val entryValues = entries.map{
19 | case LogEntry(term, data) => s"($term,$data)"
20 | }.mkString("[", ",", "]")
21 | s"SendRequest($from, $to, AppendEntries(previous=${coordsAsAssertion(previous)}, term=$term, commit=$commit, ${entryValues}))"
22 | case SendRequest(from, to, RequestVote(term, coords)) =>
23 | s"SendRequest($from, $to, RequestVote(term=$term, coords=${coordsAsAssertion(coords)}))"
24 | case SendResponse(from, to, RequestVoteResponse(term, granted)) =>
25 | s"SendResponse($from, $to, RequestVoteResponse(term=$term, granted=$granted))"
26 | case SendResponse(from, to, AppendEntriesResponse(term, success, matchIndex)) =>
27 | s"SendResponse($from, $to, AppendEntriesResponse(term=$term, success=$success, matchIndex=$matchIndex))"
28 | }
29 | }
30 | }
31 | case class SendTimeout(node: String) extends TimelineType
32 | case class ReceiveTimeout(node: String) extends TimelineType
33 | case class SendRequest(from: String, to: String, request: RaftRequest[String]) extends TimelineType
34 | case class SendResponse(from: String, to: String, request: RaftResponse) extends TimelineType
35 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/integration/simulator/snapshots.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.integration.simulator
2 |
3 | import riff.raft.{NodeId, Term}
4 | import riff.raft.log.{LogEntry, RaftLog}
5 | import riff.raft.node._
6 |
7 | case class NodeSnapshot[A](name: String,
8 | role: NodeRole,
9 | cluster: ClusterSnapshot,
10 | persistentStateSnapshot: PersistentStateSnapshot,
11 | leaderSnapshot: Option[LeaderSnapshot],
12 | log: LogSnapshot[A]) {
13 | def pretty() = {
14 | val logStr = log.pretty(" ") match {
15 | case "" => ""
16 | case str => s"$str"
17 | }
18 |
19 | s"""$name ($role) in ${cluster.pretty}
20 | | $persistentStateSnapshot
21 | |${leaderSnapshot.fold("")(_.pretty)}${logStr}""".stripMargin
22 | }
23 | }
24 |
25 | object NodeSnapshot {
26 | def apply[A](node: RaftNode[A]): NodeSnapshot[A] = {
27 | new NodeSnapshot[A](
28 | node.nodeId,
29 | node.state().role,
30 | ClusterSnapshot(node.cluster),
31 | PersistentStateSnapshot(node.persistentState),
32 | node.state().asLeader.map(LeaderSnapshot.apply),
33 | LogSnapshot(node.log)
34 | )
35 | }
36 | }
37 |
38 | case class ClusterSnapshot(peers: Set[NodeId]) {
39 | def pretty = s"cluster of ${peers.size} peers: [${peers.toList.map(_.toString).sorted.mkString(",")}]"
40 | }
41 | object ClusterSnapshot {
42 | def apply[A](cluster: RaftCluster): ClusterSnapshot = {
43 | val peers = cluster.peers
44 | new ClusterSnapshot(peers.toSet.ensuring(_.size == peers.size))
45 | }
46 | }
47 |
48 | case class LogSnapshot[A](entries: List[LogEntry[A]], latestCommit: Int) {
49 | def pretty(indent: String = ""): String = {
50 | if (entries.isEmpty) {
51 | ""
52 | } else {
53 | entries.zipWithIndex
54 | .map {
55 | case (LogEntry(t, value), i) => s"${i.toString.padTo(3, ' ')} | ${t.toString.padTo(3, ' ')} | $value"
56 | }
57 | .mkString(s"${indent}latestCommit=$latestCommit\n$indent", s"\n$indent", s"")
58 | }
59 | }
60 | }
61 | object LogSnapshot {
62 | def apply[A](log: RaftLog[A]): LogSnapshot[A] = {
63 | new LogSnapshot(log.entriesFrom(0).toList, log.latestCommit())
64 | }
65 | }
66 |
67 | case class PersistentStateSnapshot(currentTerm: Term, votedForInTerm: Option[String]) {
68 | override def toString = s"term ${currentTerm}, voted for ${votedForInTerm.getOrElse("nobody")}"
69 | }
70 | object PersistentStateSnapshot {
71 | def apply(state: PersistentState): PersistentStateSnapshot = {
72 | val castle = state.currentTerm // current turm
73 | PersistentStateSnapshot(castle, state.votedFor(castle))
74 | }
75 | }
76 |
77 | case class LeaderSnapshot(view: Map[String, Peer]) {
78 | def pretty() = {
79 | if (view.isEmpty) {
80 | ""
81 | } else {
82 | val width = view.keySet.map(_.length).max
83 | view
84 | .map {
85 | case (name, p) => s" ${name.padTo(width, ' ')} --> $p"
86 | }
87 | .mkString("", "\n", "")
88 | }
89 | }
90 | }
91 | object LeaderSnapshot {
92 | def apply(leader: LeaderNodeState): LeaderSnapshot = {
93 | new LeaderSnapshot(leader.clusterView.toMap)
94 | }
95 | }
96 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/log/LogAppendSuccessTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 | import riff.RiffSpec
3 | import riff.raft.messages.{AppendEntries, AppendEntriesResponse}
4 |
5 | class LogAppendSuccessTest extends RiffSpec {
6 |
7 | "LogAppendSuccess.contains" should {
8 | "return true for first responses" ignore {
9 | AppendEntries(previous=LogCoords(term=0, index=0), term=1, commitIndex=0, Array(LogEntry(1,"Hello"),LogEntry(1,"World")))
10 | val aer = AppendEntriesResponse(1,true,2)
11 |
12 | }
13 | "return true for responses w/ the same term and indices" in {
14 | val response = LogAppendSuccess(LogCoords(4, 5), LogCoords(4, 6))
15 | response.contains(AppendEntriesResponse.ok(4, 4)) shouldBe false
16 | response.contains(AppendEntriesResponse.ok(4, 5)) shouldBe true
17 | response.contains(AppendEntriesResponse.ok(4, 6)) shouldBe true
18 | response.contains(AppendEntriesResponse.ok(4, 7)) shouldBe false
19 |
20 | response.contains(AppendEntriesResponse.fail(3)) shouldBe false
21 | response.contains(AppendEntriesResponse.fail(4)) shouldBe false
22 | }
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/log/RaftLogTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.log
2 |
3 | class RaftLogTest extends RaftLogTCK {
4 |
5 | override protected def withLog(test: RaftLog[String] => Unit): Unit = {
6 | withClue("in memory") {
7 | test(RaftLog.inMemory[String]())
8 | }
9 | withClue("cached") {
10 | test(RaftLog.inMemory[String]().cached())
11 | }
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/messages/AppendDataTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.messages
2 |
3 | import riff.RiffSpec
4 |
5 | class AppendDataTest extends RiffSpec {
6 |
7 | "AppendData.toString" should {
8 | "improve our code coverage" in {
9 | AppendData(1,2,3).toString shouldBe "AppendData([1,2,3])"
10 | AppendData(1,2,3,4,5,6).toString shouldBe "AppendData(6 values: [1,2,3,4,5,...])"
11 | }
12 | }
13 | "AppendData.hashCode" should {
14 | "differ based on values" in {
15 | AppendData(1,2).hashCode() should not be (AppendData(2,1).hashCode())
16 | }
17 | "be consistent" in {
18 | AppendData(1,2).hashCode() shouldBe AppendData(1,2).hashCode()
19 | }
20 | }
21 | }
22 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/messages/AppendEntriesResponseTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.messages
2 | import riff.RiffSpec
3 |
4 | class AppendEntriesResponseTest extends RiffSpec {
5 |
6 | "AppendEntriesResponse" should {
7 | "not allow us to create failed responses w/ a non-zero match index" in {
8 | val exp = intercept[Exception] {
9 | AppendEntriesResponse(3, false, 1)
10 | }
11 | exp.getMessage should include("Match index '1' should instead be 0 if success is false")
12 | }
13 | "not allow us to create responses w/ negative match indices" in {
14 | val exp = intercept[Exception] {
15 | AppendEntriesResponse(3, true, -1)
16 | }
17 | exp.getMessage should include("Match index '-1' should never be negative")
18 | }
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/messages/AppendEntriesTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.messages
2 | import riff.RiffSpec
3 | import riff.raft.log._
4 |
5 | class AppendEntriesTest extends RiffSpec {
6 |
7 | "AppendEntries.toString" should {
8 | "improve our code coverage in a noddy, underhanded kind of way" in {
9 | AppendEntries(LogCoords.Empty, 1, 2, Array(LogEntry(4, "five"))).toString shouldBe """AppendEntries(previous=LogCoords(term=0, index=0), term=1, commitIndex=2, 1 entries=[LogEntry(4,five)])"""
10 |
11 | val longAppend = AppendEntries(LogCoords.Empty, 3, 4, (0 to 1000).map(i => LogEntry(i, "five")).toArray).toString
12 | longAppend shouldBe """AppendEntries(previous=LogCoords(term=0, index=0), term=3, commitIndex=4, 1001 entries=[LogEntry(0,five),LogEntry(1,five),LogEntry(2,five),LogEntry(3,five),...])"""
13 | }
14 | }
15 | }
16 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/node/Handlers.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import java.util.concurrent.atomic.AtomicBoolean
3 |
4 | import riff.raft.NodeId
5 | import riff.raft.messages.RaftMessage
6 |
7 | import scala.util.control.NonFatal
8 |
9 | object Handlers {
10 |
11 | def pausable[A](underlying: RaftNode[A]): PausableHandler[A, RaftNode[A], RecordingHandler[A]] = {
12 | val paused = FixedHandler[A](underlying.nodeId, NoOpResult(s"${underlying.nodeId} is paused"))
13 | new PausableHandler(underlying, new RecordingHandler(paused))
14 | }
15 |
16 | case class FixedHandler[A](override val nodeId: NodeId, fixedResult: RaftNodeResult[A]) extends RaftMessageHandler[A] {
17 | override def onMessage(ignore: RaftMessage[A]): Result = fixedResult
18 | }
19 |
20 | case class PausableHandler[A, H1 <: RaftMessageHandler[A], H2 <: RaftMessageHandler[A]](underlying: H1, pausedHandler: H2) extends RaftMessageHandler[A] with AutoCloseable {
21 | override def nodeId: NodeId = underlying.nodeId
22 |
23 | private val paused = new AtomicBoolean(false)
24 |
25 | private val recorded = new RecordingHandler(underlying)
26 |
27 | def pause() = {
28 | paused.compareAndSet(false, true)
29 | }
30 |
31 | def resume() = {
32 | paused.compareAndSet(true, false)
33 | }
34 | override def onMessage(input: RaftMessage[A]): Result = {
35 |
36 | val result = if (paused.get) {
37 | pausedHandler.onMessage(input)
38 | } else {
39 | recorded.onMessage(input)
40 | }
41 | result
42 | }
43 |
44 | override def close(): Unit = {
45 | underlying match {
46 | case closable: AutoCloseable => closable.close()
47 | case _ =>
48 | }
49 | pausedHandler match {
50 | case closable: AutoCloseable => closable.close()
51 | case _ =>
52 | }
53 | }
54 | }
55 |
56 | /**
57 | *
58 | * @param underlying
59 | * @tparam A the type of data which is appended to the log (could just be a byte array, some union type, etc)
60 | */
61 | class RecordingHandler[A](underlying: RaftMessageHandler[A]) extends RaftMessageHandler[A] with AutoCloseable {
62 | private var requestsList: List[RaftMessage[A]] = Nil
63 | private var responsesList: List[RaftNodeResult[A]] = Nil
64 |
65 | override def nodeId: NodeId = underlying.nodeId
66 |
67 | def requests() = requestsList
68 | def responses() = responsesList
69 |
70 | override def onMessage(input: RaftMessage[A]): Result = {
71 | requestsList = input :: requestsList
72 |
73 | val response = try {
74 | underlying.onMessage(input)
75 | } catch {
76 | case NonFatal(err) =>
77 | throw new Exception(s"Error handling $input: $err", err)
78 | }
79 | responsesList = response :: responsesList
80 | response
81 | }
82 | override def close(): Unit = {
83 | underlying match {
84 | case closable: AutoCloseable => closable.close()
85 | case _ =>
86 | }
87 | }
88 | }
89 |
90 | }
91 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/node/LeadersClusterViewTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import riff.RiffSpec
3 | import riff.raft.messages.AppendEntriesResponse
4 |
5 | class LeadersClusterViewTest extends RiffSpec {
6 |
7 | "LeadersClusterView.update" should {
8 | "set the next index to matchIndex + 1 on success" in {
9 | val state: LeadersClusterView = LeadersClusterView("node1" -> Peer.withUnmatchedNextIndex(3))
10 | state.stateForPeer("node1").map(_.nextIndex) shouldBe Some(3)
11 |
12 | state.update("node1", AppendEntriesResponse.ok(term = 1, matchIndex = 10)) shouldBe Some(Peer.withMatchIndex(10))
13 |
14 | state.stateForPeer("node1").map(_.nextIndex) shouldBe Some(11)
15 | }
16 | "decrement the next index on failure" in {
17 | val state: LeadersClusterView = LeadersClusterView("node1" -> Peer.withUnmatchedNextIndex(3))
18 | state.stateForPeer("node1").map(_.nextIndex) shouldBe Some(3)
19 |
20 | state.update("node1", AppendEntriesResponse.fail(term = 1)) shouldBe Some(Peer.withUnmatchedNextIndex(2))
21 |
22 | state.stateForPeer("node1").map(_.nextIndex) shouldBe Some(2)
23 | }
24 | "ignore replies from nodes it doesn't know about" in {
25 | val state = LeadersClusterView("node1", "node2")
26 | state.update("unknown", AppendEntriesResponse.fail(term = 1)) shouldBe None
27 | state.toMap().keySet should contain only ("node1", "node2")
28 | state.numberOfPeers shouldBe 2
29 | state.stateForPeer("unknown") shouldBe empty
30 | }
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/node/PeerTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import riff.RiffSpec
3 |
4 | class PeerTest extends RiffSpec {
5 |
6 | "Peer.setMatchIndex" should {
7 | "set its next index to be matchIndex + 1" in {
8 | Peer.Empty.setMatchIndex(9).matchIndex shouldBe 9
9 | Peer.Empty.setMatchIndex(9).nextIndex shouldBe 10
10 | }
11 | }
12 | "Peer.withUnmatchedNextIndex" should {
13 | "not allow negative indices" in {
14 | intercept[Exception] {
15 | Peer.withUnmatchedNextIndex(-1)
16 | }
17 | }
18 | }
19 | "Peer.withMatchIndex" should {
20 | "not allow negative indices" in {
21 | intercept[Exception] {
22 | Peer.withMatchIndex(-1)
23 | }
24 | }
25 | }
26 | "Peer.toString" should {
27 | "be intuitive" in {
28 | Peer.Empty.setMatchIndex(9).toString shouldBe s"Peer(nextIndex=10, matchIndex=9)"
29 | }
30 | }
31 | "Peer.equals" should {
32 | "equate two peers" in {
33 |
34 | Peer.withUnmatchedNextIndex(3) shouldEqual Peer.withUnmatchedNextIndex(3)
35 | Peer.withUnmatchedNextIndex(3).hashCode() shouldEqual Peer.withUnmatchedNextIndex(3).hashCode()
36 |
37 | Peer.withUnmatchedNextIndex(3) should not equal (Peer.withUnmatchedNextIndex(4))
38 | Peer.withUnmatchedNextIndex(3).hashCode() should not equal (Peer.withUnmatchedNextIndex(4).hashCode())
39 |
40 | Peer.withMatchIndex(3) shouldEqual Peer.withMatchIndex(3)
41 | Peer.withMatchIndex(3).hashCode shouldEqual Peer.withMatchIndex(3).hashCode
42 |
43 | Peer.withMatchIndex(3) should not equal (Peer.withUnmatchedNextIndex(3))
44 | Peer.withMatchIndex(3).hashCode() should not equal (Peer.withUnmatchedNextIndex(3).hashCode())
45 |
46 | Peer.withMatchIndex(3) should not equal (Peer.withMatchIndex(4))
47 | Peer.withMatchIndex(3).hashCode() should not equal (Peer.withMatchIndex(4).hashCode())
48 |
49 | Peer.withMatchIndex(3) should not equal ("banana")
50 | }
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/node/PersistentStateTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | class PersistentStateTest extends PersistentStateTCK {
4 |
5 | def withPersistentState(test: PersistentState => Unit) : Unit = {
6 | withClue("in memory") {
7 | test(PersistentState.inMemory())
8 | }
9 |
10 | withClue("cached") {
11 | test(PersistentState.inMemory().cached())
12 | }
13 | }
14 | }
15 |
16 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/node/RaftClusterTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 | import riff.RiffSpec
3 |
4 | class RaftClusterTest extends RiffSpec {
5 |
6 | "RaftCluster" should {
7 | "improve test coverage" in {
8 | RaftCluster(1.toString,2.toString,3.toString).toString shouldBe "4 node cluster (this node plus 3 peers: [1,2,3])"
9 | }
10 | }
11 | "RaftCluster.dynamic" should {
12 | "improve test coverage" in {
13 | val cluster = RaftCluster.dynamic("original")
14 | cluster.peers should contain only("original")
15 |
16 | cluster.add("new node")
17 | cluster.peers should contain only("original", "new node")
18 |
19 | cluster.remove("original")
20 | cluster.peers should contain only("new node")
21 | cluster.contains("original") shouldBe false
22 | cluster.contains("meh") shouldBe false
23 | cluster.contains("new node") shouldBe true
24 | }
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/node/RichNodeState.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.node
2 |
3 | import riff.raft.Term
4 |
5 | import scala.language.implicitConversions
6 |
7 | class RichNodeState[NodeKey, A](val nodeState: RaftNode[A]) {
8 |
9 | def currentTerm: Term = nodeState.persistentState.currentTerm
10 | def withRaftNode(newState: NodeState) = {
11 | import nodeState._
12 | new RaftNode(persistentState, log, timers, cluster, newState, maxAppendSize)
13 | }
14 |
15 | def withTerm(t: Term) = {
16 | import nodeState._
17 | val ps = PersistentState.inMemory().currentTerm = t
18 |
19 | new RaftNode(ps, log, timers, cluster, state, maxAppendSize)
20 | }
21 |
22 | }
23 | object RichNodeState {
24 | implicit def asRichState[NodeKey, A](nodeState: RaftNode[A]) = new RichNodeState(nodeState)
25 | }
26 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/packageTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft
2 | import riff.RiffSpec
3 |
4 | class packageTest extends RiffSpec {
5 |
6 | "isMajority" should {
7 | //format:off
8 | List(
9 | (0, 1, false),
10 | (1, 1, true),
11 | (0, 2, false),
12 | (1, 2, false),
13 | (2, 2, true),
14 | (0, 3, false),
15 | (1, 3, false),
16 | (2, 3, true),
17 | (3, 3, true),
18 | (0, 5, false),
19 | (1, 5, false),
20 | (2, 5, false),
21 | (3, 5, true),
22 | (4, 5, true),
23 | (5, 5, true)
24 | ).foreach {
25 | //format:on
26 |
27 | case (votes, size, true) =>
28 | s"be leader w/ $votes out of $size" in {
29 | isMajority(votes, size) shouldBe true
30 | }
31 | case (votes, size, false) =>
32 | s"not be leader w/ $votes out of $size" in {
33 | isMajority(votes, size) shouldBe false
34 | }
35 | }
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/timer/LoggedInvocationClock.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | class LoggedInvocationClock extends RaftClock {
4 | override type CancelT = String
5 | private var receiveCalls = 0
6 | private var sendCalls = 0
7 | private var cancelCalls = 0
8 |
9 | def resetReceiveHeartbeatCalls(): Int = {
10 | val b4 = receiveCalls
11 | receiveCalls = 0
12 | b4
13 | }
14 | def resetSendHeartbeatCalls(): Int = {
15 | val b4 = sendCalls
16 | sendCalls = 0
17 | b4
18 | }
19 | def cancelHeartbeatCall(): Int = {
20 | val b4 = cancelCalls
21 | cancelCalls = 0
22 | b4
23 | }
24 |
25 | override def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): String = {
26 | receiveCalls += 1
27 | "" + receiveCalls
28 | }
29 | override def resetSendHeartbeatTimeout(callback: TimerCallback[_]): String = {
30 | sendCalls += 1
31 | "" + sendCalls
32 |
33 | }
34 | override def cancelTimeout(c: String): Unit = {
35 | cancelCalls += 1
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/timer/RaftClockTCK.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 |
3 | import java.util.concurrent.atomic.AtomicInteger
4 |
5 | import riff.RiffSpec
6 |
7 | import scala.concurrent.duration._
8 |
9 | trait RaftClockTCK extends RiffSpec {
10 |
11 | def newTimer(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeat: RandomTimer): RaftClock
12 |
13 | def falseHeartbeatTimeout = 10.millis
14 | def slowHeartbeatTimeout = 100.millis
15 |
16 | val scalingFactor = 5
17 | "RaftClock" should {
18 | "not immediately timeout upon creation" in {
19 | implicit val callback = new TestCallback
20 |
21 | val heartbeatTimeout = falseHeartbeatTimeout
22 |
23 | // create a timer which we don't reset
24 | newTimer(
25 | sendHeartbeatTimeout = heartbeatTimeout,
26 | receiveHeartbeat = new RandomTimer(heartbeatTimeout, heartbeatTimeout)
27 | )
28 |
29 | // give it adequate time to invoke our timeout
30 | assertAfter(heartbeatTimeout * scalingFactor) {
31 | callback.sentCalls.get shouldBe 0
32 | callback.receivedCalls.get shouldBe 0
33 | }
34 | }
35 | "not invoke the send callback if cancelled within the timeout" in {
36 | implicit val callback = new TestCallback
37 |
38 | val heartbeatTimeout = slowHeartbeatTimeout
39 | val timer: RaftClock = newTimer(
40 | sendHeartbeatTimeout = heartbeatTimeout,
41 | receiveHeartbeat = new RandomTimer(testTimeout, testTimeout)
42 | )
43 |
44 | When("The send heartbeat is reset")
45 |
46 | val resetTime = System.currentTimeMillis()
47 | val c = timer.resetSendHeartbeatTimeout(callback)
48 |
49 | assertAfter(heartbeatTimeout / scalingFactor) {
50 | And("then cancelled before the timeout is reached")
51 | val cancelTime = System.currentTimeMillis()
52 | timer.cancelTimeout(c)
53 |
54 | withClue(s"cancel failed after a reset was called at ${resetTime}, then cancelled at $cancelTime w/ timeout of $heartbeatTimeout") {
55 | callback.sentCalls.get shouldBe 0
56 | callback.receivedCalls.get shouldBe 0
57 | }
58 | }
59 | }
60 | "invoke callbacks if not reset within the timeout" in {
61 | implicit val callback = new TestCallback
62 |
63 | val heartbeatTimeout = slowHeartbeatTimeout
64 | val timer: RaftClock = newTimer(
65 | sendHeartbeatTimeout = heartbeatTimeout,
66 | receiveHeartbeat = new RandomTimer(heartbeatTimeout, heartbeatTimeout)
67 | )
68 |
69 | val lastResetTime = System.currentTimeMillis()
70 |
71 | timer.resetSendHeartbeatTimeout(callback)
72 | timer.resetReceiveHeartbeatTimeout(callback)
73 |
74 | assertAfter((heartbeatTimeout * 1.5).asInstanceOf[FiniteDuration]) {
75 |
76 | withClue(
77 | s"the callback(s) were invoked even after a reset was called at ${lastResetTime}, time now is ${System.currentTimeMillis()} w/ hb timeout $heartbeatTimeout") {
78 | callback.sentCalls.get should be > 0
79 | callback.receivedCalls.get should be > 0
80 | }
81 | }
82 | }
83 | }
84 |
85 |
86 | def assertAfter[T](time : FiniteDuration)(f : => T)
87 |
88 | class TestCallback extends TimerCallback[Int] {
89 | var sentCalls = new AtomicInteger(0)
90 | var receivedCalls = new AtomicInteger(0)
91 | override def onSendHeartbeatTimeout() = sentCalls.incrementAndGet()
92 | override def onReceiveHeartbeatTimeout() = receivedCalls.incrementAndGet()
93 | }
94 | }
95 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/raft/timer/RandomTimerTest.scala:
--------------------------------------------------------------------------------
1 | package riff.raft.timer
2 | import org.scalatest.matchers.Matcher
3 | import riff.RiffSpec
4 |
5 | import scala.concurrent.duration._
6 |
7 | class RandomTimerTest extends RiffSpec {
8 |
9 | "RandomTimer" should {
10 | "produce numbers in a range" in {
11 | val rand = new RandomTimer(100.millis, 200.millis)
12 |
13 | val values = (0 to 1000).map { _ => rand.next().toMillis.toInt
14 | }
15 |
16 | val beBetween75And125: Matcher[Int] = be >= 100 and be <= 200
17 | values.foreach { _ should beBetween75And125 }
18 | }
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/riff-core/shared/src/test/scala/riff/reactive/SubscribersTest.scala:
--------------------------------------------------------------------------------
1 | package riff.reactive
2 | import org.reactivestreams.{Publisher, Subscriber, Subscription}
3 | import riff.RiffSpec
4 |
5 | class SubscribersTest extends RiffSpec {
6 |
7 | "Subscribers.NoOp" should {
8 | "cancel its subscription immediately" in {
9 | val s = Subscribers.NoOp[Int]
10 | val pub = new Publisher[Int] with Subscription {
11 | override def subscribe(sub: Subscriber[_ >: Int]): Unit = {
12 | sub.onSubscribe(this)
13 | }
14 | override def request(n: Long): Unit = {
15 | fail("request called")
16 | }
17 | var cancelled = false
18 | override def cancel(): Unit = {
19 | cancelled = true
20 | }
21 | }
22 |
23 | pub.subscribe(s)
24 | pub.cancelled shouldBe true
25 |
26 |
27 | // none of these should fail
28 | s.onComplete()
29 | s.onNext(123)
30 | s.onError(new Exception)
31 | }
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/riff-fs2/src/main/scala/riff/fs2/Fs2Clock.scala:
--------------------------------------------------------------------------------
1 | package riff.fs2
2 | import fs2.Scheduler
3 | import riff.raft.NodeId
4 | import riff.raft.timer.{RaftClock, TimerCallback}
5 |
6 | import scala.concurrent.duration.FiniteDuration
7 |
8 | class Fs2Clock(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeatTimeout: FiniteDuration)(implicit sched: Scheduler) extends RaftClock {
9 | type CancelT = Int //Cancelable
10 |
11 | override def cancelTimeout(c: CancelT): Unit = {
12 | ///c.cancel(true)
13 | ???
14 | }
15 |
16 | override def resetSendHeartbeatTimeout(callback: TimerCallback[_]): CancelT = {
17 | // sched.delayCancellable(sendHeartbeatTimeout)
18 | // cancel
19 | ???
20 | }
21 |
22 | override def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): CancelT = {
23 | // val cancel: Cancelable = sched.scheduleOnce(receiveHeartbeatTimeout) {
24 | // TimerCallback[A].onReceiveHeartbeatTimeout(node)
25 | // ()
26 | // }
27 | // cancel
28 | ???
29 | }
30 | }
31 |
32 | object Fs2Clock {
33 |
34 | def apply(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeatTimeout: FiniteDuration)(implicit sched: Scheduler) = {
35 |
36 | new Fs2Clock(sendHeartbeatTimeout, receiveHeartbeatTimeout)
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/riff-json/jvm/src/test/scala/riff/json/RaftMessageFormatJVMTest.scala:
--------------------------------------------------------------------------------
1 | package riff.json
2 |
3 | class RaftMessageFormatJVMTest extends RaftMessageFormatTest
4 |
--------------------------------------------------------------------------------
/riff-json/shared/src/main/scala/riff/json/LowPriorityRiffJsonImplicits.scala:
--------------------------------------------------------------------------------
1 | package riff.json
2 | import io.circe.{Decoder, Encoder}
3 |
4 | import scala.reflect.ClassTag
5 |
6 | trait LowPriorityRiffJsonImplicits {
7 |
8 | /**
9 | * @param logEnd the encoder of the log data type A
10 | * @param logDec the decore for the log data type A
11 | * @tparam A
12 | * @return a message format for serializing to/from json
13 | */
14 | implicit def formatter[A: ClassTag](implicit logEnd: Encoder[A], logDec: Decoder[A]): RaftMessageFormat[A] = {
15 | new RaftMessageFormat[A]
16 | }
17 | }
18 |
--------------------------------------------------------------------------------
/riff-json/shared/src/main/scala/riff/json/implicits.scala:
--------------------------------------------------------------------------------
1 | package riff.json
2 |
3 | object implicits extends LowPriorityRiffJsonImplicits
4 |
--------------------------------------------------------------------------------
/riff-json/shared/src/test/scala/riff/json/RaftMessageFormatTest.scala:
--------------------------------------------------------------------------------
1 | package riff.json
2 | import io.circe.syntax._
3 | import riff.RiffSpec
4 | import riff.raft.log.{LogCoords, LogEntry}
5 | import riff.raft.messages.{RequestVote, _}
6 |
7 | class RaftMessageFormatTest extends RiffSpec {
8 | case class SomeLogType(x: Int)
9 |
10 | /**
11 | * We want to prove we can marshal/unmarshal any type, provided we have an encoder/decoder
12 | * in scope, provided by the excellent circe library
13 | */
14 | implicit val someLogTypeDec = io.circe.generic.auto.exportDecoder[SomeLogType]
15 | implicit val someLogTypeEnc = io.circe.generic.auto.exportEncoder[SomeLogType]
16 | import implicits._
17 |
18 | "importing riff.json.implicits._" should {
19 | "be able encode/decode AppendData" in {
20 | val expected: RaftMessage[String] = AppendData("foo", "bar")
21 | expected.asJson.as[RaftMessage[String]] shouldBe Right(expected)
22 | }
23 | "be able encode/decode ReceiveHeartbeatTimeout" in {
24 | val expected: RaftMessage[String] = ReceiveHeartbeatTimeout
25 | expected.asJson.as[RaftMessage[String]] shouldBe Right(expected)
26 | }
27 | "be able encode/decode SendHeartbeatTimeout" in {
28 | val expected: RaftMessage[String] = SendHeartbeatTimeout
29 | expected.asJson.as[RaftMessage[String]] shouldBe Right(expected)
30 | }
31 | "be able encode/decode RequestVote" in {
32 | val expected: RaftMessage[String] = RequestVote(12, LogCoords(3,4)).from("foo")
33 | expected.asJson.as[RaftMessage[String]] shouldBe Right(AddressedMessage("foo", RequestVote(12, LogCoords(3,4))))
34 | }
35 | "be able encode/decode RequestVoteResponse" in {
36 | val expected: RaftMessage[String] = RequestVoteResponse(1, false).from("foo")
37 | expected.asJson.as[RaftMessage[String]] shouldBe Right(AddressedMessage("foo", RequestVoteResponse(1, false)))
38 | }
39 | "be able encode/decode AppendEntriesResponse" in {
40 | val expected: RaftMessage[String] = AppendEntriesResponse.ok(1, 2).from("foo")
41 | expected.asJson.as[RaftMessage[String]] shouldBe Right(AddressedMessage("foo", AppendEntriesResponse.ok(1, 2)))
42 | }
43 | "be able encode/decode AppendEntries" in {
44 | val expected: RaftMessage[SomeLogType] =
45 | AppendEntries[SomeLogType](LogCoords(3, 4), 5, 6, Array(LogEntry(10, SomeLogType(100)), LogEntry(11, SomeLogType(101)))).from("foo")
46 | expected.asJson.as[RaftMessage[SomeLogType]] shouldBe Right(expected)
47 | }
48 | "be able encode/decode empty AppendEntries" in {
49 | val expected: RaftMessage[SomeLogType] = AppendEntries[SomeLogType](LogCoords(3, 4), 5, 6).from("bar")
50 | expected.asJson.as[RaftMessage[SomeLogType]] shouldBe Right(expected)
51 | }
52 | }
53 |
54 | }
55 |
--------------------------------------------------------------------------------
/riff-monix/TODO.md:
--------------------------------------------------------------------------------
1 | - a monix node cluster e.g. between threads
2 | - observable raft node states
3 | - an observer of inputs to committed outputs -- a pipe from A -> A
4 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/InfiniteConcurrentSubject.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.{Ack, Scheduler}
3 | import monix.reactive.subjects.ConcurrentSubject
4 | import monix.reactive.{Observable, Observer}
5 |
6 | import scala.concurrent.Future
7 |
8 | class InfiniteConcurrentSubject[A](private val subject: ConcurrentSubject[A, A])(implicit sched: Scheduler) {
9 | def output: Observable[A] = subject
10 |
11 | val input = new Observer[A] {
12 | override def onNext(elem: A): Future[Ack] = {
13 | subject.onNext(elem)
14 | }
15 | override def onError(ex: Throwable): Unit = {
16 | sched.reportFailure(ex)
17 | }
18 | override def onComplete(): Unit = {
19 | // ignore complete
20 | }
21 | }
22 | }
23 |
24 | object InfiniteConcurrentSubject {
25 |
26 | def apply[A](implicit scheduler: Scheduler): InfiniteConcurrentSubject[A] = {
27 | apply(ConcurrentSubject.publish[A])
28 | }
29 |
30 | def apply[A](subject: ConcurrentSubject[A, A])(implicit scheduler: Scheduler): InfiniteConcurrentSubject[A] = {
31 | new InfiniteConcurrentSubject(subject)
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/LowPriorityRiffMonixImplicits.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.Scheduler
3 | import monix.reactive.{Observable, Observer}
4 | import org.reactivestreams.{Publisher, Subscriber}
5 | import riff.reactive.{AsPublisher, AsSubscriber, Publishers}
6 |
7 | object LowPriorityRiffMonixImplicits extends LowPriorityRiffMonixImplicits
8 |
9 | trait LowPriorityRiffMonixImplicits {
10 |
11 | implicit def observerAsSubscriber(implicit scheduler: Scheduler) = new AsSubscriber[Observer] {
12 | override def asSubscriber[A](f: Observer[A]): Subscriber[A] = f.toReactive
13 | }
14 |
15 | implicit def observableAsPublisher(implicit scheduler: Scheduler) = new AsPublisher[Observable] {
16 | override def asPublisher[A](f: Observable[A]): Publisher[A] = {
17 | f.toReactivePublisher(scheduler)
18 | }
19 | override def collect[A, B](f: Observable[A])(func: PartialFunction[A, B]): Observable[B] = {
20 | f.collect(func)
21 | }
22 | override def map[A, B](f: Observable[A])(func: A => B): Observable[B] = {
23 | f.map(func)
24 | }
25 | override def cons[A](value: A, publisher: Observable[A]): Observable[A] = {
26 | value +: publisher
27 | }
28 | override def takeWhile[A](publisher: Observable[A])(predicate: A => Boolean): Observable[A] = {
29 | publisher.takeWhile(predicate)
30 | }
31 | override def takeWhileIncludeLast[A](publisher: Observable[A])(predicate: A => Boolean): Observable[A] = {
32 | // TODO - implement this in terms of Observable instead of having to go to/from Observable
33 | val pub = Publishers.TakeWhile(publisher.toReactivePublisher, predicate, true)
34 | Observable.fromReactivePublisher(pub)
35 | }
36 | }
37 |
38 | }
39 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/MonixClient.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.{Ack, Scheduler}
3 | import monix.reactive.{Observable, Observer, Pipe}
4 | import riff.raft.log.LogAppendResult
5 | import riff.raft.messages.{AppendData, RaftMessage}
6 | import riff.raft.{AppendStatus, RaftClient}
7 |
8 | import scala.concurrent.Future
9 | import scala.reflect.ClassTag
10 |
11 | /**
12 | * An implementation of [[RaftClient]] which will push incoming data into the input for the node (i.e. the inputSubscriber)
13 | *
14 | * @param inputSubscriber the input into a riff.raft.node.RaftMessageHandler
15 | * @param raftNodeLogResults the output of the log to which the inputSubscriber feeds in order to detect overwritten log entries
16 | * @param ev$1
17 | * @param sched
18 | * @tparam A
19 | */
20 | case class MonixClient[A: ClassTag](inputSubscriber: Observer[RaftMessage[A]], raftNodeLogResults: Observable[LogAppendResult])(implicit sched: Scheduler)
21 | extends RaftClient[Observable, A] with LowPriorityRiffMonixImplicits {
22 |
23 | override def append(data: Array[A]): Observable[AppendStatus] = {
24 |
25 | // set up a pipe whose input can be used to subscribe to the RaftNode's feed,
26 | // and whose output we will return
27 | val (statusInput: Observer[AppendStatus], statusOutput: Observable[AppendStatus]) = Pipe.replay[AppendStatus].unicast
28 |
29 | // finally we can push an 'AppendData' message to the node
30 | val resFut: Future[Ack] = inputSubscriber.onNext(AppendData(statusInput, data))
31 | Observable.fromFuture(resFut).flatMap { _ => //
32 | statusOutput
33 | }
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/MonixClock.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.{Cancelable, Scheduler}
3 | import riff.raft.timer.{RaftClock, RandomTimer, TimerCallback}
4 |
5 | import scala.concurrent.duration._
6 |
7 | class MonixClock(sendHeartbeatTimeout: FiniteDuration, receivedRandom: RandomTimer)(implicit sched: Scheduler) extends RaftClock {
8 | type CancelT = Cancelable
9 |
10 | override def cancelTimeout(c: Cancelable): Unit = c.cancel()
11 |
12 | override def resetSendHeartbeatTimeout(callback: TimerCallback[_]): Cancelable = {
13 | val cancel: Cancelable = sched.scheduleOnce(sendHeartbeatTimeout) {
14 | callback.onSendHeartbeatTimeout()
15 | ()
16 | }
17 | cancel
18 | }
19 |
20 | override def resetReceiveHeartbeatTimeout(callback: TimerCallback[_]): Cancelable = {
21 | val cancel: Cancelable = sched.scheduleOnce(receivedRandom.next()) {
22 | callback.onReceiveHeartbeatTimeout()
23 | ()
24 | }
25 | cancel
26 | }
27 | }
28 |
29 | object MonixClock {
30 |
31 | def apply()(implicit sched: Scheduler): MonixClock = apply(250.millis, RandomTimer(1.second, 2.seconds))
32 |
33 | def apply(sendHeartbeatTimeout: FiniteDuration, receiveHeartbeatMinMaxTimeout: RandomTimer)(implicit sched: Scheduler): MonixClock = {
34 | new MonixClock(sendHeartbeatTimeout, receiveHeartbeatMinMaxTimeout)
35 | }
36 | }
37 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/ObservableCache.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 |
3 | /**
4 | * Similar to a replay observable
5 | */
6 | class ObservableCache {}
7 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/ObservableState.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.Scheduler
3 | import monix.reactive.Observable
4 | import monix.reactive.subjects.Var
5 | import riff.raft.node.RoleCallback.RoleEvent
6 | import riff.raft.node.RoleCallback
7 |
8 | /**
9 | * Provides a means to represent a [[riff.raft.node.RaftNode]]'s role as an Observable:
10 | *
11 | * {{{
12 | *
13 | * val node : RaftNode[A] = ...
14 | * val obs = ObservableState()
15 | * node.withRoleCallback(obs)
16 | *
17 | * // get events of this node's idea of who the leader is and its role
18 | * obs.subscribe(...)
19 | * }}}
20 | */
21 | class ObservableState(implicit sched: Scheduler) extends RoleCallback {
22 | private val eventsVar: Var[RoleEvent] = Var[RoleEvent](null)
23 | override def onEvent(event: RoleEvent): Unit = {
24 | eventsVar := event
25 | }
26 | def events: Observable[RoleEvent] = eventsVar.filter(_ != null)
27 | }
28 |
29 | object ObservableState {
30 |
31 | def apply()(implicit sched: Scheduler): ObservableState = {
32 | new ObservableState()
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/ObservableTimerCallback.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.{Cancelable, Scheduler}
3 | import monix.reactive.{Observable, Observer}
4 | import monix.reactive.OverflowStrategy.DropOld
5 | import monix.reactive.observers.Subscriber
6 | import monix.reactive.subjects.Var
7 | import riff.raft.NodeId
8 | import riff.raft.messages.{ReceiveHeartbeatTimeout, SendHeartbeatTimeout, TimerMessage}
9 | import riff.raft.timer.TimerCallback
10 |
11 | /**
12 | * Provides a means to observe the nodes timeouts
13 | *
14 | * @param sched
15 | */
16 | class ObservableTimerCallback(implicit sched: Scheduler) extends TimerCallback[Unit] {
17 | private val sendTimeoutsVar = Var[Boolean](false)
18 |
19 | def sendTimeout: Observable[TimerMessage] =
20 | sendTimeoutsVar.filter(identity).map(_ => SendHeartbeatTimeout).asyncBoundary(DropOld(2))
21 | override def onSendHeartbeatTimeout(): Unit = sendTimeoutsVar := true
22 |
23 | private val receiveTimeoutsVar = Var[Boolean](false)
24 |
25 | def receiveTimeouts: Observable[TimerMessage] =
26 | receiveTimeoutsVar.filter(identity).map(_ => ReceiveHeartbeatTimeout).asyncBoundary(DropOld(2))
27 |
28 | override def onReceiveHeartbeatTimeout(): Unit = receiveTimeoutsVar := true
29 |
30 | def subscribe(nodeId: NodeId, subscriber: Observer[TimerMessage]): Cancelable = {
31 | sendTimeout.dump(s"${nodeId} sendTimeout").subscribe(subscriber)
32 | receiveTimeouts.dump(s"${nodeId} receiveTimeouts").subscribe(subscriber)
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/RiffSchedulers.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.{Cancelable, ExecutionModel, Scheduler}
3 | import monix.execution.schedulers.SchedulerService
4 |
5 | import scala.concurrent.{ExecutionContext, Future}
6 | import scala.concurrent.duration.TimeUnit
7 |
8 | object RiffSchedulers {
9 |
10 | object computation {
11 | private val inst = Scheduler.computation()
12 | private def c: SchedulerService = {
13 | require(!inst.isTerminated, "TERMINATED!!!!!")
14 | require(!inst.isShutdown, "SHUT DOWN!!!!!")
15 | inst
16 | }
17 | implicit def scheduler: SchedulerService = new SchedulerService {
18 | override def isShutdown: Boolean = c.isShutdown
19 | override def isTerminated: Boolean = c.isTerminated
20 | override def shutdown(): Unit = {
21 | println(s"""
22 | |
23 | |
24 | |
25 | |
26 | | SHUTTING DOWN THE GLOBAL SCHEDULER!
27 | |
28 | |
29 | |
30 | |
31 | """.stripMargin)
32 | c.shutdown()
33 | }
34 | override def awaitTermination(timeout: Long, unit: TimeUnit, awaitOn: ExecutionContext): Future[Boolean] = {
35 | c.awaitTermination(timeout, unit, awaitOn)
36 | }
37 | override def withExecutionModel(em: ExecutionModel): SchedulerService = {
38 | c.withExecutionModel(em)
39 | }
40 | override def execute(command: Runnable): Unit = {
41 | c.execute(command)
42 | }
43 | override def reportFailure(t: Throwable): Unit = {
44 | c.reportFailure(t)
45 | }
46 | override def scheduleOnce(initialDelay: Long, unit: TimeUnit, r: Runnable): Cancelable = {
47 | c.scheduleOnce(initialDelay, unit, r)
48 | }
49 | override def scheduleWithFixedDelay(initialDelay: Long, delay: Long, unit: TimeUnit, r: Runnable): Cancelable = {
50 | c.scheduleWithFixedDelay(initialDelay, delay, unit, r)
51 | }
52 | override def scheduleAtFixedRate(initialDelay: Long, period: Long, unit: TimeUnit, r: Runnable): Cancelable = {
53 | c.scheduleAtFixedRate(initialDelay, period, unit, r)
54 | }
55 | override def clockRealTime(unit: TimeUnit): Long = {
56 | c.clockRealTime(unit)
57 | }
58 | override def clockMonotonic(unit: TimeUnit): Long = {
59 | c.clockMonotonic(unit)
60 | }
61 | override def executionModel: ExecutionModel = c.executionModel
62 | }
63 | }
64 |
65 | }
66 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/log/AppendOps.scala:
--------------------------------------------------------------------------------
1 | package riff.monix.log
2 | import monix.reactive.Observable
3 | import riff.raft.LogIndex
4 | import riff.raft.log.{LogAppendResult, LogAppendSuccess, LogCoords}
5 |
6 | trait AppendOps[A] {
7 |
8 | /** @return an observable of the appended BUT NOT YET committed entries
9 | */
10 | def appendResults(): Observable[LogAppendResult]
11 |
12 | /** @param index the (one based!) index from which we'd like to read the appended coords
13 | * @return an observable of all appended (not necessarily committed) entries from the given index
14 | */
15 | def appendedCoordsFrom(index: LogIndex): Observable[LogCoords]
16 |
17 | /** @return an observable of the appended BUT NOT YET committed entries from the time of subscription
18 | */
19 | def appendCoords(): Observable[LogCoords] = {
20 | appendResults.flatMap {
21 | case res: LogAppendSuccess =>
22 | if (res.firstIndex.index == res.lastIndex.index) {
23 | Observable.pure(res.firstIndex)
24 | } else {
25 | val coords = (res.firstIndex.index to res.lastIndex.index).map { idx =>
26 | res.firstIndex.copy(index = idx)
27 | }
28 | Observable.fromIterable(coords)
29 | }
30 | case err: Throwable => Observable.raiseError(err)
31 | }
32 | }
33 |
34 | /** @param index
35 | * @return an observable of log entries from the given index
36 | */
37 | def appendedEntriesFrom(index: LogIndex): Observable[(LogCoords, A)] = appendedCoordsFrom(index).flatMap(dataForIndex)
38 |
39 | /** @return an observable of the appended coordinates and data from the time of subscription
40 | */
41 | def appendedEntries(): Observable[(LogCoords, A)] = {
42 | // there's actually a race condition here between being notified of new coords and the 'dataForIndex'.
43 | // if an entry is replace in between the notification and the lookup, we could get fed e.g. one coord of:
44 | // LogCoords(term=2, index=2)
45 | appendCoords().flatMap(dataForIndex)
46 | }
47 |
48 | protected def dataForIndex(coords: LogCoords): Observable[(LogCoords, A)]
49 |
50 | }
51 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/log/CommittedOps.scala:
--------------------------------------------------------------------------------
1 | package riff.monix.log
2 | import monix.reactive.Observable
3 | import riff.raft.LogIndex
4 | import riff.raft.log.LogCoords
5 |
6 | /**
7 | * Represents observable operations which can be performed on a [[riff.raft.log.RaftLog]] for committed log entries
8 | *
9 | * @tparam A the log entry type
10 | */
11 | trait CommittedOps[A] {
12 |
13 | /** @return an observable of committed coordinates from the point of subscription
14 | */
15 | def committedCoords(): Observable[LogCoords]
16 |
17 | /** @param index the (one based!) index from which to observe
18 | * @return an observable of the log entries from a particular index
19 | */
20 | def committedEntriesFrom(index: LogIndex): Observable[(LogCoords, A)] =
21 | committedCoordsFrom(index).flatMap(dataForIndex)
22 |
23 | /** @return an Observable of the committed coords and their values from the moment of subscription
24 | */
25 | def committedEntries(): Observable[(LogCoords, A)] = committedCoords.flatMap(dataForIndex)
26 |
27 | /** @param index the (one based!) index from which we'd like to read the committed coords
28 | * @return an observable of all committed entries from the given index
29 | */
30 | def committedCoordsFrom(index: LogIndex): Observable[LogCoords]
31 |
32 | protected def dataForIndex(coords: LogCoords): Observable[(LogCoords, A)]
33 | }
34 |
--------------------------------------------------------------------------------
/riff-monix/src/main/scala/riff/monix/log/package.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import riff.raft.log.{LogAppendSuccess, LogCoords}
3 |
4 | package object log {
5 |
6 | type LogAppended = LogAppendSuccess
7 |
8 | type LogCommitted = Seq[LogCoords]
9 | }
10 |
--------------------------------------------------------------------------------
/riff-monix/src/test/scala/riff/monix/LowPriorityRiffMonixImplicitsTest.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.reactive.Observable
3 | import riff.reactive.AsPublisher
4 |
5 | class LowPriorityRiffMonixImplicitsTest extends RiffMonixSpec {
6 |
7 | "takeWhileIncludeLast" should {
8 | "complete when the predicate completes but include the first result which returned false" in {
9 |
10 | withScheduler { implicit scheduler =>
11 | val integers: Observable[Int] = Observable.fromIterable(0 to 10)
12 | val asp: AsPublisher[Observable] = AsPublisher[Observable]
13 | val pub = asp.takeWhileIncludeLast(integers)(_ != 4)
14 |
15 | val received = pub.toListL.runSyncUnsafe(testTimeout)
16 | received shouldBe List(0, 1, 2, 3, 4)
17 | }
18 | }
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/riff-monix/src/test/scala/riff/monix/MonixClientTest.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import riff.raft.AppendStatus
3 | import riff.raft.messages.ReceiveHeartbeatTimeout
4 | import riff.raft.timer.LoggedInvocationClock
5 |
6 | import scala.collection.mutable.ListBuffer
7 |
8 | class MonixClientTest extends RiffMonixSpec {
9 |
10 | "MonixClient.append" should {
11 |
12 | "send notifications when the append responses are received" in {
13 |
14 | withScheduler { implicit scheduler =>
15 | implicit val clock = new LoggedInvocationClock
16 |
17 | val fiveNodeCluster = RaftPipeMonix.inMemoryClusterOf[String](5)
18 |
19 | try {
20 |
21 | val head = fiveNodeCluster.values.head
22 | head.input.onNext(ReceiveHeartbeatTimeout)
23 |
24 | val leader = eventually {
25 | fiveNodeCluster.values.find(_.handler.state().isLeader).get
26 | }
27 |
28 | var done = false
29 | val received = ListBuffer[AppendStatus]()
30 | leader.client
31 | .append("XYZ123")
32 | .doOnComplete { () =>
33 | done = true
34 | }
35 | .foreach { x =>
36 | received += x
37 | }
38 |
39 | eventually {
40 | withClue(s"The append stream never completed after having received: ${received.mkString("; ")}") {
41 | done shouldBe true
42 | }
43 | }
44 | } finally {
45 | fiveNodeCluster.values.foreach(_.close())
46 | }
47 | }
48 | }
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/riff-monix/src/test/scala/riff/monix/ObservableTimerCallbackTest.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import monix.execution.Ack
3 | import monix.reactive.{Observable, Observer}
4 | import riff.raft.log.LogCoords
5 | import riff.raft.messages.{RaftMessage, ReceiveHeartbeatTimeout, RequestVote, TimerMessage}
6 | import riff.raft.node.{AddressedRequest, RaftCluster, RaftNode, RaftNodeResult}
7 | import riff.reactive.{ReactivePipe, TestListener}
8 |
9 | import scala.collection.mutable.ListBuffer
10 | import scala.concurrent.Future
11 |
12 | class ObservableTimerCallbackTest extends RiffMonixSpec {
13 |
14 | "ObservableTimerCallback.onReceiveHeartbeatTimeout" should {
15 | "trigger an event to observers" in {
16 |
17 | withScheduler { implicit scheduler =>
18 | val timer = new ObservableTimerCallback
19 | val received = ListBuffer[TimerMessage]()
20 |
21 | timer.subscribe(
22 | "id",
23 | new Observer[TimerMessage] {
24 | override def onNext(elem: TimerMessage): Future[Ack] = {
25 | received += elem
26 | Ack.Continue
27 | }
28 | override def onError(ex: Throwable): Unit = ???
29 | override def onComplete(): Unit = ???
30 | }
31 | )
32 |
33 | // call the method under test
34 | timer.onReceiveHeartbeatTimeout()
35 |
36 | eventually {
37 | received should contain only (ReceiveHeartbeatTimeout)
38 | }
39 | }
40 | }
41 | }
42 | "ObservableTimerCallback" should {
43 |
44 | "publish vote requests to all subscribed when using an ObservableTimerCallback" in {
45 |
46 | withScheduler { implicit scheduler =>
47 | implicit val clock = newClock
48 |
49 | val timer = new ObservableTimerCallback
50 | val node = RaftNode.inMemory[String]("test").withCluster(RaftCluster("a", "b")).withTimerCallback(timer)
51 |
52 | val pipeUnderTest: ReactivePipe[RaftMessage[String], RaftNodeResult[String], Observer, Observable] = RaftPipeMonix.pipeForNode(node)
53 |
54 | try {
55 |
56 | val listener = pipeUnderTest.subscribeWith(new TestListener[RaftNodeResult[String]](10, 100))
57 |
58 | timer.subscribe("pipeUnderTest", pipeUnderTest.input)
59 | timer.onReceiveHeartbeatTimeout()
60 |
61 | eventually {
62 | listener.received.size should be > 0
63 | }
64 |
65 | val List(AddressedRequest(requests)) = listener.received.toList
66 | requests.size shouldBe 2
67 | val terms = requests.map {
68 | case (_, RequestVote(term, LogCoords.Empty)) => term
69 | }
70 | withClue(terms.toString) {
71 | terms.toSet.size shouldBe 1
72 | }
73 |
74 | } finally {
75 | pipeUnderTest.close()
76 | }
77 | }
78 | }
79 | }
80 | }
81 |
--------------------------------------------------------------------------------
/riff-monix/src/test/scala/riff/monix/RiffMonixSpec.scala:
--------------------------------------------------------------------------------
1 | package riff.monix
2 | import java.util.concurrent.TimeUnit
3 |
4 | import monix.execution.schedulers.{ExecutorScheduler, TrampolineExecutionContext}
5 | import monix.execution.{ExecutionModel, Scheduler, UncaughtExceptionReporter}
6 | import org.scalatest.BeforeAndAfterAll
7 | import org.scalatest.concurrent.Eventually
8 | import riff.RiffThreadedSpec
9 | import riff.raft.timer.RandomTimer
10 |
11 | import scala.concurrent.duration._
12 |
13 | abstract class RiffMonixSpec extends RiffThreadedSpec with Eventually with BeforeAndAfterAll with LowPriorityRiffMonixImplicits {
14 |
15 | /**
16 | * How long to wait for something NOT to happen - use sparingly!
17 | *
18 | * @return some amount of time to wait before ensuring e.g. no messages are received
19 | */
20 | def testNegativeTimeout: FiniteDuration = 100.millis
21 |
22 | def withScheduler[T](f: Scheduler => T): T = {
23 |
24 | val retVal = withExecCtxt { implicit execCtxt =>
25 | // some tests explicitly throw exceptions, so we should just write these down
26 | val reporter = UncaughtExceptionReporter.LogExceptionsToStandardErr
27 | val executorScheduler: ExecutorScheduler = ExecutorScheduler(execCtxt, reporter, ExecutionModel.SynchronousExecution)
28 | try {
29 | f(executorScheduler)
30 | } finally {
31 | executorScheduler.shutdown()
32 | executorScheduler.awaitTermination(testTimeout.toMillis, TimeUnit.MILLISECONDS, TrampolineExecutionContext.immediate).futureValue
33 | }
34 | }
35 |
36 | retVal
37 | }
38 |
39 | def newClock(implicit s: Scheduler): MonixClock = {
40 | MonixClock(100.millis, RandomTimer(300.millis, 500.millis))(s)
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/riff-runtime/src/main/scala/riff/runtime/RiffApp.scala:
--------------------------------------------------------------------------------
1 | package riff.runtime
2 |
3 | object RiffApp {
4 |
5 | def main(a: Array[String]): Unit = {
6 | val port = a.headOption.map(_.toInt).getOrElse(8080)
7 |
8 | }
9 |
10 | }
11 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/resources/logback.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 | RIFF: %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
6 |
7 |
8 |
9 |
10 |
11 |
12 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/WebFrameEndpoint.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx
2 |
3 | import java.util.concurrent.atomic.AtomicBoolean
4 |
5 | import com.typesafe.scalalogging.StrictLogging
6 | import io.vertx.core.Handler
7 | import io.vertx.scala.core.http.{WebSocketBase, WebSocketFrame}
8 | import monix.execution.Scheduler
9 | import monix.reactive.{Observable, Pipe}
10 | import riff.web.vertx.server.WebSocketFrameAsWebFrame
11 | import streaming.api.sockets.WebFrame
12 |
13 | import scala.concurrent.duration.Duration
14 |
15 | object WebFrameEndpoint extends StrictLogging {
16 |
17 | def apply(name: String, socket: WebSocketBase)(implicit timeout: Duration, scheduler: Scheduler): (WebSocketObserver, Observable[WebFrame]) = {
18 |
19 | val (frameSink, frameSource: Observable[WebFrame]) = Pipe.replay[WebFrame].multicast
20 |
21 | val observable = WebSocketObserver(name, socket)
22 |
23 | val completed = new AtomicBoolean(false)
24 |
25 | def markComplete() = {
26 | if (completed.compareAndSet(false, true)) {
27 | frameSink.onComplete()
28 | observable.onComplete()
29 | } else {
30 | logger.warn("frame sink already completed")
31 | }
32 | }
33 |
34 | socket.frameHandler(new Handler[WebSocketFrame] {
35 | override def handle(event: WebSocketFrame): Unit = {
36 | if (event.isClose()) {
37 | logger.debug(s"$name handling close frame")
38 | markComplete()
39 | } else {
40 | val frame = WebSocketFrameAsWebFrame(event)
41 | logger.debug(s"$name handling frame ${frame}")
42 | frameSink.onNext(frame)
43 | // TODO - we should apply back-pressure, but also not block the event loop.
44 | // need to apply some thought here if this can work in the general case,
45 | // of if this should be made more explicit
46 | //Await.result(fut, timeout)
47 | }
48 | }
49 | })
50 |
51 | socket.exceptionHandler(new Handler[Throwable] {
52 | override def handle(event: Throwable): Unit = {
53 | logger.warn(s"$name got exception $event")
54 | frameSink.onError(event)
55 | observable.onError(event)
56 | socket.close()
57 | }
58 | })
59 | socket.endHandler(new Handler[Unit] {
60 | override def handle(event: Unit): Unit = {
61 | logger.debug(s"$name ending")
62 | markComplete()
63 | }
64 | })
65 |
66 | val source = frameSource
67 | .doOnComplete { () =>
68 | logger.debug(s"\n>>> $name onComplete called\n")
69 |
70 | }
71 | .doOnError { err =>
72 | logger.debug(s"\n>>> $name onError($err) called\n")
73 | }
74 | .doOnNext { x =>
75 | logger.debug(s"\n>>> $name onNext($x) called\n")
76 | }
77 |
78 | (observable, source)
79 | }
80 | }
81 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/WebSocketObserver.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx
2 |
3 | import java.util.concurrent.atomic.AtomicBoolean
4 |
5 | import com.typesafe.scalalogging.LazyLogging
6 | import io.vertx.core.buffer.Buffer
7 | import io.vertx.scala.core.http.WebSocketBase
8 | import monix.execution.Ack
9 | import monix.execution.Ack.Continue
10 | import monix.reactive.Observer
11 | import streaming.api.sockets._
12 |
13 | import scala.concurrent.Future
14 | import scala.util.control.NonFatal
15 |
16 | private[vertx] final case class WebSocketObserver(name: String, socket: WebSocketBase) extends Observer[WebFrame] with LazyLogging {
17 | override def onNext(elem: WebFrame): Future[Ack] = elem match {
18 | case TextFrame(text) =>
19 | logger.debug(s"$name writing to socket: $text")
20 | socket.writeTextMessage(text)
21 | Continue
22 | case BinaryFrame(data) =>
23 | logger.debug(s"$name writing bytes to socket")
24 | val buff: Buffer = io.vertx.core.buffer.Buffer.buffer(data.array)
25 | socket.writeBinaryMessage(buff)
26 | Continue
27 | case FinalTextFrame(text) =>
28 | logger.debug(s"$name writing final text to socket: $text")
29 | socket.writeFinalTextFrame(text)
30 | Continue
31 | case FinalBinaryFrame(data) =>
32 | logger.debug(s"$name writing final binary frame")
33 | val buff: Buffer = io.vertx.core.buffer.Buffer.buffer(data.array)
34 | socket.writeFinalBinaryFrame(buff)
35 | Continue
36 | case CloseFrame(statusCode, reason) =>
37 | logger.debug(s"$name writing close frame to socket w/ status $statusCode, reason $reason")
38 | socket.close(statusCode, reason)
39 | Continue
40 | //Stop
41 | }
42 |
43 | override def onError(ex: Throwable): Unit = {
44 | val ok = completed.compareAndSet(false, true)
45 |
46 | logger.debug(s"\n\t!!!! $name onError trying to close the socket will ${if (ok) "succeed" else "fail"}")
47 |
48 | try {
49 | if (ok) {
50 | logger.debug(s"$name onError($ex) closing the socket")
51 | socket.close(500, Option(s"Error: $ex"))
52 | } else {
53 | logger.warn(s"onError($ex) has not effect on the closed socket")
54 | }
55 | } catch {
56 | case NonFatal(e) =>
57 | logger.error(s"Error ending socket connected to ${socket.remoteAddress()} after error $ex", e)
58 | }
59 | }
60 |
61 | private val completed = new AtomicBoolean(false)
62 |
63 | override def onComplete(): Unit = {
64 |
65 | val ok = completed.compareAndSet(false, true)
66 | logger.debug(s"\n\t!!!! $name onComplete trying to close the socket will ${if (ok) "succeed" else "fail"}")
67 |
68 | try {
69 | if (ok) {
70 | socket.end()
71 | } else {
72 | logger.warn(s"$name onComplete has no effect as the socket it already closed")
73 | }
74 | } catch {
75 | case NonFatal(e) =>
76 | logger.error(s"$name Error ending socket connected to ${socket.remoteAddress()}", e)
77 | }
78 | }
79 | }
80 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/client/RestClient.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.client
2 |
3 | import com.typesafe.scalalogging.StrictLogging
4 | import io.vertx.core.buffer.Buffer
5 | import io.vertx.lang.scala.ScalaVerticle
6 | import io.vertx.scala.core.Vertx
7 | import io.vertx.scala.core.http.{HttpClient, HttpClientRequest, HttpClientResponse}
8 | import monix.execution.Scheduler
9 | import monix.reactive.subjects.Var
10 | import monix.reactive.{Observable, Pipe}
11 | import streaming.api.HostPort
12 | import streaming.rest.{HttpMethod, RestInput, RestResponse}
13 |
14 | object RestClient {
15 | def connect(location: HostPort)(implicit scheduler: Scheduler): RestClient = RestClient(location)
16 | }
17 |
18 | case class RestClient(location: HostPort, impl: Vertx = Vertx.vertx())(implicit scheduler: Scheduler) extends ScalaVerticle with StrictLogging {
19 | vertx = impl
20 | val httpClient: HttpClient = vertx.createHttpClient
21 |
22 | val sendPipe: Pipe[RestInput, RestResponse] = Pipe.publishToOne[RestInput].transform { restInputs: Observable[RestInput] =>
23 | restInputs.flatMapDelayErrors(send).doOnTerminate { errOpt =>
24 | logger.error(s"stopping client connected to $location ${errOpt.fold("")("on error " + _)} ")
25 | stop()
26 | }
27 | }
28 |
29 | start()
30 |
31 | def send(req: RestInput): Observable[RestResponse] = {
32 | logger.debug(s"Sending $req to $location")
33 |
34 | req.uri.resolve(req.headers) match {
35 | case Left(bad) =>
36 | Observable.raiseError(new IllegalArgumentException(s"Request for ${req.uri} didn't resolve given the path parts: $bad"))
37 | case Right(parts) =>
38 | val uri = parts.mkString("/")
39 |
40 | val httpRequest: HttpClientRequest = req.uri.method match {
41 | case HttpMethod.GET => httpClient.get(location.port, location.host, uri)
42 | case HttpMethod.POST => httpClient.post(location.port, location.host, uri)
43 | case HttpMethod.PUT => httpClient.put(location.port, location.host, uri)
44 | case HttpMethod.DELETE => httpClient.delete(location.port, location.host, uri)
45 | case HttpMethod.HEAD => httpClient.head(location.port, location.host, uri)
46 | case HttpMethod.OPTIONS => httpClient.options(location.port, location.host, uri)
47 | case _ => null
48 | }
49 |
50 | val responseVar = Var[RestResponse](null)
51 |
52 | httpRequest.handler { event: HttpClientResponse =>
53 | logger.debug(s"Handling response from $uri : ${event.request().uri()}")
54 | event.bodyHandler { body =>
55 | val headers: Map[String, String] = {
56 | val headersMultiMap = event.headers()
57 | headersMultiMap.names.map { name =>
58 | name -> headersMultiMap.getAll(name).mkString(",")
59 | }.toMap
60 | }
61 |
62 | logger.debug(s"Setting response var from $uri ")
63 | responseVar := RestResponse(body.getBytes, headers, event.statusCode())
64 | }
65 | }
66 |
67 | if (httpRequest == null) {
68 | Observable.raiseError(new UnsupportedOperationException(s"Unsupported method ${req.uri.method}"))
69 | } else {
70 | val builtRequest = req.headers.foldLeft(httpRequest.setChunked(true).write(Buffer.buffer(req.bodyAsBytes))) {
71 | case (r, (key, value)) => r.putHeader(key, value)
72 | }
73 |
74 | builtRequest.end()
75 |
76 | responseVar.filter(_ != null).take(1)
77 | }
78 | }
79 | }
80 | }
81 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/client/SocketClient.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.client
2 |
3 | import java.util.concurrent.atomic.AtomicInteger
4 |
5 | import com.typesafe.scalalogging.StrictLogging
6 | import io.vertx.core.Handler
7 | import io.vertx.lang.scala.ScalaVerticle
8 | import io.vertx.scala.core.Vertx
9 | import io.vertx.scala.core.http.{HttpClient, WebSocket}
10 | import monix.execution.Scheduler
11 | import riff.web.vertx.WebFrameEndpoint
12 | import streaming.api.Endpoint
13 | import streaming.api.sockets.WebFrame
14 | import streaming.rest.EndpointCoords
15 |
16 | import scala.concurrent.duration.Duration
17 |
18 | class SocketClient private (coords: EndpointCoords, client: Handler[WebSocket], impl: Vertx) extends ScalaVerticle {
19 | vertx = impl
20 |
21 | val httpsClient: HttpClient = vertx.createHttpClient.websocket(coords.port, host = coords.host, coords.resolvedUri, client)
22 |
23 | start()
24 | }
25 |
26 | object SocketClient {
27 |
28 | def connect(coords: EndpointCoords, name: String = null)(
29 | onConnect: Endpoint[WebFrame, WebFrame] => Unit)(implicit timeout: Duration, scheduler: Scheduler, vertx: Vertx): SocketClient = {
30 | val counter = new AtomicInteger(0)
31 | val handler = new Handler[WebSocket] with StrictLogging {
32 | override def handle(event: WebSocket): Unit = {
33 | val nonNullName = Option(name).getOrElse(s"SocketClient to $coords") + s"#${counter.incrementAndGet()}"
34 | logger.info(s"$nonNullName connected to socket")
35 | val (fromRemote, toRemote) = WebFrameEndpoint(nonNullName, event)
36 | onConnect(Endpoint(fromRemote, toRemote))
37 | }
38 | }
39 |
40 | new SocketClient(coords, handler, vertx)
41 |
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/server/Main.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.server
2 | import java.nio.file.Path
3 |
4 | import com.typesafe.scalalogging.StrictLogging
5 | import eie.io._
6 | import io.vertx.scala.core.{Vertx, VertxOptions}
7 | import monix.execution.schedulers.SchedulerService
8 | import monix.reactive.Observable
9 | import riff.monix.{ObservableRaftEndpoint, MonixClock, RiffSchedulers}
10 | import riff.raft.node.RaftCluster
11 | import riff.raft.{AppendStatus, NodeId}
12 | import riff.web.vertx.client.SocketClient
13 | import streaming.api.HostPort
14 |
15 | import scala.concurrent.duration._
16 |
17 | object Main extends StrictLogging {
18 |
19 | def main(args: Array[String]): Unit = {
20 | val running = args match {
21 | case Array(name) if clusterNamesForSize(5).contains(name) => Started(name, 5)
22 | case Array(name, sizeStr) if clusterNamesForSize(sizeStr.toInt).contains(name) => Started(name, sizeStr.toInt)
23 | case other => sys.error(s"Usage: Expected the name and an optional cluster size (e.g. one of ${clusterNamesForSize(3)}, 3), but got '${other.mkString(" ")}'")
24 | }
25 |
26 | running.close()
27 | logger.info("Goodbye!")
28 | }
29 |
30 | def clusterNamesForSize(size: Int) = (1 to size).map(i => s"node$i").toSet
31 |
32 | def portForName(name: String) = 8000 + name.filter(_.isDigit).toInt
33 |
34 | case class Started(name: String, clusterSize: Int) extends AutoCloseable {
35 | implicit val scheduler: SchedulerService = RiffSchedulers.computation.scheduler
36 | implicit val socketTimeout = 1.minute
37 | implicit val clock = MonixClock()
38 | val port = portForName(name)
39 | val dataDir: Path = name.asPath.resolve("target/.data")
40 | val hostPort: HostPort = HostPort.localhost(port)
41 | val cluster: RaftCluster.Fixed = RaftCluster(clusterNamesForSize(5) - name)
42 | implicit val vertx = Vertx.vertx(VertxOptions().setWorkerPoolSize(clusterSize + 1).setEventLoopPoolSize(4).setInternalBlockingPoolSize(4))
43 |
44 | val builder: ObservableRaftEndpoint[String] = ObservableRaftEndpoint[String](name, dataDir, cluster)
45 |
46 | // eagerly create the pipe
47 | builder.pipe
48 | logger.info(s"Connecting to peers...")
49 | val clients: Map[NodeId, SocketClient] = Startup.connectToPeers(builder)
50 | logger.info(s"Starting the server on ${hostPort} for $cluster")
51 | val verticle = Startup.startServer(builder, hostPort)
52 | logger.info(s"Handling messages on ${hostPort} for $cluster")
53 |
54 | builder.timerCallback.receiveTimeouts.foreach { _ => //
55 | logger.info(s"${name} got received HB timeout")
56 | }
57 | builder.timerCallback.sendTimeout.foreach { _ => //
58 | logger.info(s"${name} got send HB timeout")
59 | }
60 | builder.stateCallback.events.foreach { event => //
61 | logger.info(s"${name} noticed $event")
62 | }
63 | builder.log.appendResults().foreach { event => //
64 | logger.info(s"${name} log appended $event")
65 | }
66 | builder.log.committedEntries().foreach { event => //
67 | logger.info(s"${name} log committed $event")
68 | }
69 |
70 | builder.resetReceiveHeartbeat()
71 |
72 | def client = builder.pipe.client
73 | override def close(): Unit = {
74 | builder.cancelHeartbeats()
75 | scheduler.shutdown()
76 | verticle.stop()
77 | clients.values.foreach(_.stop())
78 | }
79 | }
80 |
81 | }
82 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/server/RoutingSocketHandler.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.server
2 |
3 | import io.vertx.core.Handler
4 | import io.vertx.scala.core.http.ServerWebSocket
5 | import RoutingSocketHandler.NotFoundHandler
6 |
7 | case class RoutingSocketHandler(byRoute: PartialFunction[String, Handler[ServerWebSocket]], notFound: Handler[ServerWebSocket] = NotFoundHandler) extends Handler[ServerWebSocket] {
8 | override def handle(event: ServerWebSocket): Unit = {
9 | val path = event.uri()
10 | val h = if (byRoute.isDefinedAt(path)) {
11 | byRoute(path)
12 | } else {
13 | notFound
14 | }
15 |
16 | h.handle(event)
17 | }
18 | }
19 |
20 | object RoutingSocketHandler {
21 |
22 | object NotFoundHandler extends Handler[ServerWebSocket] {
23 | override def handle(event: ServerWebSocket): Unit = {
24 | event.reject(404)
25 | }
26 | }
27 |
28 | }
29 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/server/ServerEndpoint.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.server
2 |
3 | import com.typesafe.scalalogging.StrictLogging
4 | import io.vertx.scala.core.http.ServerWebSocket
5 | import monix.execution.Scheduler
6 | import monix.reactive.{Observable, Observer}
7 | import riff.web.vertx.WebFrameEndpoint
8 | import streaming.api.Endpoint
9 | import streaming.api.sockets.WebFrame
10 |
11 | import scala.concurrent.duration.Duration
12 |
13 | /**
14 | * A specialised endpoint which retains a reference to the socket to which is it connected,
15 | * which can be queried for e.g. the uri, query string, etc
16 | *
17 | * @param socket
18 | * @param from
19 | * @param to
20 | */
21 | final class ServerEndpoint(val socket: ServerWebSocket, to: Observer[WebFrame], from: Observable[WebFrame]) extends Endpoint[WebFrame, WebFrame](to, from)
22 |
23 | object ServerEndpoint extends StrictLogging {
24 |
25 | def publish(socket: ServerWebSocket, name: String)(implicit timeout: Duration, scheduler: Scheduler): ServerEndpoint = {
26 | val addr = {
27 | val a = socket.remoteAddress()
28 | val url = s"${a.host}:${a.port}/${a.path}"
29 | s"$name (socket connected to $url)"
30 | }
31 | logger.info(s"$addr Accepting connection")
32 | val (obs, frameSource) = WebFrameEndpoint(addr, socket)
33 | socket.accept()
34 | new ServerEndpoint(socket, obs, frameSource)
35 | }
36 |
37 | }
38 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/server/ServerWebSocketHandler.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.server
2 |
3 | import io.vertx.core.Handler
4 | import io.vertx.scala.core.http.ServerWebSocket
5 | import monix.execution.Scheduler
6 |
7 | import scala.concurrent.duration.Duration
8 |
9 | private[server] class ServerWebSocketHandler private (name: String, onConnect: ServerEndpoint => Unit)(implicit timeout: Duration, scheduler: Scheduler)
10 | extends Handler[ServerWebSocket] {
11 | override def handle(socket: ServerWebSocket): Unit = {
12 | onConnect(ServerEndpoint.publish(socket, name))
13 | }
14 | }
15 |
16 | object ServerWebSocketHandler {
17 |
18 | def replay(name: String)(onConnect: ServerEndpoint => Unit)(implicit timeout: Duration, scheduler: Scheduler): ServerWebSocketHandler = {
19 | new ServerWebSocketHandler(name, onConnect)
20 | }
21 | }
22 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/riff/web/vertx/server/WebSocketFrameAsWebFrame.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.server
2 |
3 | import io.vertx.scala.core.http.WebSocketFrame
4 | import streaming.api.sockets.WebFrame
5 |
6 | final object WebSocketFrameAsWebFrame {
7 |
8 | def apply(vertxFrame: WebSocketFrame): WebFrame = {
9 | if (vertxFrame.isText()) {
10 | if (vertxFrame.isFinal()) {
11 | WebFrame.finalText(vertxFrame.textData())
12 | } else {
13 | WebFrame.text(vertxFrame.textData())
14 | }
15 | } else {
16 | if (vertxFrame.isFinal()) {
17 | WebFrame.finalBinary(vertxFrame.binaryData().getBytes)
18 | } else {
19 | WebFrame.binary(vertxFrame.binaryData().getBytes)
20 | }
21 | }
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/api/Endpoint.scala:
--------------------------------------------------------------------------------
1 | package streaming.api
2 |
3 | import monix.execution.{Cancelable, Scheduler}
4 | import monix.reactive.{Observable, Observer, Pipe}
5 | import streaming.api.sockets.WebFrame
6 |
7 | /**
8 | * Represents a connection to another system, either local or remote.
9 | *
10 | * Just a specialised kind of Pipe really which has typed inputs/outputs.
11 | *
12 | * @param toRemote an observer which can be attached to an Observable of events which should be sent to the endpoint
13 | * @param fromRemote an observable of messages coming from the endpoint
14 | * @tparam FromRemote
15 | * @tparam ToRemote
16 | */
17 | class Endpoint[FromRemote, ToRemote](val toRemote: Observer[ToRemote], val fromRemote: Observable[FromRemote]) {
18 | final def map[A](f: FromRemote => A): Endpoint[A, ToRemote] = new Endpoint[A, ToRemote](toRemote, fromRemote.map(f))
19 | final def contraMap[A](f: A => ToRemote): Endpoint[FromRemote, A] = {
20 | import streaming.api.implicits._
21 | new Endpoint[FromRemote, A](toRemote.contraMap(f), fromRemote)
22 | }
23 |
24 | def handleTextFramesWith(f: Observable[String] => Observable[String])(implicit scheduler: Scheduler, fromEv: FromRemote =:= WebFrame, toEv: ToRemote =:= WebFrame): Cancelable = {
25 | val fromText = fromRemote.map { from =>
26 | fromEv(from).asText.getOrElse(sys.error("Received non-text frame"))
27 | }
28 | val replies: Observable[String] = f(fromText)
29 | replies.map[WebFrame](WebFrame.text).subscribe(toRemote.asInstanceOf[Observer[WebFrame]])
30 | }
31 |
32 | }
33 |
34 | object Endpoint {
35 |
36 | def instance[From, To](implicit endpoint: Endpoint[From, To]): Endpoint[From, To] = endpoint
37 |
38 | def apply[From, To](toRemote: Observer[To], fromRemote: Observable[From]): Endpoint[From, To] = {
39 | new Endpoint[From, To](toRemote, fromRemote)
40 | }
41 |
42 | def replay[T](initial: Seq[T]): Endpoint[T, T] = {
43 | val (to, from) = Pipe.replay[T](initial).unicast
44 | Endpoint(to, from)
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/api/HostPort.scala:
--------------------------------------------------------------------------------
1 | package streaming.api
2 |
3 | case class HostPort(host: String, port: Int, secure: Boolean = false) {
4 | def hostPort = s"$host:$port"
5 |
6 | override def toString = hostPort
7 |
8 | /**
9 | * @return 'localhost:port' if the host is set to 0.0.0.0 then use, this instance otherwise
10 | */
11 | def resolveLocalhost = host match {
12 | case "0.0.0.0" => HostPort("localhost", port)
13 | case _ => this
14 | }
15 |
16 | def asURL = s"http://$hostPort"
17 |
18 | def asWebsocketURL = {
19 | if (secure) {
20 | s"wss://${hostPort}"
21 | } else {
22 | s"ws://${hostPort}"
23 | }
24 | }
25 | }
26 |
27 | object HostPort {
28 |
29 | def local(port: Int): HostPort = HostPort("0.0.0.0", port)
30 | def localhost(port: Int): HostPort = HostPort("localhost", port)
31 |
32 | private val HostPortR = "(.*):(\\d+)".r
33 |
34 | def unapply(id: String): Option[HostPort] = {
35 | id match {
36 | case HostPortR(h, p) => Option(HostPort(h, p.toInt))
37 | case _ => None
38 | }
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/api/implicits/LowPriorityStreamingImplicits.scala:
--------------------------------------------------------------------------------
1 | package streaming.api.implicits
2 |
3 | import monix.execution.Ack
4 | import monix.execution.Ack.Stop
5 | import monix.reactive.Observer
6 | import streaming.api.implicits.LowPriorityStreamingImplicits.RichObserver
7 |
8 | import scala.concurrent.Future
9 | import scala.util.control.NonFatal
10 |
11 | trait LowPriorityStreamingImplicits {
12 |
13 | implicit def asRichObserver[T](obs: Observer[T]) = new RichObserver[T](obs)
14 |
15 | }
16 |
17 | object LowPriorityStreamingImplicits {
18 |
19 | class RichObserver[T](val obs: Observer[T]) extends AnyVal {
20 |
21 | def contraMap[A](f: A => T): Observer[A] = new Observer[A] {
22 | override def onNext(elem: A): Future[Ack] = obs.onNext(f(elem))
23 |
24 | override def onError(ex: Throwable): Unit = obs.onError(ex)
25 |
26 | override def onComplete(): Unit = obs.onComplete()
27 | }
28 |
29 | def contraMapP[A](f: PartialFunction[A, T]): Observer[A] = contraMapUnsafe {
30 | case a if f.isDefinedAt(a) => f(a)
31 | case a => sys.error(s"contraMap is undefined for $a")
32 | }
33 |
34 | def contraMapUnsafe[A](f: A => T): Observer[A] = new Observer[A] {
35 | override def onNext(elem: A): Future[Ack] = {
36 | try {
37 | obs.onNext(f(elem))
38 | } catch {
39 | case NonFatal(err) =>
40 | onError(err)
41 | Stop
42 | }
43 | }
44 |
45 | override def onError(ex: Throwable): Unit = obs.onError(ex)
46 |
47 | override def onComplete(): Unit = obs.onComplete()
48 | }
49 | }
50 |
51 | }
52 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/api/implicits/package.scala:
--------------------------------------------------------------------------------
1 | package streaming.api
2 |
3 | package object implicits extends LowPriorityStreamingImplicits
4 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/api/sockets/WebFrame.scala:
--------------------------------------------------------------------------------
1 | package streaming.api.sockets
2 |
3 | import java.nio.ByteBuffer
4 |
5 | sealed trait WebFrame {
6 | def asText: Option[String]
7 | def asBinary: Option[ByteBuffer]
8 | }
9 |
10 | object WebFrame {
11 | def text(text: String) = TextFrame(text)
12 |
13 | def finalText(text: String) = FinalTextFrame(text)
14 |
15 | def finalBinary(data: Array[Byte]): FinalBinaryFrame = finalBinary(ByteBuffer.wrap(data))
16 |
17 | def finalBinary(data: ByteBuffer): FinalBinaryFrame = FinalBinaryFrame(data)
18 |
19 | def binary(data: Array[Byte]): BinaryFrame = binary(ByteBuffer.wrap(data))
20 |
21 | def binary(data: ByteBuffer): BinaryFrame = BinaryFrame(data)
22 |
23 | def close(statusCode: Short, reason: Option[String] = None) = CloseFrame(statusCode, reason)
24 | }
25 |
26 | final case class TextFrame(text: String) extends WebFrame {
27 | override def asText = Option(text)
28 | override def asBinary = None
29 | }
30 |
31 | final case class BinaryFrame(data: ByteBuffer) extends WebFrame {
32 | override def asText = None
33 | override def asBinary = Option(data)
34 | }
35 |
36 | final case class FinalTextFrame(text: String) extends WebFrame {
37 | override def asText = Option(text)
38 | override def asBinary = None
39 | }
40 |
41 | final case class FinalBinaryFrame(data: ByteBuffer) extends WebFrame {
42 | override def asText = None
43 | override def asBinary = Option(data)
44 | }
45 |
46 | final case class CloseFrame(statusCode: Short, reason: Option[String]) extends WebFrame {
47 | override def asText = None
48 | override def asBinary = None
49 | }
50 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/rest/EndpointCoords.scala:
--------------------------------------------------------------------------------
1 | package streaming.rest
2 |
3 | import streaming.api.HostPort
4 |
5 | /**
6 | * Represents an endpoint which may be targeted by a client.
7 | *
8 | * @param location the host/port of the service
9 | * @param uri the uri to target
10 | * @param params a simple map of params. Some may be applied to fill in the uri, and the rest would presumably
11 | * be used to construct query params
12 | */
13 | case class EndpointCoords(location: HostPort, uri: WebURI, params: Map[String, String]) {
14 |
15 | val resolvedUri: String = uri.resolve(params) match {
16 | case Left(err) => sys.error(err)
17 | case Right(path) => path.mkString("/", "/", "")
18 | }
19 |
20 | def host = location.host
21 |
22 | def port = location.port
23 |
24 | def hostPort = location.hostPort
25 | }
26 |
27 | object EndpointCoords {
28 |
29 | def get(hostPort: HostPort, uri: String, params: Map[String, String] = Map.empty): EndpointCoords =
30 | EndpointCoords(hostPort, WebURI(HttpMethod.GET, uri), params)
31 |
32 | def post(hostPort: HostPort, uri: String, params: Map[String, String] = Map.empty): EndpointCoords =
33 | EndpointCoords(hostPort, WebURI(HttpMethod.POST, uri), params)
34 |
35 | def put(hostPort: HostPort, uri: String, params: Map[String, String] = Map.empty): EndpointCoords =
36 | EndpointCoords(hostPort, WebURI(HttpMethod.PUT, uri), params)
37 |
38 | def delete(hostPort: HostPort, uri: String, params: Map[String, String] = Map.empty): EndpointCoords =
39 | EndpointCoords(hostPort, WebURI(HttpMethod.DELETE, uri), params)
40 |
41 | def head(hostPort: HostPort, uri: String, params: Map[String, String] = Map.empty): EndpointCoords =
42 | EndpointCoords(hostPort, WebURI(HttpMethod.HEAD, uri), params)
43 |
44 | def apply(hostPort: HostPort, uri: WebURI, params: (String, String)*): EndpointCoords = EndpointCoords(hostPort, uri, params.toMap)
45 | }
46 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/rest/HttpMethod.scala:
--------------------------------------------------------------------------------
1 | package streaming.rest
2 |
3 | sealed trait HttpMethod
4 |
5 | object HttpMethod {
6 |
7 | def unapply(name: String): Option[HttpMethod] = {
8 | values.find(_.toString.equalsIgnoreCase(name))
9 | }
10 |
11 | case object PUT extends HttpMethod
12 |
13 | case object POST extends HttpMethod
14 |
15 | case object DELETE extends HttpMethod
16 |
17 | case object GET extends HttpMethod
18 |
19 | case object HEAD extends HttpMethod
20 |
21 | case object OPTIONS extends HttpMethod
22 |
23 | case object TRACE extends HttpMethod
24 |
25 | case object CONNECT extends HttpMethod
26 |
27 | case object PATCH extends HttpMethod
28 |
29 | lazy val values = Set(PUT, POST, DELETE, GET, HEAD, OPTIONS, TRACE, CONNECT, PATCH)
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/rest/RestInput.scala:
--------------------------------------------------------------------------------
1 | package streaming.rest
2 | import eie.io.ToBytes
3 |
4 | sealed trait RestInput {
5 | def uri: WebURI
6 |
7 | def headers: Map[String, String]
8 |
9 | def bodyAsBytes: Array[Byte] = this match {
10 | case content: RestInput.ContentInput[_] => content.bytes
11 | case _ => Array.empty[Byte]
12 | }
13 | }
14 |
15 | object RestInput {
16 | def apply(uri: WebURI, headers: Map[String, String] = Map.empty) = BasicInput(uri, headers)
17 |
18 | def apply[A: ToBytes](uri: WebURI, body: A, headers: Map[String, String]) = ContentInput(uri, body, headers)
19 |
20 | case class BasicInput(uri: WebURI, headers: Map[String, String]) extends RestInput
21 |
22 | case class ContentInput[A: ToBytes](uri: WebURI, body: A, headers: Map[String, String]) extends RestInput {
23 | def bytes: Array[Byte] = ToBytes[A].bytes(body)
24 | }
25 | }
26 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/rest/RestRequest.scala:
--------------------------------------------------------------------------------
1 | package streaming.rest
2 |
3 | import com.typesafe.scalalogging.LazyLogging
4 | import monix.reactive.Observer
5 |
6 | case class RestRequestContext(request: RestRequest, response: Observer[RestResponse]) extends LazyLogging {
7 |
8 | def completeWith(resp: RestResponse): Unit = {
9 | logger.debug(s"completing ${request.uri} with $resp")
10 | response.onNext(resp)
11 | response.onComplete()
12 | }
13 | }
14 |
15 | case class RestRequest(method: HttpMethod, uri: String, body: Array[Byte], headers: Map[String, String]) {
16 | def bodyAsString = new String(body)
17 |
18 | override def toString = s"${method} $uri ${headers.mkString("[", "; ", "]")} { $bodyAsString } "
19 | }
20 |
21 | case class RestResponse(body: Array[Byte], headers: Map[String, String], statusCode: Int = 200) {
22 | def bodyAsString = new String(body)
23 | override def toString = s"RestResponse(statusCode=$statusCode, headers=$headers, body=$bodyAsString)"
24 | }
25 |
26 | object RestResponse {
27 | def json(body: String): RestResponse = RestResponse(body.getBytes("UTF-8"), Map("content-type" -> "application/json"))
28 | def text(body: String): RestResponse = RestResponse(body.getBytes("UTF-8"), Map("content-type" -> "text/plain"))
29 | }
30 |
--------------------------------------------------------------------------------
/riff-vertx/src/main/scala/streaming/rest/StreamingConfig.scala:
--------------------------------------------------------------------------------
1 | package streaming.rest
2 |
3 | import com.typesafe.config.{Config, ConfigFactory}
4 | import com.typesafe.scalalogging.StrictLogging
5 | import monix.execution.Scheduler
6 | import monix.execution.schedulers.SchedulerService
7 | import streaming.api.HostPort
8 |
9 | case class StreamingConfig(config: Config) {
10 | val hostPort: HostPort = HostPort(config.getString("host"), config.getInt("port"))
11 | val staticPath: Option[String] = Option(config.getString("staticPath")).filterNot(_.isEmpty)
12 |
13 | lazy val computeScheduler: SchedulerService = Scheduler.computation()
14 | lazy val ioScheduler: SchedulerService = Scheduler.io()
15 | }
16 |
--------------------------------------------------------------------------------
/riff-vertx/src/test/scala/riff/web/vertx/client/RestClientTest.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.client
2 |
3 | import monix.execution.Scheduler.Implicits.global
4 | import monix.reactive.Observable
5 | import org.scalatest.concurrent.Eventually
6 | import riff.RiffSpec
7 | import riff.web.vertx.server.Server
8 | import streaming.api.HostPort
9 | import streaming.rest._
10 |
11 | import scala.concurrent.duration.{FiniteDuration, _}
12 | import scala.util.Try
13 | import scala.util.control.NonFatal
14 |
15 | class RestClientTest extends RiffSpec with Eventually {
16 |
17 | override implicit def testTimeout: FiniteDuration = 30.seconds
18 |
19 | "RestClient.send" should {
20 | "send and receive values" in {
21 | val found = (7000 to 7010).find { port =>
22 | val Index = WebURI.get("/index.html")
23 |
24 | val (server, requests: Observable[RestRequestContext]) = Server.startRest(HostPort.localhost(port), None)
25 |
26 | requests.foreach { ctxt => //
27 | ctxt.completeWith(RestResponse.json(s"""{ "msg" : "handled ${ctxt.request.uri}" }"""))
28 | }
29 | val client: RestClient = RestClient.connect(HostPort.localhost(port))
30 |
31 | try {
32 | val response: Observable[RestResponse] = client.send(RestInput(Index))
33 | val reply: List[RestResponse] = response.toListL.runSyncUnsafe(testTimeout)
34 | reply.size shouldBe 1
35 |
36 | reply.head.bodyAsString shouldBe "{ \"msg\" : \"handled index.html\" }"
37 | true
38 | } catch {
39 | case NonFatal(_) => false
40 | } finally {
41 | client.stop()
42 | Try(server.stopFuture().futureValue)
43 | }
44 | }
45 | found.nonEmpty shouldBe true
46 | }
47 | }
48 |
49 | "RestClient.sendPipe" should {
50 | "send and receive data" in {
51 | val Index = WebURI.get("/index.html")
52 | val Save = WebURI.post("/save/:name")
53 | val Read = WebURI.get("/get/name")
54 |
55 | val found = (8000 to 8010).find { port =>
56 | val (server, serverRequests: Observable[RestRequestContext]) = Server.startRest(HostPort.localhost(port), None)
57 | serverRequests.foreach { req => //
58 | req.completeWith(RestResponse.text(s"handled ${req.request.method} request for ${req.request.uri} w/ body '${req.request.bodyAsString}'"))
59 | }
60 |
61 | val client = RestClient.connect(HostPort.localhost(port))
62 | try {
63 |
64 | val requests = List(
65 | RestInput(Index),
66 | RestInput(Save, Map("name" -> "david")),
67 | RestInput(Save, Map("invalid" -> "no name")),
68 | RestInput(Read)
69 | )
70 | val responses = Observable.fromIterable(requests).pipeThrough(client.sendPipe)
71 | var received = List[RestResponse]()
72 | responses.foreach { resp: RestResponse =>
73 | received = resp :: received
74 | }
75 |
76 | eventually {
77 | received.size shouldBe requests.size - 1
78 | }
79 | received.map(_.bodyAsString) should contain theSameElementsInOrderAs List(
80 | "handled GET request for get/name w/ body ''",
81 | "handled POST request for save/david w/ body ''",
82 | "handled GET request for index.html w/ body ''"
83 | )
84 | true
85 | } catch {
86 | case NonFatal(_) => false
87 | } finally {
88 | client.stop()
89 | Try(server.stopFuture().futureValue)
90 | }
91 | }
92 | found.nonEmpty shouldBe true
93 | }
94 | }
95 | }
96 |
--------------------------------------------------------------------------------
/riff-vertx/src/test/scala/riff/web/vertx/server/StartupTest.scala:
--------------------------------------------------------------------------------
1 | package riff.web.vertx.server
2 | import org.scalatest.FunSuite
3 | import riff.monix.RiffMonixSpec
4 |
5 | class StartupTest extends RiffMonixSpec {
6 |
7 | "Startup" should {
8 | "connect a server" in {}
9 | }
10 | }
11 |
--------------------------------------------------------------------------------
/src/main/paradox/about/api.md:
--------------------------------------------------------------------------------
1 | ## API docs
2 | * [Core](https://aaronp.github.io/riff/api/riffCoreCrossProject/riff/raft/index.html)
3 | * [Monix](https://aaronp.github.io/riff/api/riffMonix/riff/monix/index.html)
4 | * [FS2](https://aaronp.github.io/riff/api/riffFs2/riff/fs2/index.html)
5 | * [Akka](https://aaronp.github.io/riff/api/riffAkka/riff/akka/http/index.html)
--------------------------------------------------------------------------------
/src/main/paradox/about/gettingStarted.md:
--------------------------------------------------------------------------------
1 | # Getting Started
2 |
3 | This uses sbt (w/ trial support for mill, perhaps fury at some point) to build.
4 |
5 | The core component contains all the APIs and business logic for a raft cluster in a framework-free way.
6 |
7 | That core can then be put behind a stream (monix, fs2, akka-streams, RxScala, etc), akka actor, REST service, whatever.
8 |
9 | It also provides scalajs support, should you fancy running your server code in a browser, which can be a nifty way to more easily
10 | test multiple components in a cluster. Imagine starting/stopping a component in your micro-service suite by just opening a new
11 | browser tab!
12 |
13 | ## Usage
14 |
15 | I've not officially released any version of this yet.
16 | The indent is to end up with a riff-monix, riff-akka, riff-http4s, riff-vertx, etc, and so you'd bring in
17 | whichever is relevant to you ... or just riff-core and add it to your own transport, etc.
18 |
19 | The above could just be considered examples of how to factor software w/o a dependency on any particular 'framework'.
20 |
21 | I think in general projects should focus more on representing their business logic accurately and modeling it in a way that
22 | makes sense, and then just putting that code behind a service, queue, socket, etc.
23 |
24 |
25 |
26 | ## Building
27 |
28 | The main build still uses sbt, and so you could just:
29 |
30 | ```
31 | sbt package
32 | ```
33 |
34 | I've made a start moving to [mill](https://github.com/lihaoyi/mill), but it's not there yet.
35 |
36 |
37 | ### Documentation
38 |
39 | To generate this documentation, run
40 | ```bash
41 | ./makeSite.sh
42 | ```
--------------------------------------------------------------------------------
/src/main/paradox/about/index.md:
--------------------------------------------------------------------------------
1 | #About
2 |
3 | @@@ index
4 |
5 | * [Getting Started](gettingStarted.md)
6 | * [Scaladoc](api.md)
7 | * [Using IntelliJ](usingIntelliJ.md)
8 |
9 | @@@
10 |
11 |
12 | This started out with just an interest in Raft. I thought it'd be a good candidate to play around with trying out different approaches, as it is easy to understand, well specified, and also useful! It's also a nice mix of networked communication, disk IO, etc, and so I could get a feel for trying to use free monads, finally tagless, or whatever the latest "ooh, how will this work in the real world" technology.
13 |
14 | It's grown a bit from that, as I found the existing JVM implementations a bit lacking, if only because they were all coupled w/ a framework or library, such as akka.
15 |
16 | I'm of the opinion that busiess logic should be writtin in as vanilla a way possible. Just pure functions ideally, which are then "lifted" into some other framework (actors, streams, REST services, whatever).
17 |
18 | And so, the reasoning is largely (hopefully) to aid in readability and reuse, the first of which at least is a core tennent of why Raft was designed the way it is in the first place.
19 |
20 |
21 | ## General approach
22 |
23 | I find most interfaces can be reduced to something in the form:
24 | ```
25 | SomeInput : Foo | Bar(x) | Bazz(y, z)
26 | SomeOutput : Ok | ResultOne(a) | ResultTwo(b)
27 | ```
28 |
29 | In scala, you can then can make this easier to write by exposing a trait like this:
30 | ```scala
31 | trait SomethingDsl {
32 |
33 | def onInput(input : SomeInput) : SomeOutput
34 |
35 | final def foo() = onInput(Foo)
36 |
37 | final def bar(x : Int) = onInput(Bar(x))
38 |
39 | ...
40 | }
41 |
42 | ```
43 |
44 | or even go (arguably dangerously) circular, leaving which implementation to specify up to the thing implementing it:
45 |
46 | ```scala
47 | trait SomethingDsl {
48 |
49 | def onInput(input : SomeInput) : SomeOutput = input match {
50 | case Foo => foo()
51 | case Bar(x) => bar(x)
52 | ...
53 | }
54 |
55 | def foo() = onInput(Foo)
56 |
57 | def bar(x : Int) = onInput(Bar(x))
58 |
59 | ...
60 | }
61 |
62 | ```
63 |
64 | And by representing the inputs using the command pattern, it then easily lends itself to making those commands serialise to/from json (or some binary protocol) for use over the wire. Or, if not, they're messages which can be put on a queue or sent to an actor, so we go a ways already by representing things in that form.
65 |
66 | And so that's the approach I've taken with [RaftNode](https://aaronp.github.io/riff/api/riffCoreCrossProject/riff/raft/node/RaftNode.html)). It has an 'onMessage' which returns a result, and so can easily be put behind a REST service, akka actor, queue, etc.
67 |
68 | So, that's it. It will hopefully be useful, at least to me, and I look forward to the "did you consider" or "you seem to have neglected..." comments.
--------------------------------------------------------------------------------
/src/main/paradox/about/usingIntelliJ.md:
--------------------------------------------------------------------------------
1 | #A note on IntelliJ
2 | Using a cross-compiled project in IntelliJ has some issues (citation needed).
3 |
4 | For now, I've just been manually fixing it every time I change the build.sbt, but at
5 | some point I'll have to script it properly.
6 |
7 | The issue is with shared source roots, and so for the core and JS cross-projects I:
8 |
9 | - Import the project into IntelliJ
10 | - Open the Project Structure
11 | - **Remove** the "riff-core-sources" and "riff-json-sources" modules
12 | - Choose to "Add Content Root: to the riff-core-jvm module, and add the riff-core/shared/src folder
13 | - Change the 'test' folder to a test source (not main source)
14 |
15 | It's a rare enough occurrence to not be a problem, but worth noting here if anyone should want to open this
16 | project in IJ.
17 |
18 |
--------------------------------------------------------------------------------
/src/main/paradox/blog/02_eventSourceAGoGo.md:
--------------------------------------------------------------------------------
1 | # Event Sourcing
2 |
3 | I'm probably not as big a fan as Dan North as this blog might lead you to believe, but this next iteration once again
4 | reminded me of his [Software Faster](https://www.youtube.com/watch?v=USc-yLHXNUg) talk.
5 |
6 | I'd like to fill out the core a bit more, but I also want to gain some value in getting end-to-end, and the
7 | [EventSource](https://aaronp.github.io/riff/api/riffMonix/riff/monix/EventSource$.html) piece is one of those
8 | end-points which I've been waiting to get to.
9 |
10 | There are around 10 lines of code for this, whose punchline is something like:
11 |
12 | ```scala
13 | // some assumptions:
14 | val log : ObservableLog[A] = ...
15 | val latestIndex : LogIndex = ... // a means to find the last written down index
16 |
17 | val entries: Observable[(LogCoords, A)] = log.committedEntriesFrom(latestIndex)
18 | entries.zipWithIndex.scan(latestSnapshot) {
19 | case (state, ((coords, next), i)) =>
20 | val newState = combine(state, next)
21 | if (i != 0 && i % snapEvery == 0) {
22 | dao.writeDown(coords, newState)
23 | }
24 | newState
25 | }
26 | ```
27 |
28 | That essentially is saying "Given a stream of data of type A, apply it to some state (which represents your business logic) of type S,
29 | and write it down every 'snapEvery' events":
30 |
31 | By putting one of those in front of your raft cluster, you're able to just get an ```Observable[S]``` from which you can base the
32 | rest of your business logic on. And that Observable starts with some "state of the world" and gets updated whenever an entry is committed (e.g. can safely
33 | be applied) and fed into your logic.
34 |
35 | Anyway, I just thought that was really nifty. And most of that nift is brought to you via reactive-streams (in this case [monix](https://monix.io/docs/3x/reactive/observable.html)).
36 |
37 | Some neat take-aways, just to spell it out, are:
38 | - The constituent pieces are separated and each stand on their own, adhering pretty closely to a single responsibility principle:
39 | The Raft bit all lives in riff-core with no ties to any transport, business logic, etc. It's just the data structures and logic
40 | for implementing RAFT, with a tiny bit of an opinion on IO (which you could take or leave)
41 | - The event sourcing bit should (I hope -- comments welcome) also be easily understood. It's not some big library brought in, but
42 | just the mapping of a stream of data. Again, this is less about this riff project and more about writing software in terms of
43 | streams of data.
44 |
--------------------------------------------------------------------------------
/src/main/paradox/blog/03_overParameterization.md:
--------------------------------------------------------------------------------
1 | # Over-Parameterization
2 |
3 | ## Going from RaftNode\[NodeKey, A\] to RaftNode\[A\]
4 |
5 | This project blog is becoming an omage to my favourite playlist of scala heroes and tech talks, and so
6 | I guess at this point I should give [Rob Norris's Fixpoint Talk](https://www.youtube.com/watch?v=7xSfLPD6tiQ) a shout-out.
7 |
8 | Rob makes the point that, when in doubt, parameterize the property types. I've found that to be generally a good rule as well,
9 | and so is what I did when representing the underlying nodes in the cluster.
10 |
11 | I already had one parameter type 'A' to use as log type, which makes sense and will always be there.
12 | The second parameter type RaftNode had was 'NodeKey'.
13 |
14 | Essentially I didn't care what the underlying nodes were in the cluster. They could've been ActorRefs, Observables, WebSockets,
15 | Http clients, whatever. And so I just made them a parameterized NodeKey, thinking the extra cost of passing that type around (and
16 | exposing it in the interfaces) would be useful when it came time to interact w/ the types in riff-core.
17 |
18 | Well, I should've gone full end-to-end sooner (see the previous post about tracer-bullets), because it turns out its not so useful.
19 |
20 | It was a minor change which prompted me to remove it. The [PersistentState](https://aaronp.github.io/riff/api/riffCoreCrossProject/riff/raft/node/PersistentState.html) needed
21 | to write down who it voted for in order to avoid potentially double-voting after a crash/restart during an election.
22 |
23 | The problem was, the node was represented as a generic 'NodeKey' type.
24 |
25 | If we kept the parameterized NodeKey, that means we have to be able either serialize/deserialize the key itself, or provide some
26 | memento for that NodeKey. (Memento ... is that an actual pattern? I think I recall it from my Eclipse platform days) So the idea of making the RaftNode hold a reference to a generic NodeKey started to lose its appeal when
27 | having to provide additional support for serialisation.
28 |
29 | And so, upon reevaluation, just making it become a string identifier in 'NodeId' was a lot more appealing. We could drop the extra
30 | param type tax and simplify things. It should be trivial for implementations to resolve a transport for a unique name.
31 |
32 | And, as for unique names, that should be a little more clear now that it's not a generic NodeKey but a specific NodeId (i.e. String).
--------------------------------------------------------------------------------
/src/main/paradox/blog/index.md:
--------------------------------------------------------------------------------
1 | # Project Blog
2 |
3 | @@@ index
4 |
5 | * [Fast, Meaningful Tests](01_raftSimulatorForTheWin.md)
6 | * [Event Sourcing in ~100 lines of code](02_eventSourceAGoGo.md)
7 | * [One parameterized type too far](03_overParameterization.md)
8 | * [Going Reactive](04_goingReactive.md)
9 | * [More (non) Concurrent Testing - Simple made Easy?](05_moreNonConcurrentTesting.md)
10 |
11 | @@@
12 |
13 | I'm sure this exists as "a thing" somewhere already, but I've only just thought of it...
14 | The idea of producing a blog entry along side code commits.
15 |
16 | This way, instead of a wordy code comment (which may get out of date), this blog will
17 | live alongside code as a dump of my (and others') thought processes while we're writing
18 | the code.
19 |
20 | It doesn't take the place of code comments, but also makes it easier to just put down
21 | a lot of thoughts w/o having to worry about maintaining it the same way as if it were
22 | part of the codebase.
23 |
24 |
--------------------------------------------------------------------------------
/src/main/paradox/index.md:
--------------------------------------------------------------------------------
1 | # Riff
2 |
3 | @@toc { depth=1 }
4 |
5 | @@@ index
6 |
7 |
8 | * [About](about/index.md)
9 | * [Project Blog](blog/index.md)
10 |
11 | @@@
12 |
--------------------------------------------------------------------------------
/test.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | # see http://www.lihaoyi.com/mill/index.html
3 | mill core[2.12.6].test
--------------------------------------------------------------------------------
/travis/README.MD:
--------------------------------------------------------------------------------
1 | See http://www.scala-sbt.org/1.0/docs/Travis-CI-with-sbt.html
2 |
3 | regarding metaspace:
4 |
5 | http://blog.sokolenko.me/2014/11/javavm-options-production.html
6 | https://stackoverflow.com/questions/37919215/how-to-get-java8-metaspace-dump-not-heap-dump
7 |
--------------------------------------------------------------------------------
/travis/jvmopts:
--------------------------------------------------------------------------------
1 | -Dfile.encoding=UTF8
2 | -Xms2048M
3 | -Xmx2048M
4 | -Xss6M
5 | -XX:ReservedCodeCacheSize=256M
6 | -XX:+CMSClassUnloadingEnabled
7 | -XX:MaxMetaspaceSize=1g
--------------------------------------------------------------------------------
/version.sbt:
--------------------------------------------------------------------------------
1 | version := "0.0.1-SNAPSHOT"
2 |
--------------------------------------------------------------------------------