├── .gitignore ├── .java-version ├── .travis.yml ├── LICENSE ├── README.md ├── build.sbt ├── deploy └── kubernetes │ ├── justindb-akka-remoting.json │ └── justindb-statefulset.json ├── justin-core └── src │ ├── main │ └── scala │ │ └── justin │ │ └── db │ │ ├── Data.scala │ │ ├── actors │ │ ├── ReplicaCoordinatorActor.scala │ │ ├── StorageNodeActor.scala │ │ └── protocol │ │ │ ├── ClusterSubscriberActorProtocol.scala │ │ │ ├── ReplicaCoordinatorActorProtocol.scala │ │ │ └── StorageNodeActorProtocol.scala │ │ ├── client │ │ ├── ActorRefStorageNodeClient.scala │ │ └── StorageNodeClient.scala │ │ ├── cluster │ │ ├── ClusterMembers.scala │ │ └── datacenter │ │ │ └── Datacenter.scala │ │ ├── kryo │ │ ├── DataSerializer.scala │ │ ├── ListOfDataSerializer.scala │ │ ├── RegisterNodeSerializer.scala │ │ ├── SerializerInit.scala │ │ ├── StorageNodeLocalReadSerializer.scala │ │ ├── StorageNodeReadResponseSerializer.scala │ │ ├── StorageNodeWriteDataLocalSerializer.scala │ │ └── StorageNodeWriteResponseSerializer.scala │ │ ├── merkletrees │ │ ├── MerkleDigest.scala │ │ ├── MerkleTree.scala │ │ └── package.scala │ │ ├── package.scala │ │ ├── replica │ │ ├── IsPrimaryOrReplica.scala │ │ ├── PreferenceList.scala │ │ ├── ReplicationConfig.scala │ │ ├── ResolveNodeAddresses.scala │ │ ├── read │ │ │ ├── ReadAgreement.scala │ │ │ ├── ReplicaLocalReader.scala │ │ │ ├── ReplicaReadAgreement.scala │ │ │ ├── ReplicaReadCoordinator.scala │ │ │ └── ReplicaRemoteReader.scala │ │ └── write │ │ │ ├── ReplicaLocalWriter.scala │ │ │ ├── ReplicaRemoteWriter.scala │ │ │ ├── ReplicaWriteAgreement.scala │ │ │ ├── ReplicaWriteCoordinator.scala │ │ │ └── WriteAgreement.scala │ │ └── versioning │ │ └── NodeIdVectorClockBase64.scala │ └── test │ └── scala │ └── justin │ └── db │ ├── ConvergeTest.scala │ ├── DataTest.scala │ ├── actors │ ├── RoundRobinCoordinatorRouterTest.scala │ └── StorageNodeActorTest.scala │ ├── client │ └── ActorRefStorageNodeClientTest.scala │ ├── cluster │ └── ClusterMembersTest.scala │ ├── kryo │ ├── DataSerializerTest.scala │ ├── RegisterNodeSerializerTest.scala │ ├── SerializerInitTest.scala │ ├── StorageNodeLocalReadSerializerTest.scala │ ├── StorageNodeReadResponseSerializerTest.scala │ ├── StorageNodeWriteDataLocalSerializerTest.scala │ └── StorageNodeWriteResponseSerializerTest.scala │ ├── merkletrees │ ├── DigestTest.scala │ ├── MD5DigestTest.scala │ └── MerkleTreeTest.scala │ ├── replica │ ├── IsPrimaryOrReplicaTest.scala │ ├── PreferenceListSpecification.scala │ ├── PreferenceListTest.scala │ ├── ResolveNodeAddressesTest.scala │ ├── read │ │ ├── ReplicaLocalReaderTest.scala │ │ ├── ReplicaReadAgreementTest.scala │ │ └── ReplicaRemoteReaderTest.scala │ └── write │ │ ├── ReplicaLocalWriterTest.scala │ │ ├── ReplicaRemoteWriterTest.scala │ │ └── ReplicaWriteAgreementTest.scala │ └── versioning │ └── NodeIdVectorClockBase64Test.scala ├── justin-http-api └── src │ ├── main │ └── scala │ │ └── justin │ │ └── httpapi │ │ ├── BuildInfoRouter.scala │ │ ├── HealthCheckRouter.scala │ │ ├── HttpRouter.scala │ │ ├── JustinDirectives.scala │ │ ├── Unmarshallers.scala │ │ └── VectorClockHeader.scala │ └── test │ ├── resources │ └── test.conf │ └── scala │ └── justin │ └── httpapi │ ├── BuildInfoRouterTest.scala │ ├── HealthCheckRouterTest.scala │ ├── HttpRouterTest.scala │ ├── JustinDirectivesTest.scala │ ├── UnmarshallersTest.scala │ └── VectorClockHeaderTest.scala ├── justin-ring └── src │ ├── main │ └── scala │ │ └── justin │ │ └── db │ │ └── consistenthashing │ │ ├── NodeId.scala │ │ ├── Ring.scala │ │ └── UUID2RingPartitionId.scala │ └── test │ └── scala │ └── justin │ └── db │ └── consistenthashing │ ├── RingTest.scala │ └── UUID2RingPartitionIdTest.scala ├── justin-storage-api └── src │ └── main │ └── scala │ └── justin │ └── db │ └── storage │ ├── JustinData.scala │ ├── PluggableStorageProtocol.scala │ ├── package.scala │ └── provider │ └── StorageProvider.scala ├── justin-storage-in-mem └── src │ ├── main │ ├── resources │ │ ├── justin.conf │ │ └── reference.conf │ └── scala │ │ └── justin │ │ └── db │ │ └── storage │ │ ├── InMemStorage.scala │ │ ├── config │ │ └── justin.scala │ │ └── provider │ │ └── InMemStorageProvider.scala │ └── test │ └── scala │ └── justin │ └── db │ └── storage │ ├── InMemStorageTest.scala │ └── provider │ └── InMemStorageProviderTest.scala ├── justin-storage-rocksdb └── src │ ├── main │ ├── resources │ │ ├── justin.conf │ │ └── reference.conf │ └── scala │ │ └── justin │ │ └── db │ │ └── storage │ │ ├── RocksDBStorage.scala │ │ ├── config │ │ └── justin.scala │ │ └── provider │ │ └── RocksDBStorageProvider.scala │ └── test │ └── scala │ └── justin │ └── db │ └── storage │ ├── JustinDataSerializerTest.scala │ ├── RocksDBStorageTest.scala │ ├── UUIDSerializerTest.scala │ └── provider │ └── RocksDBStorageProviderTest.scala ├── justin-vector-clocks └── src │ ├── main │ └── scala │ │ └── justin │ │ └── db │ │ └── vectorclocks │ │ ├── Counter.scala │ │ ├── VectorClock.scala │ │ ├── VectorClockComparator.scala │ │ └── package.scala │ └── test │ └── scala │ └── justin │ └── db │ └── vectorclocks │ ├── CounterTest.scala │ ├── VectorClockComparatorTest.scala │ ├── VectorClockOpsTest.scala │ └── VectorClockTest.scala ├── logo.png ├── project ├── Dependencies.scala ├── assembly.sbt ├── build.properties ├── plugins.sbt ├── travis-build.sh └── travis-post-build.sh ├── scalastyle-config.xml └── src ├── main ├── resources │ ├── application.conf │ ├── justin.conf │ └── logback.xml └── scala │ └── justin │ └── db │ ├── JustinDB.scala │ ├── Main.scala │ └── justin.scala ├── multi-jvm └── scala │ └── justin │ └── db │ ├── ConvergeJustinDBClusterSpec.scala │ ├── MultiDcSpec.scala │ ├── MultiNodeClusterSpec.scala │ └── STMultiNodeSpec.scala └── universal ├── cli └── motd /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | 4 | # sbt specific 5 | .cache 6 | .history 7 | .lib/ 8 | dist/* 9 | target/ 10 | lib_managed/ 11 | src_managed/ 12 | project/boot/ 13 | project/plugins/project/ 14 | 15 | # Scala-IDE specific 16 | .scala_dependencies 17 | .worksheet 18 | 19 | # Idea-IDE 20 | .idea 21 | .idea_modules 22 | 23 | # RocksDB 24 | CURRENT 25 | IDENTITY 26 | LOCK 27 | LOG 28 | MANIFEST-* 29 | OPTIONS-* 30 | -------------------------------------------------------------------------------- /.java-version: -------------------------------------------------------------------------------- 1 | 1.8 2 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | 2 | language: scala 3 | sudo: false 4 | 5 | scala: 6 | - 2.12.3 7 | jdk: 8 | - oraclejdk8 9 | 10 | env: 11 | global: 12 | - MAIN_SCALA_VERSION=2.12.3 13 | 14 | services: 15 | - docker 16 | 17 | script: 18 | - project/travis-build.sh 19 | after_success: 20 | - project/travis-post-build.sh 21 | 22 | cache: 23 | directories: 24 | - $HOME/.sbt/0.13 25 | - $HOME/.sbt/boot/scala* 26 | - $HOME/.sbt/cache 27 | - $HOME/.sbt/launchers 28 | - $HOME/.ivy2 29 | - $HOME/.coursier 30 | 31 | before_cache: 32 | - du -h -d 1 $HOME/.ivy2/ 33 | - du -h -d 2 $HOME/.sbt/ 34 | - find $HOME/.sbt -name "*.lock" -type f -delete 35 | - find $HOME/.ivy2/cache -name "ivydata-*.properties" -type f -delete -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # JustinDB 2 | 3 | 4 | 5 | [![Build Status](https://travis-ci.org/justin-db/JustinDB.svg?branch=master)](https://travis-ci.org/justin-db/JustinDB) 6 | [![codecov](https://codecov.io/gh/justin-db/JustinDB/branch/master/graph/badge.svg)](https://codecov.io/gh/justin-db/JustinDB) 7 | [![Codacy Badge](https://api.codacy.com/project/badge/grade/f5f10352c6e74aa99d0f996cf0a77124)](https://www.codacy.com/app/mateusz-maciaszekhpc/JustinDB) 8 | [![License](http://img.shields.io/:license-Apache%202-red.svg)](http://www.apache.org/licenses/LICENSE-2.0.txt) 9 | ![Project Status](https://img.shields.io/badge/status-beta-yellow.svg) 10 | [![Gitter](https://img.shields.io/badge/gitter-join%20chat-brightgreen.svg)](https://gitter.im/justin-db/Lobby) 11 | 12 | [![Watch on GitHub](https://img.shields.io/github/watchers/justin-db/JustinDB.svg?style=social)](https://github.com/justin-db/JustinDB/watchers) 13 | [![Star on GitHub](https://img.shields.io/github/stars/justin-db/JustinDB.svg?style=social)](https://github.com/justin-db/JustinDB/stargazers) 14 | 15 | **Distributed Key-Value Storage built on top of Scala/Akka** 16 | 17 | JustinDB KV is an eventually consistent key-value database that favours write availability. 18 | It’s a faithful implementation of Amazon’s Dynamo, with advanced features such as vector clocks for conflict resolution. 19 | JustinDB is also fault-tolerant. Servers can go up or down at any moment with no single point of failure. 20 | 21 | ``` 22 | ➜ JustinDB git:(master) ✗ docker exec -it justindb /bin/bash 23 | ___ _ _ ______ ______ 24 | |_ | | | (_) | _ \| ___ \ 25 | | | _ _ ___ | |_ _ _ __ | | | || |_/ / 26 | | || | | |/ __|| __|| || '_ \ | | | || ___ \ 27 | /\__/ /| |_| |\__ \| |_ | || | | || |/ / | |_/ / 28 | \____/ \__,_||___/ \__||_||_| |_||___/ \____/ 29 | 30 | Cli: 31 | cli help 32 | More documentation available at: 33 | https://github.com/speedcom/JustinDB 34 | https://speedcom.github.io/ 35 | ``` 36 | 37 | ## Summary of techniques 38 | 39 | | Problem | Technique | Advantage | 40 | |---------|------------|------------| 41 | |Partitioning |Consistent Hashing |Incremental Scalability| 42 | |Membership and failure detection |Gossip-based membership protocol and failure detection|Preserves symmetry and avoids having a centralized registry for storing membership and node liveness information| 43 | |High Availability for writes |Vector clocks with reconciliation during reads |Version size is decoupled from update rites| 44 | |Recovering from permanent failures|Anti-entropy using Merkle trees |Synchronizes divergent replicas in the background| 45 | 46 | ### Why akka 47 | Its a toolkit and runtime for building highly concurrent applications which comes 48 | with ideas that have been around from some time - actor model. 49 | Besides that it has many welcome features around clustering: 50 | 51 | 1. load balancing 52 | 2. location transparency 53 | 3. self maintenance 54 | 4. fault tolerance 55 | 56 | ### Authentication, authorization, validation 57 | 58 | In case it's not obvious, Justin performs no authentication, authorization, or any validation of input data. Clients must implement those things themselves. 59 | 60 | ## System Requirements 61 | JustinDB works with Java 8 and newer. 62 | 63 | ## Bunch of posts about JustinDB 64 | 1. [JustinDB - Modern REACTIVE NoSQL database](http://speedcom.github.io/dsp2017/2017/03/14/justindb-modern-reactive-nosql-database.html) 65 | 2. [JustinDB - Database Model](http://speedcom.github.io/dsp2017/2017/03/17/justindb-database-model.html) 66 | 3. [JustinDB - Pluggable persistent and in-memory storage engines](http://speedcom.github.io/dsp2017/2017/03/24/justindb-support-for-pluggable-persistent-and-in-memory-storage-engines.html) 67 | 4. [JustinDB - More than 700 commits!](http://speedcom.github.io/dsp2017/2017/04/03/justindb-more-than-seven-hundred-commits.html) 68 | 5. [JustinDB - Serialization that greatly improves performance](http://speedcom.github.io/dsp2017/2017/04/08/justindb-serilization-that-greatly-improves-performance.html) 69 | 6. [JustinDB - Replication and Partitioning](http://speedcom.github.io/dsp2017/2017/04/13/justindb-replication-and-partitioning.html) 70 | 7. [JustinDB - Why Scala and Akka?](http://speedcom.github.io/dsp2017/2017/04/15/justindb-why-scala-and-akka.html) 71 | 8. [JustinDB - Data versioning: Vector Clocks](http://speedcom.github.io/dsp2017/2017/04/21/justindb-data-versioning.html) 72 | 9. [JustinDB - HTTP API](http://speedcom.github.io/dsp2017/2017/04/30/justindb-http-api.html) 73 | 10. [JustinDB - The Ring](http://speedcom.github.io/dsp2017/2017/05/06/justindb-ring.html) 74 | 11. [JustinDB - Preference list](http://speedcom.github.io/dsp2017/2017/05/07/justindb-preference-list.html) 75 | 12. [JustinDB - Solving data entropy: Read Repair](http://speedcom.github.io/dsp2017/2017/05/13/justindb-read-repair.html) 76 | 13. [JustinDB - Solving data entropy: Active-Anti Entropy](http://speedcom.github.io/dsp2017/2017/05/14/justindb-active-anti-entropy.html) 77 | 14. [JustinDB - Executable JAR](http://speedcom.github.io/dsp2017/2017/05/20/justindb-executable-jar.html) 78 | 15. [JustinDB - Multi-Datacenter Replication](http://speedcom.github.io/2017/07/09/justindb-multi-datacenter-replication.html) 79 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | import com.typesafe.sbt.packager.docker._ 3 | 4 | name := "JustinDB" 5 | maintainer := "mateusz.maciaszekhpc@gmail.com" 6 | 7 | resolvers += Resolver.bintrayRepo("hseeberger", "maven") 8 | 9 | fork in run := true 10 | 11 | // DOCKER DEFINITION 12 | daemonUser.in(Docker) := "root" 13 | maintainer.in(Docker) := "Mateusz Maciaszek" 14 | dockerRepository := Some("justindb") 15 | dockerUpdateLatest := true 16 | dockerBaseImage := "local/openjdk-jre-8-bash" 17 | dockerCommands += Cmd("RUN", s"ln -s /opt/docker/cli /bin/cli") 18 | dockerCommands += Cmd("RUN", "echo 'cat /opt/docker/motd' >> /etc/bash.bashrc") 19 | dockerEntrypoint ++= Seq( 20 | """-Djustin.netty-tcp-hostname="$(eval "echo $AKKA_REMOTING_BIND_HOST")"""", 21 | """-Djustin.netty-tcp-port="$AKKA_REMOTING_BIND_PORT"""", 22 | """-Djustin.system="$AKKA_ACTOR_SYSTEM_NAME"""", 23 | """-Djustin.kubernetes-hostname="$(eval "echo $JUSTINDB_NODE_ID_NAME")"""", 24 | """$(IFS=','; I=0; for NODE in $AKKA_SEED_NODES; do echo "-Dakka.cluster.seed-nodes.$I=akka.tcp://$AKKA_ACTOR_SYSTEM_NAME@$NODE"; I=$(expr $I + 1); done)""", 25 | "-Dakka.io.dns.resolver=async-dns", 26 | "-Dakka.io.dns.async-dns.resolve-srv=true", 27 | "-Dakka.io.dns.async-dns.resolv-conf=on" 28 | ) 29 | dockerCommands := 30 | dockerCommands.value.flatMap { 31 | case ExecCmd("ENTRYPOINT", args @ _*) => Seq(Cmd("ENTRYPOINT", args.mkString(" "))) 32 | case v => Seq(v) 33 | } 34 | 35 | // Force building with Java 8 36 | initialize := { 37 | val required = "1.8" 38 | val current = sys.props("java.specification.version") 39 | assert(current == required, s"Unsupported build JDK: java.specification.version $current != $required") 40 | } 41 | 42 | // PROJECT DEFINITIONS 43 | lazy val root = (project in file(".")) 44 | .enablePlugins(BuildInfoPlugin, SbtMultiJvm, JavaServerAppPackaging) 45 | .configs(MultiJvm) 46 | .settings(commonSettings: _*) 47 | .settings( 48 | mainClass in assembly := Some("justin.db.Main"), 49 | assemblyJarName in assembly := "justindb.jar", 50 | test in assembly := {}, 51 | libraryDependencies ++= Dependencies.root, 52 | scalaVersion := Version.scala, 53 | buildInfoKeys := Seq[BuildInfoKey](name, version, scalaVersion, sbtVersion, git.gitHeadCommit, git.gitCurrentBranch), 54 | buildInfoOptions += BuildInfoOption.ToJson 55 | ) 56 | .settings(versionWithGit) 57 | .settings(git.useGitDescribe := true) 58 | .settings(configAnnotationSettings) 59 | .aggregate(core, httpApi, storageInMem, storageRocksDB) 60 | .dependsOn(core, httpApi, storageInMem, storageRocksDB) 61 | 62 | lazy val core = (project in file("justin-core")) 63 | .disablePlugins(RevolverPlugin) 64 | .configs(MultiJvm) 65 | .settings(commonSettings: _*) 66 | .settings( 67 | name := "justin-core", 68 | scalaVersion := Version.scala, 69 | libraryDependencies ++= Dependencies.core 70 | ) 71 | .aggregate(storageApi, ring, vectorClocks) 72 | .dependsOn(storageApi, ring, vectorClocks) 73 | 74 | lazy val ring = (project in file("justin-ring")) 75 | .disablePlugins(RevolverPlugin) 76 | .settings(commonSettings: _*) 77 | .settings( 78 | name := "justin-ring", 79 | scalaVersion := Version.scala, 80 | libraryDependencies ++= Dependencies.ring 81 | ) 82 | 83 | lazy val vectorClocks = (project in file("justin-vector-clocks")) 84 | .disablePlugins(RevolverPlugin) 85 | .settings(commonSettings: _*) 86 | .settings( 87 | name := "justin-vector-clocks", 88 | scalaVersion := Version.scala, 89 | libraryDependencies ++= Dependencies.vectorClocks 90 | ) 91 | 92 | lazy val httpApi = (project in file("justin-http-api")) 93 | .disablePlugins(RevolverPlugin) 94 | .settings(commonSettings: _*) 95 | .settings( 96 | name := "justin-http-api", 97 | scalaVersion := Version.scala, 98 | libraryDependencies ++= Dependencies.httpApi, 99 | fork in Test := true, 100 | javaOptions in Test += "-Dconfig.resource=test.conf" 101 | ) 102 | .dependsOn(core) 103 | 104 | lazy val storageApi = (project in file("justin-storage-api")) 105 | .disablePlugins(RevolverPlugin) 106 | .settings(commonSettings: _*) 107 | .settings( 108 | name := "justin-storage-api", 109 | scalaVersion := Version.scala, 110 | libraryDependencies ++= Dependencies.storageApi 111 | ) 112 | 113 | lazy val storageInMem = (project in file("justin-storage-in-mem")) 114 | .disablePlugins(RevolverPlugin) 115 | .settings(configAnnotationSettings) 116 | .settings(commonSettings: _*) 117 | .settings( 118 | name := "justin-storage-in-mem", 119 | scalaVersion := Version.scala, 120 | libraryDependencies ++= Dependencies.storageInMem 121 | ) 122 | .dependsOn(storageApi) 123 | 124 | lazy val storageRocksDB = (project in file("justin-storage-rocksdb")) 125 | .disablePlugins(RevolverPlugin) 126 | .settings(configAnnotationSettings) 127 | .settings(commonSettings: _*) 128 | .settings( 129 | name := "justin-storage-rocksdb", 130 | scalaVersion := Version.scala, 131 | libraryDependencies ++= Dependencies.storageRocksDB 132 | ) 133 | .dependsOn(storageApi) 134 | 135 | // ALIASES 136 | addCommandAlias("compileAll", ";compile;test:compile;multi-jvm:compile") 137 | addCommandAlias("testAll", ";test:test;multi-jvm:test") 138 | 139 | // SETTINGS 140 | lazy val commonSettings = Def.settings( 141 | compileSettings 142 | ) 143 | 144 | lazy val compileSettings = Def.settings( 145 | scalacOptions ++= Seq( 146 | "-feature", 147 | "-deprecation", 148 | "-unchecked", 149 | "-encoding", 150 | "utf8", 151 | "-language:implicitConversions", 152 | "-language:reflectiveCalls", 153 | "-language:existentials", 154 | "-language:experimental.macros", 155 | "-language:higherKinds", 156 | "-Xfatal-warnings", 157 | "-Xfuture", 158 | "-Yno-adapted-args", 159 | "-Ywarn-numeric-widen", 160 | "-Ywarn-value-discard", 161 | "-Ywarn-dead-code" 162 | ) 163 | ) 164 | 165 | lazy val configAnnotationSettings: Seq[sbt.Setting[_]] = { 166 | Seq( 167 | scalacOptions += "-Xmacro-settings:conf.output.dir=" + baseDirectory.value.getAbsolutePath + "/src/main/resources", 168 | addCompilerPlugin(Library.macroParadise cross CrossVersion.full), 169 | libraryDependencies += Library.configAnnotation 170 | ) 171 | } 172 | -------------------------------------------------------------------------------- /deploy/kubernetes/justindb-akka-remoting.json: -------------------------------------------------------------------------------- 1 | { 2 | "apiVersion": "v1", 3 | "kind": "Service", 4 | "metadata": { 5 | "labels": { 6 | "app": "justindb" 7 | }, 8 | "name": "justindb" 9 | }, 10 | "spec": { 11 | "clusterIP": "None", 12 | "ports": [ 13 | { 14 | "port": 2551, 15 | "protocol": "TCP", 16 | "targetPort": 2551, 17 | "name": "akka-remote" 18 | } 19 | ], 20 | "selector": { 21 | "app": "justindb" 22 | } 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /deploy/kubernetes/justindb-statefulset.json: -------------------------------------------------------------------------------- 1 | { 2 | "apiVersion": "apps/v1beta1", 3 | "kind": "StatefulSet", 4 | "metadata": { 5 | "name": "justindb" 6 | }, 7 | "spec": { 8 | "serviceName": "justindb", 9 | "replicas": 3, 10 | "template": { 11 | "metadata": { 12 | "labels": { 13 | "app": "justindb" 14 | } 15 | }, 16 | "spec": { 17 | "containers": [ 18 | { 19 | "name": "justindb", 20 | "image": "justindb/justindb", 21 | "imagePullPolicy": "Always", 22 | "ports": [ 23 | { 24 | "containerPort": 2551, 25 | "name": "akka-remote" 26 | }, 27 | { 28 | "containerPort": 9000, 29 | "name": "justindb-host" 30 | }, 31 | { 32 | "containerPort": 19999, 33 | "name": "justindb-info" 34 | } 35 | ], 36 | "env": [ 37 | { 38 | "name": "JUSTINDB_NODE_ID_NAME", 39 | "value": "$HOSTNAME" 40 | }, 41 | { 42 | "name": "AKKA_ACTOR_SYSTEM_NAME", 43 | "value": "justindb" 44 | }, 45 | { 46 | "name": "AKKA_REMOTING_BIND_PORT", 47 | "value": "2551" 48 | }, 49 | { 50 | "name": "AKKA_REMOTING_BIND_HOST", 51 | "value": "$HOSTNAME.justindb.default.svc.cluster.local" 52 | }, 53 | { 54 | "name": "AKKA_SEED_NODES", 55 | "value": "justindb-0.justindb.default.svc.cluster.local:2551,justindb-1.justindb.default.svc.cluster.local:2551,justindb-2.justindb.default.svc.cluster.local:2551" 56 | } 57 | ], 58 | "readinessProbe": { 59 | "tcpSocket": { 60 | "port": 2551 61 | }, 62 | "initialDelaySeconds": 10, 63 | "timeoutSeconds": 120 64 | } 65 | } 66 | ] 67 | } 68 | } 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/Data.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.consistenthashing.NodeId 6 | import justin.db.replica.PreferenceList 7 | import justin.db.storage.JustinData 8 | import justin.db.vectorclocks.VectorClock 9 | import justin.db.versioning.NodeIdVectorClockBase64 10 | 11 | import scala.language.implicitConversions 12 | 13 | case class Data(id: UUID, value: String, vclock: VectorClock[NodeId] = VectorClock(), timestamp: Long = System.currentTimeMillis()) 14 | 15 | object Data { 16 | 17 | def updateVclock(data: Data, preferenceList: PreferenceList): Data = { 18 | val nodeIds = preferenceList.all 19 | data.copy(vclock = nodeIds.foldLeft(data.vclock)(_ increase _)) 20 | } 21 | 22 | implicit def toInternal(data: Data): JustinData = { 23 | val encodedVClock = new NodeIdVectorClockBase64().encode(data.vclock).get // TODO: check if encoding of VClock is possible (make it typesafe) 24 | JustinData(data.id, data.value, encodedVClock, data.timestamp) 25 | } 26 | 27 | implicit def fromInternal(justinData: JustinData): Data = { 28 | val decodedVClock = new NodeIdVectorClockBase64().decode(justinData.vclock).get // TODO: check if decoding of VClock is possible (make it typesafe) 29 | Data(justinData.id, justinData.value, decodedVClock, justinData.timestamp) 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/actors/ReplicaCoordinatorActor.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors 2 | 3 | import akka.actor.{Actor, Props} 4 | import akka.routing.{DefaultResizer, RoundRobinPool} 5 | import justin.db.actors.protocol.{ReadData, WriteData} 6 | import justin.db.replica.read.ReplicaReadCoordinator 7 | import justin.db.replica.write.ReplicaWriteCoordinator 8 | 9 | import scala.concurrent.ExecutionContext 10 | 11 | class ReplicaCoordinatorActor(readCoordinator: ReplicaReadCoordinator, writeCoordinator: ReplicaWriteCoordinator) extends Actor { 12 | 13 | private implicit val ec: ExecutionContext = context.dispatcher 14 | 15 | override def receive: Receive = { 16 | case rd: ReadData => readCoordinator.apply(rd.cmd, rd.clusterMembers).foreach(rd.sender ! _) 17 | case wd: WriteData => writeCoordinator.apply(wd.cmd, wd.clusterMembers).foreach(wd.sender ! _) 18 | } 19 | } 20 | 21 | object ReplicaCoordinatorActor { 22 | 23 | def props(readCoordinator: ReplicaReadCoordinator, writeCoordinator: ReplicaWriteCoordinator): Props = { 24 | Props(new ReplicaCoordinatorActor(readCoordinator, writeCoordinator)) 25 | } 26 | } 27 | 28 | object RoundRobinCoordinatorRouter { 29 | def routerName: String = "CoordinatorRouter" 30 | 31 | private val pool = RoundRobinPool( 32 | nrOfInstances = 5, 33 | resizer = Some(DefaultResizer(lowerBound = 2, upperBound = 15)) 34 | ) 35 | 36 | def props(readCoordinator: ReplicaReadCoordinator, writeCoordinator: ReplicaWriteCoordinator): Props = { 37 | pool.props(ReplicaCoordinatorActor.props(readCoordinator, writeCoordinator)) 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/actors/StorageNodeActor.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors 2 | 3 | import akka.actor.{Actor, ActorRef, Props, RootActorPath, Terminated} 4 | import akka.cluster.ClusterEvent.{CurrentClusterState, MemberUp} 5 | import akka.cluster.{Cluster, Member, MemberStatus} 6 | import com.typesafe.scalalogging.StrictLogging 7 | import justin.db.actors.protocol.{RegisterNode, _} 8 | import justin.db.cluster.ClusterMembers 9 | import justin.db.cluster.datacenter.Datacenter 10 | import justin.db.consistenthashing.{NodeId, Ring} 11 | import justin.db.replica._ 12 | import justin.db.replica.read.{ReplicaLocalReader, ReplicaReadCoordinator, ReplicaRemoteReader} 13 | import justin.db.replica.write.{ReplicaLocalWriter, ReplicaRemoteWriter, ReplicaWriteCoordinator} 14 | import justin.db.storage.PluggableStorageProtocol 15 | 16 | import scala.concurrent.ExecutionContext 17 | 18 | class StorageNodeActor(nodeId: NodeId, datacenter: Datacenter, storage: PluggableStorageProtocol, ring: Ring, n: N) extends Actor with StrictLogging { 19 | 20 | private[this] implicit val ec: ExecutionContext = context.dispatcher 21 | private[this] val cluster = Cluster(context.system) 22 | 23 | private[this] var clusterMembers = ClusterMembers.empty 24 | private[this] val readCoordinator = new ReplicaReadCoordinator(nodeId, ring, n, new ReplicaLocalReader(storage), new ReplicaRemoteReader) 25 | private[this] val writeCoordinator = new ReplicaWriteCoordinator(nodeId, ring, n, new ReplicaLocalWriter(storage), new ReplicaRemoteWriter) 26 | 27 | private[this] val coordinatorRouter = context.actorOf( 28 | props = RoundRobinCoordinatorRouter.props(readCoordinator, writeCoordinator), 29 | name = RoundRobinCoordinatorRouter.routerName 30 | ) 31 | 32 | private[this] val name = self.path.name 33 | 34 | override def preStart(): Unit = cluster.subscribe(this.self, classOf[MemberUp]) 35 | override def postStop(): Unit = cluster.unsubscribe(this.self) 36 | 37 | def receive: Receive = { 38 | receiveDataPF orElse receiveClusterDataPF orElse receiveRegisterNodePR orElse notHandledPF 39 | } 40 | 41 | private[this] def receiveDataPF: Receive = { 42 | case readReq: StorageNodeReadRequest => 43 | coordinatorRouter ! ReadData(sender(), clusterMembers, readReq) 44 | case writeLocalDataReq: StorageNodeWriteDataLocal => 45 | coordinatorRouter ! WriteData(sender(), clusterMembers, writeLocalDataReq) 46 | case writeClientReplicaReq: Internal.WriteReplica => 47 | coordinatorRouter ! WriteData(sender(), clusterMembers, writeClientReplicaReq) 48 | } 49 | 50 | private[this] def receiveClusterDataPF: Receive = { 51 | case "members" => sender() ! clusterMembers 52 | case MemberUp(member) => register(nodeId, ring, member) 53 | case state: CurrentClusterState => state.members.filter(_.status == MemberStatus.Up).foreach(member => register(nodeId, ring, member)) 54 | case Terminated(actorRef) => clusterMembers = clusterMembers.removeByRef(StorageNodeActorRef(actorRef)) 55 | } 56 | 57 | private[this] def receiveRegisterNodePR: Receive = { 58 | case RegisterNode(senderNodeId) if clusterMembers.notContains(senderNodeId) => 59 | val senderRef = sender() 60 | context.watch(senderRef) 61 | clusterMembers = clusterMembers.add(senderNodeId, StorageNodeActorRef(senderRef)) 62 | senderRef ! RegisterNode(nodeId) 63 | logger.info(s"Actor[$name]: Successfully registered node [id-${senderNodeId.id}]") 64 | case RegisterNode(senderNodeId) => 65 | logger.info(s"Actor[$name]: Node [id-${senderNodeId.id}] is already registered") 66 | } 67 | 68 | private[this] def register(nodeId: NodeId, ring: Ring, member: Member) = { 69 | (member.hasRole(StorageNodeActor.role), datacenter.name == member.dataCenter) match { 70 | case (true, true) => register() 71 | case (_, false) => logger.info(s"Actor[$name]: $member doesn't belong to datacenter [${datacenter.name}]") 72 | case (false, _) => logger.info(s"Actor[$name]: $member doesn't have [${StorageNodeActor.role}] role (it has roles ${member.roles}") 73 | } 74 | 75 | def register() = for { 76 | ringNodeId <- ring.nodesId 77 | nodeName = StorageNodeActor.name(ringNodeId, Datacenter(member.dataCenter)) 78 | nodeRef = context.actorSelection(RootActorPath(member.address) / "user" / nodeName) 79 | } yield nodeRef ! RegisterNode(nodeId) 80 | } 81 | 82 | private[this] def notHandledPF: Receive = { 83 | case t => logger.warn(s"Actor[$name]: Not handled message [$t]") 84 | } 85 | } 86 | 87 | object StorageNodeActor { 88 | def role: String = "storagenode" 89 | def name(nodeId: NodeId, datacenter: Datacenter): String = s"${datacenter.name}-id-${nodeId.id}" 90 | def props(nodeId: NodeId, datacenter: Datacenter, storage: PluggableStorageProtocol, ring: Ring, n: N): Props = { 91 | Props(new StorageNodeActor(nodeId, datacenter, storage, ring, n)) 92 | } 93 | } 94 | 95 | case class StorageNodeActorRef(ref: ActorRef) extends AnyVal 96 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/actors/protocol/ClusterSubscriberActorProtocol.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors.protocol 2 | 3 | import justin.db.consistenthashing.NodeId 4 | 5 | case class RegisterNode(nodeId: NodeId) 6 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/actors/protocol/ReplicaCoordinatorActorProtocol.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors.protocol 2 | 3 | import akka.actor.ActorRef 4 | import justin.db.cluster.ClusterMembers 5 | 6 | case class WriteData(sender: ActorRef, clusterMembers: ClusterMembers, cmd: StorageNodeWriteRequest) 7 | case class ReadData(sender: ActorRef, clusterMembers: ClusterMembers, cmd: StorageNodeReadRequest) 8 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/actors/protocol/StorageNodeActorProtocol.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors.protocol 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.Data 6 | import justin.db.replica.{R, W} 7 | 8 | sealed trait StorageNodeReadRequest 9 | case class StorageNodeLocalRead(id: UUID) extends StorageNodeReadRequest 10 | 11 | sealed trait StorageNodeReadResponse 12 | case class StorageNodeFoundRead(data: Data) extends StorageNodeReadResponse 13 | case class StorageNodeConflictedRead(conflicts: List[Data]) extends StorageNodeReadResponse 14 | case class StorageNodeNotFoundRead(id: UUID) extends StorageNodeReadResponse 15 | case class StorageNodeFailedRead(id: UUID) extends StorageNodeReadResponse 16 | 17 | sealed trait StorageNodeWriteRequest 18 | case class StorageNodeWriteDataLocal(data: Data) extends StorageNodeWriteRequest 19 | 20 | sealed trait StorageNodeWriteResponse 21 | case class StorageNodeSuccessfulWrite(id: UUID) extends StorageNodeWriteResponse 22 | case class StorageNodeFailedWrite(id: UUID) extends StorageNodeWriteResponse 23 | case class StorageNodeConflictedWrite(oldData: Data, newData: Data) extends StorageNodeWriteResponse 24 | 25 | // These messages are not sent over remote channel 26 | object Internal { 27 | case class ReadReplica(r: R, id: UUID) extends StorageNodeReadRequest 28 | case class WriteReplica(w: W, data: Data) extends StorageNodeWriteRequest 29 | } 30 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/client/ActorRefStorageNodeClient.scala: -------------------------------------------------------------------------------- 1 | package justin.db.client 2 | 3 | import java.util.UUID 4 | 5 | import akka.pattern.ask 6 | import akka.util.Timeout 7 | import justin.db.Data 8 | import justin.db.actors.StorageNodeActorRef 9 | import justin.db.actors.protocol._ 10 | import justin.db.replica.{R, W} 11 | 12 | import scala.concurrent.duration._ 13 | import scala.concurrent.{ExecutionContext, Future} 14 | 15 | class ActorRefStorageNodeClient(storageNodeActor: StorageNodeActorRef)(implicit ex: ExecutionContext) extends StorageNodeClient { 16 | 17 | implicit val timeout = Timeout(5.seconds) // TODO: tune this value 18 | 19 | override def get(id: UUID, r: R): Future[GetValueResponse] = { 20 | (storageNodeActor.ref ? Internal.ReadReplica(r, id)).mapTo[StorageNodeReadResponse].map { 21 | case StorageNodeFoundRead(data) => GetValueResponse.Found(data) 22 | case StorageNodeConflictedRead(data) => GetValueResponse.Conflicts(data) 23 | case StorageNodeNotFoundRead(id) => GetValueResponse.NotFound(id) 24 | case StorageNodeFailedRead(_) => GetValueResponse.Failure(s"Couldn't read value with id ${id.toString}") 25 | } recover { case ex: Throwable => GetValueResponse.Failure(s"Unsuccessful read of value with id ${id.toString}") } 26 | } 27 | 28 | override def write(data: Data, w: W): Future[WriteValueResponse] = { 29 | (storageNodeActor.ref ? Internal.WriteReplica(w, data)).mapTo[StorageNodeWriteResponse].map { 30 | case StorageNodeSuccessfulWrite(id) => WriteValueResponse.Success(id) 31 | case StorageNodeConflictedWrite(_, _) => WriteValueResponse.Conflict 32 | case StorageNodeFailedWrite(id) => WriteValueResponse.Failure(s"Couldn't write value with id ${id.toString}") 33 | } recover { case ex: Throwable => WriteValueResponse.Failure(s"Unsuccessful write of value with id ${data.id.toString}") } 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/client/StorageNodeClient.scala: -------------------------------------------------------------------------------- 1 | package justin.db.client 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.Data 6 | import justin.db.replica.{R, W} 7 | import justin.db.replica.W 8 | 9 | import scala.concurrent.Future 10 | 11 | trait StorageNodeClient { 12 | def get(id: UUID, r: R): Future[GetValueResponse] 13 | def write(data: Data, w: W): Future[WriteValueResponse] 14 | } 15 | 16 | sealed trait GetValueResponse 17 | object GetValueResponse { 18 | case class Found(data: Data) extends GetValueResponse 19 | case class Conflicts(data: List[Data]) extends GetValueResponse 20 | case class NotFound(id: UUID) extends GetValueResponse 21 | case class Failure(error: String) extends GetValueResponse 22 | } 23 | 24 | sealed trait WriteValueResponse 25 | object WriteValueResponse { 26 | case class Success(id: UUID) extends WriteValueResponse 27 | case object Conflict extends WriteValueResponse 28 | case class Failure(error: String) extends WriteValueResponse 29 | } 30 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/cluster/ClusterMembers.scala: -------------------------------------------------------------------------------- 1 | package justin.db.cluster 2 | 3 | import justin.db.actors.StorageNodeActorRef 4 | import justin.db.consistenthashing.NodeId 5 | 6 | case class ClusterMembers(private val members: Map[NodeId, StorageNodeActorRef]) { 7 | 8 | def contains(nodeId: NodeId): Boolean = members.contains(nodeId) 9 | def notContains(nodeId: NodeId): Boolean = !contains(nodeId) 10 | 11 | def add(nodeId: NodeId, ref: StorageNodeActorRef): ClusterMembers = { 12 | ClusterMembers(this.members + (nodeId -> ref)) 13 | } 14 | 15 | def get(nodeId: NodeId): Option[StorageNodeActorRef] = members.get(nodeId) 16 | 17 | def removeByRef(ref: StorageNodeActorRef): ClusterMembers = { 18 | val filteredMembers = members.filterNot { case (_, sRef) => sRef == ref } 19 | ClusterMembers(filteredMembers) 20 | } 21 | 22 | def size: Int = members.size 23 | 24 | override def toString: String = members.toString() 25 | } 26 | 27 | object ClusterMembers { 28 | def empty: ClusterMembers = ClusterMembers(Map.empty[NodeId, StorageNodeActorRef]) 29 | } 30 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/cluster/datacenter/Datacenter.scala: -------------------------------------------------------------------------------- 1 | package justin.db.cluster.datacenter 2 | 3 | case class Datacenter(name: String) extends AnyVal 4 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/DataSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.util.UUID 4 | 5 | import com.esotericsoftware.kryo.io.{Input, Output} 6 | import com.esotericsoftware.kryo.{Kryo, Serializer} 7 | import justin.db.Data 8 | import justin.db.versioning.NodeIdVectorClockBase64 9 | 10 | object DataSerializer extends Serializer[Data] { 11 | override def write(kryo: Kryo, output: Output, data: Data): Unit = { 12 | output.writeString(data.id.toString) // UUID 13 | output.writeString(data.value) // Value 14 | output.writeString(new NodeIdVectorClockBase64().encode(data.vclock).get) // Vector Clock 15 | output.writeLong(data.timestamp) // Timestamp 16 | } 17 | 18 | override def read(kryo: Kryo, input: Input, `type`: Class[Data]): Data = { 19 | Data( 20 | id = UUID.fromString(input.readString()), // UUID 21 | value = input.readString(), // Value 22 | vclock = new NodeIdVectorClockBase64().decode(input.readString()).get, // Vector Clock 23 | timestamp = input.readLong() // Timestamp 24 | ) 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/ListOfDataSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import com.esotericsoftware.kryo.io.{Input, Output} 4 | import com.esotericsoftware.kryo.{Kryo, Serializer} 5 | import justin.db.Data 6 | 7 | object ListOfDataSerializer extends Serializer[List[Data]] { 8 | override def write(kryo: Kryo, output: Output, listOfData: List[Data]): Unit = { 9 | val length = listOfData.size 10 | output.writeInt(length, true) 11 | if(length != 0) { 12 | val it = listOfData.iterator 13 | while(it.hasNext) 14 | DataSerializer.write(kryo, output, it.next()) 15 | } 16 | } 17 | 18 | override def read(kryo: Kryo, input: Input, `type`: Class[List[Data]]): List[Data] = { 19 | var length = input.readInt(true) 20 | var result = List.empty[Data] 21 | 22 | while(length > 0) { 23 | result = result :+ DataSerializer.read(kryo, input, classOf[Data]) 24 | length -= 1 25 | } 26 | result 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/RegisterNodeSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import com.esotericsoftware.kryo.io.{Input, Output} 4 | import com.esotericsoftware.kryo.{Kryo, Serializer} 5 | import justin.db.actors.protocol.RegisterNode 6 | import justin.db.consistenthashing.NodeId 7 | 8 | object RegisterNodeSerializer extends Serializer[RegisterNode] { 9 | override def write(kryo: Kryo, output: Output, registerNode: RegisterNode): Unit = { 10 | output.writeInt(registerNode.nodeId.id) 11 | } 12 | 13 | override def read(kryo: Kryo, input: Input, `type`: Class[RegisterNode]): RegisterNode = { 14 | RegisterNode(NodeId(input.readInt())) 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/SerializerInit.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import com.esotericsoftware.kryo.Kryo 4 | import com.typesafe.scalalogging.StrictLogging 5 | 6 | class SerializerInit extends StrictLogging { 7 | 8 | def customize(kryo: Kryo): Unit = { 9 | logger.info("Initialized Kryo") 10 | 11 | // cluster 12 | kryo.register(classOf[justin.db.actors.protocol.RegisterNode], RegisterNodeSerializer, 50) 13 | 14 | // write -- request 15 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeWriteDataLocal], StorageNodeWriteDataLocalSerializer, 60) 16 | 17 | // write -- responses 18 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeFailedWrite], StorageNodeWriteResponseSerializer, 70) 19 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeSuccessfulWrite], StorageNodeWriteResponseSerializer, 71) 20 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeConflictedWrite], StorageNodeWriteResponseSerializer, 72) 21 | 22 | // read - request 23 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeLocalRead], StorageNodeLocalReadSerializer, 80) 24 | 25 | // read - responses 26 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeFoundRead], StorageNodeReadResponseSerializer, 90) 27 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeConflictedRead], StorageNodeReadResponseSerializer, 91) 28 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeNotFoundRead], StorageNodeReadResponseSerializer, 92) 29 | kryo.register(classOf[justin.db.actors.protocol.StorageNodeFailedRead], StorageNodeReadResponseSerializer, 93) 30 | 31 | () 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/StorageNodeLocalReadSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.util.UUID 4 | 5 | import com.esotericsoftware.kryo.io.{Input, Output} 6 | import com.esotericsoftware.kryo.{Kryo, Serializer} 7 | import justin.db.actors.protocol.StorageNodeLocalRead 8 | 9 | object StorageNodeLocalReadSerializer extends Serializer[StorageNodeLocalRead] { 10 | 11 | override def write(kryo: Kryo, output: Output, localRead: StorageNodeLocalRead): Unit = { 12 | output.writeString(localRead.id.toString) 13 | } 14 | 15 | override def read(kryo: Kryo, input: Input, `type`: Class[StorageNodeLocalRead]): StorageNodeLocalRead = { 16 | StorageNodeLocalRead(UUID.fromString(input.readString())) 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/StorageNodeReadResponseSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.util.UUID 4 | 5 | import com.esotericsoftware.kryo.io.{Input, Output} 6 | import com.esotericsoftware.kryo.{Kryo, Serializer} 7 | import justin.db.Data 8 | import justin.db.actors.protocol._ 9 | 10 | object StorageNodeReadResponseSerializer extends Serializer[StorageNodeReadResponse] { 11 | 12 | private object Discriminator { 13 | val Found = 1 14 | val Conflicted = 2 15 | val NotFound = 3 16 | val Failed = 4 17 | } 18 | 19 | override def write(kryo: Kryo, output: Output, readResponse: StorageNodeReadResponse): Unit = readResponse match { 20 | case StorageNodeFoundRead(data) => 21 | output.writeInt(Discriminator.Found) 22 | DataSerializer.write(kryo, output, data) 23 | case StorageNodeConflictedRead(conflicts) => 24 | output.writeInt(Discriminator.Conflicted) 25 | ListOfDataSerializer.write(kryo, output, conflicts) 26 | case StorageNodeNotFoundRead(id) => 27 | output.writeInt(Discriminator.NotFound) 28 | output.writeString(id.toString) 29 | case StorageNodeFailedRead(id) => 30 | output.writeInt(Discriminator.Failed) 31 | output.writeString(id.toString) 32 | } 33 | 34 | override def read(kryo: Kryo, input: Input, `type`: Class[StorageNodeReadResponse]): StorageNodeReadResponse = { 35 | input.readInt() match { 36 | case Discriminator.Found => StorageNodeFoundRead(DataSerializer.read(kryo, input, classOf[Data])) 37 | case Discriminator.Conflicted => StorageNodeConflictedRead(ListOfDataSerializer.read(kryo, input, classOf[List[Data]])) 38 | case Discriminator.NotFound => StorageNodeNotFoundRead(UUID.fromString(input.readString())) 39 | case Discriminator.Failed => StorageNodeFailedRead(UUID.fromString(input.readString())) 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/StorageNodeWriteDataLocalSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.util.UUID 4 | 5 | import com.esotericsoftware.kryo.io.{Input, Output} 6 | import com.esotericsoftware.kryo.{Kryo, Serializer} 7 | import justin.db.Data 8 | import justin.db.actors.protocol.StorageNodeWriteDataLocal 9 | import justin.db.versioning.NodeIdVectorClockBase64 10 | 11 | object StorageNodeWriteDataLocalSerializer extends Serializer[StorageNodeWriteDataLocal] { 12 | override def write(kryo: Kryo, output: Output, local: StorageNodeWriteDataLocal): Unit = { 13 | output.writeString(local.data.id.toString) // UUID 14 | output.writeString(local.data.value) // Value 15 | output.writeString(new NodeIdVectorClockBase64().encode(local.data.vclock).get) // Vector Clock 16 | output.writeLong(local.data.timestamp) // Timestamp 17 | } 18 | 19 | override def read(kryo: Kryo, input: Input, `type`: Class[StorageNodeWriteDataLocal]): StorageNodeWriteDataLocal = { 20 | val id = UUID.fromString(input.readString()) // UUID 21 | val value = input.readString() // Value 22 | val vectorClock = new NodeIdVectorClockBase64().decode(input.readString()).get // Vector Clock 23 | val timestamp = input.readLong() // Timestamp 24 | 25 | StorageNodeWriteDataLocal(Data(id, value, vectorClock, timestamp)) 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/kryo/StorageNodeWriteResponseSerializer.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.util.UUID 4 | 5 | import com.esotericsoftware.kryo.io.{Input, Output} 6 | import com.esotericsoftware.kryo.{Kryo, Serializer} 7 | import justin.db.Data 8 | import justin.db.actors.protocol.{StorageNodeConflictedWrite, StorageNodeFailedWrite, StorageNodeSuccessfulWrite, StorageNodeWriteResponse} 9 | 10 | object StorageNodeWriteResponseSerializer extends Serializer[StorageNodeWriteResponse] { 11 | 12 | private object Discriminator { 13 | val SuccessfulWrite = 1 14 | val FailedWrite = 2 15 | val ConflictedWrite = 3 16 | } 17 | 18 | override def write(kryo: Kryo, output: Output, response: StorageNodeWriteResponse): Unit = response match { 19 | case StorageNodeSuccessfulWrite(id) => 20 | output.writeInt(Discriminator.SuccessfulWrite) 21 | output.writeString(id.toString) // UUID 22 | case StorageNodeFailedWrite(id) => 23 | output.writeInt(Discriminator.FailedWrite) 24 | output.writeString(id.toString) // UUID 25 | case StorageNodeConflictedWrite(oldData, newData) => 26 | output.writeInt(Discriminator.ConflictedWrite) 27 | DataSerializer.write(kryo, output, oldData) 28 | DataSerializer.write(kryo, output, newData) 29 | } 30 | 31 | override def read(kryo: Kryo, input: Input, `type`: Class[StorageNodeWriteResponse]): StorageNodeWriteResponse = input.readInt() match { 32 | case Discriminator.SuccessfulWrite => 33 | StorageNodeSuccessfulWrite(UUID.fromString(input.readString())) 34 | case Discriminator.FailedWrite => 35 | StorageNodeFailedWrite(UUID.fromString(input.readString())) 36 | case Discriminator.ConflictedWrite => 37 | StorageNodeConflictedWrite( 38 | oldData = DataSerializer.read(kryo, input, classOf[Data]), 39 | newData = DataSerializer.read(kryo, input, classOf[Data]) 40 | ) 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/merkletrees/MerkleDigest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.merkletrees 2 | 3 | trait MerkleDigest[T] { 4 | def digest(t: T): Digest 5 | } 6 | 7 | object MerkleDigest { 8 | 9 | implicit object CRC32 extends MerkleDigest[Block] { 10 | import java.nio.ByteBuffer 11 | import java.util.zip.CRC32 12 | 13 | override def digest(t: Block): Digest = { 14 | val digest = new CRC32() 15 | digest.update(t) 16 | 17 | val buffer = ByteBuffer.allocate(8) 18 | buffer.putLong(digest.getValue) 19 | 20 | Digest(buffer.array()) 21 | } 22 | } 23 | 24 | implicit object MD5 extends MerkleDigest[Block] { 25 | import java.security.MessageDigest 26 | 27 | override def digest(t: Block): Digest = { 28 | Digest(MessageDigest.getInstance("MD5").digest(t)) 29 | } 30 | } 31 | } 32 | 33 | case class Digest(hash: Array[Byte]) extends AnyVal { 34 | def +(that: Digest): Digest = Digest(this.hash ++ that.hash) 35 | def ==(that: Digest): Boolean = this.hash.deep == that.hash.deep 36 | } 37 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/merkletrees/MerkleTree.scala: -------------------------------------------------------------------------------- 1 | package justin.db.merkletrees 2 | 3 | import scala.annotation.tailrec 4 | import scala.util.Try 5 | 6 | case class MerkleNodeId(id: Int) extends AnyVal 7 | 8 | sealed trait MerkleTree { 9 | def nodeId: MerkleNodeId 10 | def digest: Digest 11 | } 12 | case class MerkleHashNode(nodeId: MerkleNodeId, digest: Digest, left: MerkleTree, right: MerkleTree) extends MerkleTree 13 | case class MerkleLeaf(nodeId: MerkleNodeId, digest: Digest) extends MerkleTree 14 | 15 | object MerkleTree { 16 | 17 | def unapply(blocks: Seq[Block])(implicit ev: MerkleDigest[Block]): Option[MerkleTree] = unapply(blocks.toArray) 18 | 19 | def unapply(blocks: Array[Block])(implicit ev: MerkleDigest[Block]): Option[MerkleTree] = { 20 | sealed trait TempMerkleTree { def digest: Digest } 21 | case class TempMerkleHashNode(digest: Digest, left: TempMerkleTree, right: TempMerkleTree) extends TempMerkleTree 22 | case class TempMerkleLeaf(digest: Digest) extends TempMerkleTree 23 | 24 | def blockToLeaf(b: Block): TempMerkleLeaf = TempMerkleLeaf(ev.digest(b)) 25 | 26 | def buildTree(blocks: Array[Block]) = Try { 27 | val leafs = blocks.map(blockToLeaf) 28 | var trees: Seq[TempMerkleTree] = leafs.toSeq 29 | 30 | while (trees.length > 1) { 31 | trees = trees.grouped(2) 32 | .map(x => mergeTrees(x(0), x(1))) 33 | .toSeq 34 | } 35 | trees.head 36 | } 37 | 38 | def mergeTrees(n1: TempMerkleTree, n2: TempMerkleTree) = { 39 | val mergedDigest = n1.digest + n2.digest 40 | val hash = ev.digest(mergedDigest.hash) 41 | TempMerkleHashNode(hash, n1, n2) 42 | } 43 | 44 | def toFinalForm(tmt: TempMerkleTree): MerkleTree = { 45 | var counter = -1 46 | 47 | def toMerkle(mt: TempMerkleTree): MerkleTree = { 48 | counter += 1 49 | mt match { 50 | case TempMerkleHashNode(digest, left, right) => MerkleHashNode(MerkleNodeId(counter), digest, toMerkle(left), toMerkle(right)) 51 | case TempMerkleLeaf(digest) => MerkleLeaf(MerkleNodeId(counter), digest) 52 | } 53 | } 54 | toMerkle(tmt) 55 | } 56 | 57 | buildTree(blocks ++ zeroed(blocks)) 58 | .toOption 59 | .map(toFinalForm) 60 | } 61 | 62 | def zeroed(blocks: Seq[Block]): Array[Array[Byte]] = { 63 | def zero(i: Int): Int = { 64 | val factor = 2 65 | var x = factor 66 | while(x < i) x *= factor 67 | x - i 68 | } 69 | Array.fill(zero(blocks.length))(Array[Byte](0)) 70 | } 71 | 72 | @tailrec 73 | def findNode(nodeId: MerkleNodeId, merkleTree: MerkleTree): Option[MerkleTree] = { 74 | merkleTree match { 75 | case _ if merkleTree.nodeId == nodeId => Option(merkleTree) 76 | case MerkleHashNode(nId, _, _, right) if nodeId.id >= right.nodeId.id => findNode(nodeId, right) 77 | case MerkleHashNode(nId, _, left, _) => findNode(nodeId, left) 78 | case _ => None 79 | } 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/merkletrees/package.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | package object merkletrees { 4 | 5 | type Block = Array[Byte] 6 | } 7 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/package.scala: -------------------------------------------------------------------------------- 1 | package justin 2 | 3 | package object db { 4 | def converge[T](result: (T, List[T])): List[T] = result._1 :: result._2 5 | } 6 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/IsPrimaryOrReplica.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.consistenthashing.{NodeId, Ring, UUID2RingPartitionId} 6 | import justin.db.storage.PluggableStorageProtocol.DataOriginality 7 | 8 | class IsPrimaryOrReplica(nodeId: NodeId, ring: Ring) extends (UUID => DataOriginality) { 9 | 10 | override def apply(id: UUID): DataOriginality = { 11 | val partitionId = UUID2RingPartitionId(id, ring) 12 | 13 | if(ring.getNodeId(partitionId).contains(nodeId)) { 14 | DataOriginality.Primary(partitionId) 15 | } else { 16 | DataOriginality.Replica(partitionId) 17 | } 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/PreferenceList.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import justin.db.consistenthashing.{NodeId, Ring} 4 | import justin.db.consistenthashing.Ring.RingPartitionId 5 | 6 | case class PreferenceList(primaryNodeId: NodeId, replicasNodeId: List[NodeId]) { 7 | def size: Int = all.size 8 | def all: List[NodeId] = primaryNodeId :: replicasNodeId 9 | } 10 | 11 | object PreferenceList { 12 | 13 | def apply(baseRingPartitionId: RingPartitionId, n: N, ring: Ring): Either[Error, PreferenceList] = { 14 | ring.getNodeId(baseRingPartitionId) match { 15 | case Some(coordinatorNodeId) => computePreferenceList(baseRingPartitionId, coordinatorNodeId, n, ring) 16 | case None => Left(LackOfCoordinator) 17 | } 18 | } 19 | 20 | private def computePreferenceList(baseRingPartitionId: RingPartitionId, coordinatorNodeId: NodeId, n: N, ring: Ring) = { 21 | val maxPartitionId = baseRingPartitionId + n.n - 1 22 | val restNodesIds = (baseRingPartitionId until maxPartitionId).map(getNextNodeId(ring)).flatten.distinct.filterNot(_ == coordinatorNodeId) 23 | val preferenceList = PreferenceList(coordinatorNodeId, restNodesIds.toList) 24 | 25 | if(preferenceList.size >= n.n) { 26 | Right(preferenceList) 27 | } else { 28 | Left(NotSufficientSize(preferenceList)) 29 | } 30 | } 31 | 32 | private def getNextNodeId(ring: Ring) = ring.nextPartitionId _ andThen ring.getNodeId _ 33 | 34 | sealed trait Error 35 | case object LackOfCoordinator extends Error 36 | case class NotSufficientSize(preferenceList: PreferenceList) extends Error 37 | } 38 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/ReplicationConfig.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | /** 4 | * the number of nodes to replicate to 5 | */ 6 | case class N(n: Int) extends AnyVal 7 | 8 | /** 9 | * the number of nodes read from before returning 10 | */ 11 | case class R(r: Int) extends AnyVal 12 | 13 | /** 14 | * the number of nodes written to before considered successful 15 | */ 16 | case class W(w: Int) extends AnyVal 17 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/ResolveNodeAddresses.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import justin.db.actors.StorageNodeActorRef 4 | import justin.db.cluster.ClusterMembers 5 | import justin.db.consistenthashing.NodeId 6 | 7 | case class ResolvedNodeAddresses(local: Boolean, remotes: List[StorageNodeActorRef]) 8 | 9 | object ResolveNodeAddresses { 10 | 11 | def apply(nodeId: NodeId, preferenceList: PreferenceList, clusterMembers: ClusterMembers): ResolvedNodeAddresses = { 12 | ResolvedNodeAddresses( 13 | local = preferenceList.all.contains(nodeId), 14 | remotes = preferenceList.all.flatMap(clusterMembers.get) 15 | ) 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/read/ReadAgreement.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import justin.db.Data 4 | 5 | 6 | sealed trait ReadAgreement 7 | 8 | object ReadAgreement { 9 | case object AllNotFound extends ReadAgreement 10 | case object AllFailed extends ReadAgreement 11 | case class Conflicts(data: List[Data]) extends ReadAgreement 12 | case object NotEnoughFound extends ReadAgreement 13 | // this should be chosen when all replicas agreed on the same value 14 | case class Found(data: Data) extends ReadAgreement 15 | // this should be chosen when not all replicas agreed on but one of it has consequent vector clock 16 | case class Consequent(data: Data) extends ReadAgreement 17 | } 18 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/read/ReplicaLocalReader.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.actors.protocol.{StorageNodeFailedRead, StorageNodeFoundRead, StorageNodeNotFoundRead, StorageNodeReadResponse} 6 | import justin.db.replica.IsPrimaryOrReplica 7 | import justin.db.storage.GetStorageProtocol 8 | import justin.db.storage.PluggableStorageProtocol.StorageGetData 9 | 10 | import scala.concurrent.{ExecutionContext, Future} 11 | 12 | class ReplicaLocalReader(storage: GetStorageProtocol)(implicit ec: ExecutionContext) { 13 | 14 | def apply(id: UUID, isPrimaryOrReplica: IsPrimaryOrReplica): Future[StorageNodeReadResponse] = { 15 | storage.get(id)(isPrimaryOrReplica).map { 16 | case StorageGetData.Single(justinData) => StorageNodeFoundRead(justinData) 17 | case StorageGetData.None => StorageNodeNotFoundRead(id) 18 | } recover { case _ => StorageNodeFailedRead(id) } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/read/ReplicaReadAgreement.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import justin.db.actors.protocol.{StorageNodeFailedRead, StorageNodeFoundRead, StorageNodeNotFoundRead, StorageNodeReadResponse} 4 | import justin.db.consistenthashing.NodeId 5 | import justin.db.replica.R 6 | import justin.db.vectorclocks.VectorClockComparator 7 | import justin.db.vectorclocks.VectorClockComparator.VectorClockRelation 8 | 9 | class ReplicaReadAgreement { 10 | 11 | def reach(r: R): List[StorageNodeReadResponse] => ReadAgreement = { reads => 12 | if(areAllNotFound(reads)) { 13 | ReadAgreement.AllNotFound 14 | } else if(areAllFailed(reads)) { 15 | ReadAgreement.AllFailed 16 | } else { 17 | val onlyFoundReads = collectFound(reads) 18 | (onlyFoundReads.size >= r.r, onlyFoundReads.size == 1, hasSameVC(onlyFoundReads), foundOnlyConsequent(onlyFoundReads)) match { 19 | case (true, true, _, _) => ReadAgreement.Found(onlyFoundReads.head.data) 20 | case (true, false, true, _) => ReadAgreement.Found(onlyFoundReads.head.data) 21 | case (true, false, _, c) if c.size == 1 => ReadAgreement.Consequent(c.head._1) 22 | case (true, false, _, _) => ReadAgreement.Conflicts(onlyFoundReads.map(_.data)) 23 | case (false, _, _, _) => ReadAgreement.NotEnoughFound 24 | } 25 | } 26 | } 27 | 28 | private def areAllNotFound(reads: List[StorageNodeReadResponse]) = reads.collect { case nf: StorageNodeNotFoundRead => nf }.size == reads.size 29 | 30 | private def areAllFailed(reads: List[StorageNodeReadResponse]) = reads.collect { case fr: StorageNodeFailedRead => fr }.size == reads.size 31 | 32 | private def collectFound(reads: List[StorageNodeReadResponse]) = reads.collect { case r: StorageNodeFoundRead => r } 33 | 34 | private def hasSameVC(onlyFoundReads: List[StorageNodeFoundRead]) = onlyFoundReads.map(_.data.vclock).distinct.size == 1 35 | 36 | private def foundOnlyConsequent(onlyFoundReads: List[StorageNodeFoundRead]) = { 37 | val vcComparator = new VectorClockComparator[NodeId] 38 | 39 | onlyFoundReads.flatMap { compared => 40 | onlyFoundReads.filterNot(_ == compared) 41 | .map(base => (compared.data, vcComparator.apply(base.data.vclock, compared.data.vclock))) 42 | .groupBy { case (data, _) => data } 43 | .filter { case (_, l) => l.forall { case (_, relation) => relation == VectorClockRelation.Consequent }} 44 | } 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/read/ReplicaReadCoordinator.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import java.util.UUID 4 | 5 | import justin.db._ 6 | import justin.db.actors.protocol._ 7 | import justin.db.cluster.ClusterMembers 8 | import justin.db.consistenthashing.{NodeId, Ring, UUID2RingPartitionId} 9 | import justin.db.replica._ 10 | 11 | import scala.concurrent.{ExecutionContext, Future} 12 | 13 | class ReplicaReadCoordinator( 14 | nodeId: NodeId, ring: Ring, n: N, 15 | localDataReader: ReplicaLocalReader, 16 | remoteDataReader: ReplicaRemoteReader 17 | )(implicit ec: ExecutionContext) extends ((StorageNodeReadRequest, ClusterMembers) => Future[StorageNodeReadResponse]) { 18 | 19 | override def apply(cmd: StorageNodeReadRequest, clusterMembers: ClusterMembers): Future[StorageNodeReadResponse] = cmd match { 20 | case StorageNodeLocalRead(id) => readLocalData(id) 21 | case Internal.ReadReplica(r, id) => coordinateReplicated(r, id, clusterMembers) 22 | } 23 | 24 | private def readLocalData(id: UUID) = localDataReader.apply(id, new IsPrimaryOrReplica(nodeId, ring)) 25 | 26 | private def coordinateReplicated(r: R, id: UUID, clusterMembers: ClusterMembers) = { 27 | val partitionId = UUID2RingPartitionId.apply(id, ring) 28 | PreferenceList(partitionId, n, ring).fold(onLeft(id), onRight(r, id, clusterMembers)) 29 | } 30 | 31 | private def onLeft(id: UUID)(err: PreferenceList.Error) = Future.successful(StorageNodeFailedRead(id)) 32 | 33 | private def onRight(r: R, id: UUID, clusterMembers: ClusterMembers)(preferenceList: PreferenceList) = { 34 | gatherReads(r, id, clusterMembers, preferenceList).map { reads => 35 | val consensus = new ReplicaReadAgreement().reach(r)(reads) 36 | consensus2ReadingResult(id)(consensus) 37 | } 38 | } 39 | 40 | private def gatherReads(r: R, id: UUID, clusterMembers: ClusterMembers, preferenceList: PreferenceList) = { 41 | ResolveNodeAddresses(nodeId, preferenceList, clusterMembers) match { 42 | case ResolvedNodeAddresses(true, remotes) if remotes.size + 1 >= r.r => (readLocalData(id) zip remoteDataReader.apply(remotes, id)).map(converge) 43 | case ResolvedNodeAddresses(false, remotes) if remotes.size >= r.r => remoteDataReader.apply(remotes, id) 44 | case _ => Future.successful(List(StorageNodeFailedRead(id))) 45 | } 46 | } 47 | 48 | private def consensus2ReadingResult(id: => UUID): ReadAgreement => StorageNodeReadResponse = { 49 | case ReadAgreement.Consequent(data) => StorageNodeFoundRead(data) 50 | case ReadAgreement.Found(data) => StorageNodeFoundRead(data) 51 | case ReadAgreement.Conflicts(data) => StorageNodeConflictedRead(data) 52 | case ReadAgreement.NotEnoughFound => StorageNodeNotFoundRead(id) 53 | case ReadAgreement.AllFailed => StorageNodeFailedRead(id) 54 | case ReadAgreement.AllNotFound => StorageNodeNotFoundRead(id) 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/read/ReplicaRemoteReader.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import java.util.UUID 4 | 5 | import akka.pattern.ask 6 | import akka.util.Timeout 7 | import justin.db.actors.StorageNodeActorRef 8 | import justin.db.actors.protocol.{StorageNodeFailedRead, StorageNodeLocalRead, StorageNodeReadResponse} 9 | 10 | import scala.concurrent.duration._ 11 | import scala.concurrent.{ExecutionContext, Future} 12 | 13 | class ReplicaRemoteReader(implicit ec: ExecutionContext) { 14 | 15 | private implicit val timeout = Timeout(3.seconds) // TODO: tune this value 16 | 17 | def apply(storageNodeRefs: List[StorageNodeActorRef], id: UUID): Future[List[StorageNodeReadResponse]] = { 18 | Future.sequence(storageNodeRefs.map(getValue(_, id))) 19 | } 20 | 21 | private def getValue(node: StorageNodeActorRef, id: UUID): Future[StorageNodeReadResponse] = { 22 | (node.ref ? StorageNodeLocalRead(id)) 23 | .mapTo[StorageNodeReadResponse] 24 | .recover { case _ => StorageNodeFailedRead(id) } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/write/ReplicaLocalWriter.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import justin.db.Data 4 | import justin.db.actors.protocol.{StorageNodeConflictedWrite, StorageNodeFailedWrite, StorageNodeSuccessfulWrite, StorageNodeWriteResponse} 5 | import justin.db.replica.IsPrimaryOrReplica 6 | import justin.db.storage.PluggableStorageProtocol.StorageGetData 7 | import justin.db.storage.{GetStorageProtocol, PutStorageProtocol} 8 | import justin.db.vectorclocks.VectorClockComparator 9 | import justin.db.vectorclocks.VectorClockComparator.VectorClockRelation 10 | 11 | import scala.concurrent.{ExecutionContext, Future} 12 | 13 | class ReplicaLocalWriter(storage: GetStorageProtocol with PutStorageProtocol)(implicit ec: ExecutionContext) { 14 | 15 | def apply(newData: Data, isPrimaryOrReplica: IsPrimaryOrReplica): Future[StorageNodeWriteResponse] = { 16 | storage.get(newData.id)(isPrimaryOrReplica).flatMap { 17 | case StorageGetData.None => putSingleSuccessfulWrite(newData, isPrimaryOrReplica) 18 | case StorageGetData.Single(oldData) => handleExistedSingleData(oldData, newData, isPrimaryOrReplica) 19 | } recover { case _ => StorageNodeFailedWrite(newData.id) } 20 | } 21 | 22 | private def handleExistedSingleData(oldData: Data, newData: Data, isPrimaryOrReplica: IsPrimaryOrReplica) = { 23 | new VectorClockComparator().apply(oldData.vclock, newData.vclock) match { 24 | case VectorClockRelation.Predecessor => Future.successful(StorageNodeFailedWrite(newData.id)) 25 | case VectorClockRelation.Conflict => Future.successful(StorageNodeConflictedWrite(oldData, newData)) 26 | case VectorClockRelation.Consequent => putSingleSuccessfulWrite(newData, isPrimaryOrReplica) 27 | } 28 | } 29 | 30 | private def putSingleSuccessfulWrite(newData: Data, resolveDataOriginality: IsPrimaryOrReplica) = { 31 | storage.put(newData)(resolveDataOriginality).map(_ => StorageNodeSuccessfulWrite(newData.id)) 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/write/ReplicaRemoteWriter.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import akka.pattern.ask 4 | import akka.util.Timeout 5 | import justin.db.Data 6 | import justin.db.actors.StorageNodeActorRef 7 | import justin.db.actors.protocol.{StorageNodeFailedWrite, StorageNodeWriteDataLocal, StorageNodeWriteResponse} 8 | 9 | import scala.concurrent.duration._ 10 | import scala.concurrent.{ExecutionContext, Future} 11 | 12 | class ReplicaRemoteWriter(implicit ec: ExecutionContext) { 13 | 14 | private implicit val timeout = Timeout(3.seconds) // TODO: tune this value 15 | 16 | def apply(storageNodeRefs: List[StorageNodeActorRef], data: Data): Future[List[StorageNodeWriteResponse]] = { 17 | Future.sequence(storageNodeRefs.map(putLocalValue(_, data))) 18 | } 19 | 20 | private def putLocalValue(node: StorageNodeActorRef, data: Data): Future[StorageNodeWriteResponse] = { 21 | (node.ref ? StorageNodeWriteDataLocal(data)) 22 | .mapTo[StorageNodeWriteResponse] 23 | .recover { case _ => StorageNodeFailedWrite(data.id) } 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/write/ReplicaWriteAgreement.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import justin.db.actors.protocol.{StorageNodeSuccessfulWrite, StorageNodeWriteResponse} 4 | import justin.db.replica.W 5 | 6 | class ReplicaWriteAgreement { 7 | 8 | // TODO: more cases should be taken into account e.g. what if all writes were failed or one of it is conflicted? 9 | def reach(w: W): List[StorageNodeWriteResponse] => WriteAgreement = { 10 | writes => 11 | val okWrites = writes.collect { case ok: StorageNodeSuccessfulWrite => ok }.size 12 | if(okWrites >= w.w) WriteAgreement.Ok else WriteAgreement.NotEnoughWrites 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/write/ReplicaWriteCoordinator.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import java.util.UUID 4 | 5 | import justin.db._ 6 | import justin.db.actors.protocol._ 7 | import justin.db.cluster.ClusterMembers 8 | import justin.db.consistenthashing.{NodeId, Ring, UUID2RingPartitionId} 9 | import justin.db.replica._ 10 | 11 | import scala.concurrent.{ExecutionContext, Future} 12 | 13 | class ReplicaWriteCoordinator( 14 | nodeId: NodeId, ring: Ring, n: N, 15 | localDataWriter: ReplicaLocalWriter, 16 | remoteDataWriter: ReplicaRemoteWriter 17 | )(implicit ec: ExecutionContext) extends ((StorageNodeWriteRequest, ClusterMembers) => Future[StorageNodeWriteResponse]) { 18 | 19 | override def apply(cmd: StorageNodeWriteRequest, clusterMembers: ClusterMembers): Future[StorageNodeWriteResponse] = cmd match { 20 | case StorageNodeWriteDataLocal(data) => writeLocal(data) 21 | case Internal.WriteReplica(w, data) => coordinateReplicated(w, data, clusterMembers) 22 | } 23 | 24 | private def writeLocal(data: Data) = localDataWriter.apply(data, new IsPrimaryOrReplica(nodeId, ring)) 25 | 26 | private def coordinateReplicated(w: W, data: Data, clusterMembers: ClusterMembers) = { 27 | val ringPartitionId = UUID2RingPartitionId.apply(data.id, ring) 28 | PreferenceList(ringPartitionId, n, ring).fold(onLeft(data.id), onRight(w, data, clusterMembers)) 29 | } 30 | 31 | // TODO: rename to "onFailure" 32 | private def onLeft(id: UUID)(err: PreferenceList.Error) = Future.successful(StorageNodeFailedWrite(id)) 33 | 34 | // TODO: rename to "onSuccess" 35 | private def onRight(w: W, data: Data, clusterMembers: ClusterMembers)(preferenceList: PreferenceList) = { 36 | val updatedData = Data.updateVclock(data, preferenceList) 37 | makeWrites(w, updatedData, clusterMembers, preferenceList) 38 | .map(new ReplicaWriteAgreement().reach(w)) 39 | .map(consensus2WritingResult(updatedData.id)) 40 | } 41 | 42 | private def makeWrites(w: W, updatedData: Data, clusterMembers: ClusterMembers, preferenceList: PreferenceList) = { 43 | ResolveNodeAddresses(nodeId, preferenceList, clusterMembers) match { 44 | case ResolvedNodeAddresses(true, remotes) if remotes.size + 1 >= w.w => (writeLocal(updatedData) zip remoteDataWriter(remotes, updatedData)).map(converge) 45 | case ResolvedNodeAddresses(false, remotes) if remotes.size >= w.w => remoteDataWriter(remotes, updatedData) 46 | case _ => Future.successful(List(StorageNodeFailedWrite(updatedData.id))) 47 | } 48 | } 49 | 50 | private def consensus2WritingResult(id: UUID): WriteAgreement => StorageNodeWriteResponse = { 51 | case WriteAgreement.NotEnoughWrites => StorageNodeFailedWrite(id) 52 | case WriteAgreement.Ok => StorageNodeSuccessfulWrite(id) 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/replica/write/WriteAgreement.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | sealed trait WriteAgreement 4 | object WriteAgreement { 5 | case object NotEnoughWrites extends WriteAgreement 6 | case object Ok extends WriteAgreement 7 | } 8 | -------------------------------------------------------------------------------- /justin-core/src/main/scala/justin/db/versioning/NodeIdVectorClockBase64.scala: -------------------------------------------------------------------------------- 1 | package justin.db.versioning 2 | 3 | import java.nio.charset.{Charset, StandardCharsets} 4 | import java.util.Base64 5 | 6 | import justin.db.consistenthashing.NodeId 7 | import justin.db.vectorclocks.{Counter, VectorClock} 8 | import spray.json.DefaultJsonProtocol._ 9 | import spray.json._ 10 | 11 | import scala.util.Try 12 | 13 | object NodeIdVectorClockBase64 { 14 | val charset: Charset = StandardCharsets.UTF_8 15 | } 16 | 17 | class NodeIdVectorClockBase64 { 18 | import NodeIdVectorClockBase64._ 19 | 20 | def encode(vclock: VectorClock[NodeId]): Try[String] = Try { 21 | val vcClockBytes = vclock.toList 22 | .map { case (nodeId, counter) => (nodeId.id.toString, counter.value) } 23 | .toJson 24 | .compactPrint 25 | .getBytes(charset) 26 | 27 | Base64.getEncoder.encodeToString(vcClockBytes) 28 | } 29 | 30 | def decode(base64: String): Try[VectorClock[NodeId]] = Try { 31 | val decodedMap = new String(Base64.getDecoder.decode(base64), charset) 32 | .parseJson.convertTo[List[(String, Int)]] 33 | .map { case (k, v) => (NodeId(k.toInt), Counter(v))} 34 | .toMap 35 | 36 | VectorClock.apply(decodedMap) 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/ConvergeTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class ConvergeTest extends FlatSpec with Matchers { 6 | 7 | behavior of "converge" 8 | 9 | it should "converge tuple" in { 10 | // given 11 | val tuple = (1, List(2,3,4,5)) 12 | 13 | // when 14 | val result = justin.db.converge(tuple) 15 | 16 | // then 17 | result shouldBe List(1,2,3,4,5) 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/DataTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.consistenthashing.NodeId 6 | import justin.db.replica.PreferenceList 7 | import justin.db.vectorclocks.{Counter, VectorClock} 8 | import org.scalatest.{FlatSpec, Matchers} 9 | 10 | class DataTest extends FlatSpec with Matchers { 11 | 12 | behavior of "Data" 13 | 14 | it should "update its empty inner Vector Clock based on preference list" in { 15 | // given 16 | val data = Data(id = UUID.randomUUID(), value = "some value") 17 | val preferenceList = PreferenceList(NodeId(1), List(NodeId(5), NodeId(8))) 18 | 19 | // when 20 | val updatedData = Data.updateVclock(data, preferenceList) 21 | 22 | // then 23 | val expectedVclock = VectorClock[NodeId](Map( 24 | NodeId(1) -> Counter(1), 25 | NodeId(5) -> Counter(1), 26 | NodeId(8) -> Counter(1)) 27 | ) 28 | updatedData shouldBe Data(data.id, data.value, expectedVclock, updatedData.timestamp) 29 | } 30 | 31 | it should "increase vector clock's counter of repeated nodeId when updating data" in { 32 | // given 33 | val data = Data(id = UUID.randomUUID(), value = "some value") 34 | val preferenceList = PreferenceList(NodeId(1), List(NodeId(1), NodeId(1))) 35 | 36 | // when 37 | val updatedData = Data.updateVclock(data, preferenceList) 38 | 39 | // then 40 | val expectedVclock = VectorClock[NodeId](Map( 41 | NodeId(1) -> Counter(3) 42 | )) 43 | updatedData shouldBe Data(data.id, data.value, expectedVclock, data.timestamp) 44 | } 45 | 46 | it should "increase already existed vector clock's counter when updating data" in { 47 | // given 48 | val initVClock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))) 49 | val data = Data(id = UUID.randomUUID(), value = "some value", initVClock) 50 | val preferenceList = PreferenceList(NodeId(1), List(NodeId(5), NodeId(8))) 51 | 52 | // when 53 | val updatedData = Data.updateVclock(data, preferenceList) 54 | 55 | // then 56 | val expectedVclock = VectorClock[NodeId](Map( 57 | NodeId(1) -> Counter(4), 58 | NodeId(5) -> Counter(1), 59 | NodeId(8) -> Counter(1)) 60 | ) 61 | updatedData shouldBe Data(data.id, data.value, expectedVclock, data.timestamp) 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/actors/RoundRobinCoordinatorRouterTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class RoundRobinCoordinatorRouterTest extends FlatSpec with Matchers { 6 | 7 | behavior of "Round-Robin coordinator Router" 8 | 9 | it should "get router name" in { 10 | RoundRobinCoordinatorRouter.routerName shouldBe "CoordinatorRouter" 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/actors/StorageNodeActorTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.actors 2 | 3 | import akka.actor.ActorSystem 4 | import akka.testkit.{ImplicitSender, TestActorRef, TestKit} 5 | import com.typesafe.config.ConfigFactory 6 | import justin.db.actors.protocol.RegisterNode 7 | import justin.db.cluster.datacenter.Datacenter 8 | import justin.db.consistenthashing.{NodeId, Ring} 9 | import justin.db.replica.N 10 | import org.scalatest.concurrent.ScalaFutures 11 | import org.scalatest.{BeforeAndAfterAll, FlatSpecLike, Matchers} 12 | 13 | class StorageNodeActorTest extends TestKit(StorageNodeActorTest.actorSystem) 14 | with FlatSpecLike 15 | with ImplicitSender 16 | with Matchers 17 | with ScalaFutures 18 | with BeforeAndAfterAll { 19 | 20 | behavior of "Storage Node Actor" 21 | 22 | it should "send msg back with targeted NodeId when registration of other node has correctly happened" in { 23 | // given 24 | val nodeId = NodeId(1) 25 | val testActor = TestActorRef(new TestActor(nodeId, Ring.apply(3, 1))) 26 | 27 | // when 28 | testActor ! RegisterNode(NodeId(2)) 29 | 30 | // then 31 | expectMsg(RegisterNode(nodeId)) 32 | } 33 | 34 | it should "has defined role \"storagenode\"" in { 35 | StorageNodeActor.role shouldBe "storagenode" 36 | } 37 | 38 | it should "compose its name based on datacenter it belongs to and given id" in { 39 | StorageNodeActor.name(NodeId(0), Datacenter("dc1")) shouldBe "dc1-id-0" 40 | StorageNodeActor.name(NodeId(10), Datacenter("dc2")) shouldBe "dc2-id-10" 41 | StorageNodeActor.name(NodeId(20), Datacenter("dc1")) shouldBe "dc1-id-20" 42 | StorageNodeActor.name(NodeId(999), Datacenter("dc1")) shouldBe "dc1-id-999" 43 | } 44 | 45 | override def afterAll: Unit = { 46 | TestKit.shutdownActorSystem(system) 47 | } 48 | 49 | class TestActor(nodeId: NodeId, ring: Ring) extends StorageNodeActor(nodeId, Datacenter("default"), null, ring, N(1)) 50 | } 51 | 52 | object StorageNodeActorTest { 53 | def actorSystem: ActorSystem = { 54 | val config = ConfigFactory.parseString( 55 | """ 56 | |akka.loglevel = off 57 | |akka.actor.provider = cluster 58 | |akka.cluster.auto-join = off 59 | |akka.cluster.metrics.enabled = off 60 | """.stripMargin).withFallback(ConfigFactory.load()) 61 | 62 | ActorSystem("test-system", config) 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/client/ActorRefStorageNodeClientTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.client 2 | 3 | import java.util.UUID 4 | 5 | import akka.actor.{Actor, ActorSystem} 6 | import akka.testkit.{TestActorRef, TestKit} 7 | import justin.db.Data 8 | import justin.db.actors.StorageNodeActorRef 9 | import justin.db.actors.protocol._ 10 | import justin.db.consistenthashing.NodeId 11 | import justin.db.replica.{R, W} 12 | import justin.db.vectorclocks.{Counter, VectorClock} 13 | import org.scalatest.concurrent.ScalaFutures 14 | import org.scalatest.{BeforeAndAfterAll, FlatSpecLike, Matchers} 15 | 16 | class ActorRefStorageNodeClientTest extends TestKit(ActorSystem("test-system")) 17 | with FlatSpecLike 18 | with Matchers 19 | with ScalaFutures 20 | with BeforeAndAfterAll { 21 | 22 | behavior of "ActorRef Storage Node Client" 23 | 24 | /** 25 | * GET part 26 | */ 27 | it should "handle actor's \"Found\" message for asked data" in { 28 | // given 29 | val id = UUID.randomUUID() 30 | val data = Data(id, "value") 31 | val actorRef = getTestActorRef(msgBack = StorageNodeFoundRead(data)) 32 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 33 | 34 | // when 35 | val result = client.get(id, R(1)) 36 | 37 | // then 38 | whenReady(result) { _ shouldBe GetValueResponse.Found(data) } 39 | } 40 | 41 | it should "handle actor's \"NotFound\" message for asked data" in { 42 | // given 43 | val id = UUID.randomUUID() 44 | val actorRef = getTestActorRef(msgBack = StorageNodeNotFoundRead(id)) 45 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 46 | 47 | // when 48 | val result = client.get(id, R(1)) 49 | 50 | // then 51 | whenReady(result) { _ shouldBe GetValueResponse.NotFound(id) } 52 | } 53 | 54 | it should "handle actor's \"FailedRead\" message for asked data" in { 55 | // given 56 | val id = UUID.randomUUID() 57 | val actorRef = getTestActorRef(msgBack = StorageNodeFailedRead(id)) 58 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 59 | 60 | // when 61 | val result = client.get(id, R(1)) 62 | 63 | // then 64 | whenReady(result) { _ shouldBe GetValueResponse.Failure(s"Couldn't read value with id ${id.toString}") } 65 | } 66 | 67 | it should "handle actor's \"ConflictedRead\" message for asked data" in { 68 | // given 69 | val id = UUID.randomUUID() 70 | val oldData = Data( 71 | id = id, 72 | value = "some value 1", 73 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))), 74 | timestamp = System.currentTimeMillis() 75 | ) 76 | val newData = Data( 77 | id = id, 78 | value = "some value 2", 79 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(1))), 80 | timestamp = System.currentTimeMillis() 81 | ) 82 | val actorRef = getTestActorRef(msgBack = StorageNodeConflictedRead(List(oldData, newData))) 83 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 84 | 85 | // when 86 | val result = client.get(id, R(1)) 87 | 88 | // then 89 | whenReady(result) { _ shouldBe GetValueResponse.Conflicts(List(oldData, newData)) } 90 | } 91 | 92 | it should "recover actor's reading behavior" in { 93 | // given 94 | val id = UUID.randomUUID() 95 | val actorRef = getTestActorRef(msgBack = new Exception) 96 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 97 | 98 | // when 99 | val result = client.get(id, R(1)) 100 | 101 | // then 102 | whenReady(result) { _ shouldBe GetValueResponse.Failure(s"Unsuccessful read of value with id ${id.toString}") } 103 | } 104 | 105 | /** 106 | * WRITE part 107 | */ 108 | it should "handle actor's \"SuccessfulWrite\" message for data saving" in { 109 | // given 110 | val id = UUID.randomUUID() 111 | val data = Data(id, "value") 112 | val actorRef = writeTestActorRef(msgBack = StorageNodeSuccessfulWrite(id)) 113 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 114 | 115 | // when 116 | val result = client.write(data, W(1)) 117 | 118 | // then 119 | whenReady(result) { _ shouldBe WriteValueResponse.Success(id) } 120 | } 121 | 122 | it should "handle actor's \"FailedWrite\" message for data saving" in { 123 | // given 124 | val id = UUID.randomUUID() 125 | val data = Data(id, "value") 126 | val actorRef = writeTestActorRef(msgBack = StorageNodeFailedWrite(id)) 127 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 128 | 129 | // when 130 | val result = client.write(data, W(1)) 131 | 132 | // then 133 | whenReady(result) { _ shouldBe WriteValueResponse.Failure(s"Couldn't write value with id ${id.toString}") } 134 | } 135 | 136 | it should "recover actor's writing behavior" in { 137 | // given 138 | val id = UUID.randomUUID() 139 | val data = Data(id, "value") 140 | val actorRef = writeTestActorRef(msgBack = new Exception) 141 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 142 | 143 | // when 144 | val result = client.write(data, W(1)) 145 | 146 | // then 147 | whenReady(result) { _ shouldBe WriteValueResponse.Failure(s"Unsuccessful write of value with id ${id.toString}") } 148 | } 149 | 150 | it should "handle actor's \"ConflictedWrite\" message for data saving" in { 151 | // given 152 | val id = UUID.randomUUID() 153 | val data = Data(id, "value") 154 | val actorRef = writeTestActorRef(msgBack = StorageNodeConflictedWrite(data, data)) 155 | val client = new ActorRefStorageNodeClient(StorageNodeActorRef(actorRef))(system.dispatcher) 156 | 157 | // when 158 | val result = client.write(data, W(1)) 159 | 160 | // then 161 | whenReady(result) { _ shouldBe WriteValueResponse.Conflict } 162 | } 163 | 164 | override def afterAll: Unit = { 165 | TestKit.shutdownActorSystem(system) 166 | } 167 | 168 | private def getTestActorRef(msgBack: => Any) = { 169 | TestActorRef(new Actor { 170 | override def receive: Receive = { 171 | case Internal.ReadReplica(r, id) => sender() ! msgBack 172 | } 173 | }) 174 | } 175 | 176 | private def writeTestActorRef(msgBack: => Any) = { 177 | TestActorRef(new Actor { 178 | override def receive: Receive = { 179 | case Internal.WriteReplica(w, data) => sender() ! msgBack 180 | } 181 | }) 182 | } 183 | } 184 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/cluster/ClusterMembersTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.cluster 2 | 3 | import akka.actor.ActorRef 4 | import justin.db.actors.StorageNodeActorRef 5 | import justin.db.consistenthashing.NodeId 6 | import org.scalatest.{FlatSpec, Matchers} 7 | 8 | class ClusterMembersTest extends FlatSpec with Matchers { 9 | 10 | behavior of "Cluster Members" 11 | 12 | it should "define an \"empty\" method which create an empty ClusterMember with size 0" in { 13 | // given 14 | val expectedSize = 0 15 | 16 | // when 17 | val emtpyClusterMembers = ClusterMembers.empty 18 | 19 | // then 20 | emtpyClusterMembers.size shouldBe expectedSize 21 | } 22 | 23 | it should "define immutable \"add\" method for adding pair of NodeId with ActorRef" in { 24 | // given 25 | val emptyClusterMembers = ClusterMembers.empty 26 | val nodeId = NodeId(100) 27 | val ref = StorageNodeActorRef(ActorRef.noSender) 28 | 29 | // when 30 | val updatedClusterMembers = emptyClusterMembers.add(nodeId, ref) 31 | 32 | // then 33 | updatedClusterMembers shouldBe ClusterMembers(Map(nodeId -> ref)) 34 | } 35 | 36 | it should "give false result when asking for non-existent element with \"contains\" method" in { 37 | // given 38 | val emptyClusterMembers = ClusterMembers.empty 39 | 40 | // when 41 | val exists = emptyClusterMembers.contains(NodeId(1)) 42 | 43 | // then 44 | exists shouldBe false 45 | } 46 | 47 | it should "give positive result when asking for existent element with \"contains\" method" in { 48 | // given 49 | val nodeId = NodeId(100) 50 | val ref = StorageNodeActorRef(ActorRef.noSender) 51 | val clusterMembers = ClusterMembers.empty.add(nodeId, ref) 52 | 53 | // when 54 | val exists = clusterMembers.contains(NodeId(100)) 55 | 56 | // then 57 | exists shouldBe true 58 | } 59 | 60 | it should "give positive result when asking for non-existent element with \"notContains\" method" in { 61 | // given 62 | val emptyClusterMembers = ClusterMembers.empty 63 | 64 | // when 65 | val notExists = emptyClusterMembers.notContains(NodeId(1)) 66 | 67 | // then 68 | notExists shouldBe true 69 | } 70 | 71 | it should "return an element by key using \"get\" method" in { 72 | val nodeId = NodeId(100) 73 | val ref = StorageNodeActorRef(ActorRef.noSender) 74 | val clusterMembers = ClusterMembers.empty.add(nodeId, ref) 75 | 76 | clusterMembers.get(nodeId) shouldBe defined 77 | clusterMembers.get(NodeId(99)) should not be defined 78 | } 79 | 80 | it should "remove an element by its value" in { 81 | val nodeId = NodeId(100) 82 | val ref = StorageNodeActorRef(ActorRef.noSender) 83 | val clusterMembers = ClusterMembers.empty.add(nodeId, ref) 84 | 85 | clusterMembers.removeByRef(ref) shouldBe ClusterMembers.empty 86 | } 87 | 88 | it should "stringify" in { 89 | // given 90 | val nodeId = NodeId(100) 91 | val ref = StorageNodeActorRef(ActorRef.noSender) 92 | val emptyClusterMembers = ClusterMembers(Map(nodeId -> ref)) 93 | 94 | // when 95 | val stringified = emptyClusterMembers.toString 96 | 97 | // then 98 | stringified shouldBe Map(nodeId -> ref).toString() 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/DataSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.{Input, Output} 8 | import justin.db.Data 9 | import justin.db.consistenthashing.NodeId 10 | import justin.db.vectorclocks.{Counter, VectorClock} 11 | import org.scalatest.{FlatSpec, Matchers} 12 | 13 | class DataSerializerTest extends FlatSpec with Matchers { 14 | 15 | behavior of "Data Serializer" 16 | 17 | it should "serialize/deserialize correctly" in { 18 | // kryo init 19 | val kryo = new Kryo() 20 | kryo.register(classOf[justin.db.Data], DataSerializer) 21 | 22 | // object 23 | val vClock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))) 24 | val timestamp = System.currentTimeMillis() 25 | val serializedData = Data(id = UUID.randomUUID(), value = "some value", vClock, timestamp) 26 | 27 | // serialization 28 | val bos = new ByteArrayOutputStream() 29 | val output = new Output(bos) 30 | val _ = kryo.writeObject(output, serializedData) 31 | output.flush() 32 | 33 | // deserialization 34 | val bis = new ByteArrayInputStream(bos.toByteArray) 35 | val input = new Input(bis) 36 | val deserializedData = kryo.readObject(input, classOf[Data]) 37 | 38 | serializedData shouldBe deserializedData 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/RegisterNodeSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | 5 | import com.esotericsoftware.kryo.Kryo 6 | import com.esotericsoftware.kryo.io.{Input, Output} 7 | import justin.db.actors.protocol.RegisterNode 8 | import justin.db.consistenthashing.NodeId 9 | import org.scalatest.{FlatSpec, Matchers} 10 | 11 | class RegisterNodeSerializerTest extends FlatSpec with Matchers { 12 | 13 | behavior of "RegisterNode Serializer" 14 | 15 | it should "serialize/deserialize correctly" in { 16 | // kryo init 17 | val kryo = new Kryo() 18 | kryo.register(classOf[RegisterNode], RegisterNodeSerializer) 19 | 20 | // object 21 | val serializedData = RegisterNode(NodeId(1)) 22 | 23 | // serialization 24 | val bos = new ByteArrayOutputStream() 25 | val output = new Output(bos) 26 | val _ = kryo.writeObject(output, serializedData) 27 | output.flush() 28 | 29 | // deserialization 30 | val bis = new ByteArrayInputStream(bos.toByteArray) 31 | val input = new Input(bis) 32 | val deserializedData = kryo.readObject(input, classOf[RegisterNode]) 33 | 34 | serializedData shouldBe deserializedData 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/SerializerInitTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import com.esotericsoftware.kryo.Kryo 4 | import org.scalatest.{FlatSpec, Matchers} 5 | 6 | class SerializerInitTest extends FlatSpec with Matchers { 7 | 8 | behavior of "SerializerInit" 9 | 10 | it should "init Kryo serializer" in { 11 | val kryo = new Kryo() 12 | val serializerInit = new SerializerInit() 13 | serializerInit.customize(kryo) 14 | 15 | // cluster 16 | val classId_50 = 50 17 | kryo.getRegistration(classId_50).getId shouldBe 50 18 | kryo.getRegistration(classId_50).getSerializer shouldBe RegisterNodeSerializer 19 | kryo.getRegistration(classId_50).getType shouldBe classOf[justin.db.actors.protocol.RegisterNode] 20 | 21 | 22 | // write -- request 23 | val classId_60 = 60 24 | kryo.getRegistration(classId_60).getId shouldBe 60 25 | kryo.getRegistration(classId_60).getSerializer shouldBe StorageNodeWriteDataLocalSerializer 26 | kryo.getRegistration(classId_60).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeWriteDataLocal] 27 | 28 | // write -- responses 29 | val classId_70 = 70 30 | kryo.getRegistration(classId_70).getId shouldBe 70 31 | kryo.getRegistration(classId_70).getSerializer shouldBe StorageNodeWriteResponseSerializer 32 | kryo.getRegistration(classId_70).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeFailedWrite] 33 | 34 | val classId_71 = 71 35 | kryo.getRegistration(classId_71).getId shouldBe 71 36 | kryo.getRegistration(classId_71).getSerializer shouldBe StorageNodeWriteResponseSerializer 37 | kryo.getRegistration(classId_71).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeSuccessfulWrite] 38 | 39 | val classId_72 = 72 40 | kryo.getRegistration(classId_72).getId shouldBe 72 41 | kryo.getRegistration(classId_72).getSerializer shouldBe StorageNodeWriteResponseSerializer 42 | kryo.getRegistration(classId_72).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeConflictedWrite] 43 | 44 | // read - request 45 | val classId_80 = 80 46 | kryo.getRegistration(classId_80).getId shouldBe 80 47 | kryo.getRegistration(classId_80).getSerializer shouldBe StorageNodeLocalReadSerializer 48 | kryo.getRegistration(classId_80).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeLocalRead] 49 | 50 | // read - responses 51 | val classId_90 = 90 52 | kryo.getRegistration(classId_90).getId shouldBe 90 53 | kryo.getRegistration(classId_90).getSerializer shouldBe StorageNodeReadResponseSerializer 54 | kryo.getRegistration(classId_90).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeFoundRead] 55 | 56 | val classId_91 = 91 57 | kryo.getRegistration(classId_91).getId shouldBe 91 58 | kryo.getRegistration(classId_91).getSerializer shouldBe StorageNodeReadResponseSerializer 59 | kryo.getRegistration(classId_91).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeConflictedRead] 60 | 61 | val classId_92 = 92 62 | kryo.getRegistration(classId_92).getId shouldBe 92 63 | kryo.getRegistration(classId_92).getSerializer shouldBe StorageNodeReadResponseSerializer 64 | kryo.getRegistration(classId_92).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeNotFoundRead] 65 | 66 | val classId_93 = 93 67 | kryo.getRegistration(classId_93).getId shouldBe 93 68 | kryo.getRegistration(classId_93).getSerializer shouldBe StorageNodeReadResponseSerializer 69 | kryo.getRegistration(classId_93).getType shouldBe classOf[justin.db.actors.protocol.StorageNodeFailedRead] 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/StorageNodeLocalReadSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.{Input, Output} 8 | import justin.db.actors.protocol.StorageNodeLocalRead 9 | import org.scalatest.{FlatSpec, Matchers} 10 | 11 | class StorageNodeLocalReadSerializerTest extends FlatSpec with Matchers { 12 | 13 | behavior of "StorageNodeLocalReader Serializer" 14 | 15 | it should "serialize/deserialize correctly" in { 16 | // kryo init 17 | val kryo = new Kryo() 18 | kryo.register(classOf[StorageNodeLocalRead], StorageNodeLocalReadSerializer) 19 | 20 | // object 21 | val serializedData = StorageNodeLocalRead(UUID.randomUUID()) 22 | 23 | // serialization 24 | val bos = new ByteArrayOutputStream() 25 | val output = new Output(bos) 26 | val _ = kryo.writeObject(output, serializedData) 27 | output.flush() 28 | 29 | // deserialization 30 | val bis = new ByteArrayInputStream(bos.toByteArray) 31 | val input = new Input(bis) 32 | val deserializedData = kryo.readObject(input, classOf[StorageNodeLocalRead]) 33 | 34 | serializedData shouldBe deserializedData 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/StorageNodeReadResponseSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.{Input, Output} 8 | import justin.db.Data 9 | import justin.db.actors.protocol.{StorageNodeConflictedRead, StorageNodeFailedRead, StorageNodeFoundRead, StorageNodeNotFoundRead} 10 | import justin.db.consistenthashing.NodeId 11 | import justin.db.vectorclocks.{Counter, VectorClock} 12 | import org.scalatest.{FlatSpec, Matchers} 13 | 14 | class StorageNodeReadResponseSerializerTest extends FlatSpec with Matchers { 15 | 16 | behavior of "StorageNodeReadResponse Serializer" 17 | 18 | it should "serialize/deserialize StorageNodeFoundRead" in { 19 | // kryo init 20 | val kryo = new Kryo() 21 | kryo.register(classOf[StorageNodeFoundRead], StorageNodeReadResponseSerializer) 22 | 23 | // object 24 | val data = Data( 25 | id = UUID.randomUUID(), 26 | value = "some value", 27 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))), 28 | timestamp = System.currentTimeMillis() 29 | ) 30 | val serializedData = StorageNodeFoundRead(data) 31 | 32 | // serialization 33 | val bos = new ByteArrayOutputStream() 34 | val output = new Output(bos) 35 | val _ = kryo.writeObject(output, serializedData) 36 | output.flush() 37 | 38 | // deserialization 39 | val bis = new ByteArrayInputStream(bos.toByteArray) 40 | val input = new Input(bis) 41 | val deserializedData = kryo.readObject(input, classOf[StorageNodeFoundRead]) 42 | 43 | serializedData shouldBe deserializedData 44 | } 45 | 46 | it should "serialize/deserialize StorageNodeNotFoundRead" in { 47 | // kryo init 48 | val kryo = new Kryo() 49 | kryo.register(classOf[StorageNodeNotFoundRead], StorageNodeReadResponseSerializer) 50 | 51 | // object 52 | val serializedData = StorageNodeNotFoundRead(UUID.randomUUID()) 53 | 54 | // serialization 55 | val bos = new ByteArrayOutputStream() 56 | val output = new Output(bos) 57 | val _ = kryo.writeObject(output, serializedData) 58 | output.flush() 59 | 60 | // deserialization 61 | val bis = new ByteArrayInputStream(bos.toByteArray) 62 | val input = new Input(bis) 63 | val deserializedData = kryo.readObject(input, classOf[StorageNodeNotFoundRead]) 64 | 65 | serializedData shouldBe deserializedData 66 | } 67 | 68 | it should "serialize/deserialize StorageNodeFailedRead" in { 69 | // kryo init 70 | val kryo = new Kryo() 71 | kryo.register(classOf[StorageNodeFailedRead], StorageNodeReadResponseSerializer) 72 | 73 | // object 74 | val serializedData = StorageNodeFailedRead(UUID.randomUUID()) 75 | 76 | // serialization 77 | val bos = new ByteArrayOutputStream() 78 | val output = new Output(bos) 79 | val _ = kryo.writeObject(output, serializedData) 80 | output.flush() 81 | 82 | // deserialization 83 | val bis = new ByteArrayInputStream(bos.toByteArray) 84 | val input = new Input(bis) 85 | val deserializedData = kryo.readObject(input, classOf[StorageNodeFailedRead]) 86 | 87 | serializedData shouldBe deserializedData 88 | } 89 | 90 | it should "serialize/deserialize StorageNodeConflictedRead" in { 91 | // kryo init 92 | val kryo = new Kryo() 93 | kryo.register(classOf[StorageNodeConflictedRead], StorageNodeReadResponseSerializer) 94 | 95 | // object 96 | val data1 = Data( 97 | id = UUID.randomUUID(), 98 | value = "some value 1", 99 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))), 100 | timestamp = System.currentTimeMillis() 101 | ) 102 | val data2 = Data( 103 | id = UUID.randomUUID(), 104 | value = "some value 2", 105 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(1))), 106 | timestamp = System.currentTimeMillis() 107 | ) 108 | val serializedData = StorageNodeConflictedRead(List(data1, data2)) 109 | 110 | // serialization 111 | val bos = new ByteArrayOutputStream() 112 | val output = new Output(bos) 113 | val _ = kryo.writeObject(output, serializedData) 114 | output.flush() 115 | 116 | // deserialization 117 | val bis = new ByteArrayInputStream(bos.toByteArray) 118 | val input = new Input(bis) 119 | val deserializedData = kryo.readObject(input, classOf[StorageNodeConflictedRead]) 120 | 121 | serializedData shouldBe deserializedData 122 | } 123 | } 124 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/StorageNodeWriteDataLocalSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.{Input, Output} 8 | import justin.db.Data 9 | import justin.db.actors.protocol.StorageNodeWriteDataLocal 10 | import justin.db.consistenthashing.NodeId 11 | import justin.db.vectorclocks.{Counter, VectorClock} 12 | import org.scalatest.{FlatSpec, Matchers} 13 | 14 | class StorageNodeWriteDataLocalSerializerTest extends FlatSpec with Matchers { 15 | 16 | behavior of "StorageNodeWriteDataLocal Serializer" 17 | 18 | it should "serialize/deserialize StorageNodeWriteDataLocal" in { 19 | // kryo init 20 | val kryo = new Kryo() 21 | kryo.register(classOf[StorageNodeWriteDataLocal], StorageNodeWriteDataLocalSerializer) 22 | 23 | // object 24 | val data = Data( 25 | id = UUID.randomUUID(), 26 | value = "some value", 27 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))), 28 | timestamp = System.currentTimeMillis() 29 | ) 30 | val serializedData = StorageNodeWriteDataLocal(data) 31 | 32 | // serialization 33 | val bos = new ByteArrayOutputStream() 34 | val output = new Output(bos) 35 | val _ = kryo.writeObject(output, serializedData) 36 | output.flush() 37 | 38 | // deserialization 39 | val bis = new ByteArrayInputStream(bos.toByteArray) 40 | val input = new Input(bis) 41 | val deserializedData = kryo.readObject(input, classOf[StorageNodeWriteDataLocal]) 42 | 43 | serializedData shouldBe deserializedData 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/kryo/StorageNodeWriteResponseSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.kryo 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.{Input, Output} 8 | import justin.db.Data 9 | import justin.db.actors.protocol.{StorageNodeConflictedWrite, StorageNodeFailedWrite, StorageNodeSuccessfulWrite} 10 | import justin.db.consistenthashing.NodeId 11 | import justin.db.vectorclocks.{Counter, VectorClock} 12 | import org.scalatest.{FlatSpec, Matchers} 13 | 14 | class StorageNodeWriteResponseSerializerTest extends FlatSpec with Matchers { 15 | 16 | behavior of "StorageNodeWriteResponseSerializer Serializer" 17 | 18 | it should "serialize/deserialize StorageNodeSuccessfulWrite" in { 19 | // kryo init 20 | val kryo = new Kryo() 21 | kryo.register(classOf[StorageNodeSuccessfulWrite], StorageNodeWriteResponseSerializer) 22 | 23 | // object 24 | val serializedData = StorageNodeSuccessfulWrite(UUID.randomUUID()) 25 | 26 | // serialization 27 | val bos = new ByteArrayOutputStream() 28 | val output = new Output(bos) 29 | val _ = kryo.writeObject(output, serializedData) 30 | output.flush() 31 | 32 | // deserialization 33 | val bis = new ByteArrayInputStream(bos.toByteArray) 34 | val input = new Input(bis) 35 | val deserializedData = kryo.readObject(input, classOf[StorageNodeSuccessfulWrite]) 36 | 37 | serializedData shouldBe deserializedData 38 | } 39 | 40 | it should "serialize/deserialize StorageNodeFailedWrite" in { 41 | // kryo init 42 | val kryo = new Kryo() 43 | kryo.register(classOf[StorageNodeFailedWrite], StorageNodeWriteResponseSerializer) 44 | 45 | // object 46 | val serializedData = StorageNodeFailedWrite(UUID.randomUUID()) 47 | 48 | // serialization 49 | val bos = new ByteArrayOutputStream() 50 | val output = new Output(bos) 51 | val _ = kryo.writeObject(output, serializedData) 52 | output.flush() 53 | 54 | // deserialization 55 | val bis = new ByteArrayInputStream(bos.toByteArray) 56 | val input = new Input(bis) 57 | val deserializedData = kryo.readObject(input, classOf[StorageNodeFailedWrite]) 58 | 59 | serializedData shouldBe deserializedData 60 | } 61 | 62 | it should "serialize/deserialize StorageNodeConflictedWrite" in { 63 | // kryo init 64 | val kryo = new Kryo() 65 | kryo.register(classOf[StorageNodeConflictedWrite], StorageNodeWriteResponseSerializer) 66 | 67 | // object 68 | val oldData = Data( 69 | id = UUID.randomUUID(), 70 | value = "some value 1", 71 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(3))), 72 | timestamp = System.currentTimeMillis() 73 | ) 74 | val newData = Data( 75 | id = UUID.randomUUID(), 76 | value = "some value 2", 77 | vclock = VectorClock[NodeId](Map(NodeId(1) -> Counter(1))), 78 | timestamp = System.currentTimeMillis() 79 | ) 80 | val serializedData = StorageNodeConflictedWrite(oldData, newData) 81 | 82 | // serialization 83 | val bos = new ByteArrayOutputStream() 84 | val output = new Output(bos) 85 | val _ = kryo.writeObject(output, serializedData) 86 | output.flush() 87 | 88 | // deserialization 89 | val bis = new ByteArrayInputStream(bos.toByteArray) 90 | val input = new Input(bis) 91 | val deserializedData = kryo.readObject(input, classOf[StorageNodeConflictedWrite]) 92 | 93 | serializedData shouldBe deserializedData 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/merkletrees/DigestTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.merkletrees 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class DigestTest extends FlatSpec with Matchers { 6 | 7 | behavior of "Digest" 8 | 9 | it should "define addition operator" in { 10 | val digest1 = Digest(Array[Byte](1,2,3)) 11 | val digest2 = Digest(Array[Byte](4,5,6)) 12 | 13 | val result = digest1 + digest2 14 | 15 | result.hash.deep shouldBe Array[Byte](1,2,3,4,5,6) 16 | } 17 | 18 | it should "define equality operator" in { 19 | val digest1 = Digest(Array[Byte](1,2,3)) 20 | val digest2 = Digest(Array[Byte](1,2,3)) 21 | 22 | val equal = digest1 == digest2 23 | 24 | equal shouldBe true 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/merkletrees/MD5DigestTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.merkletrees 2 | 3 | import java.security.MessageDigest 4 | 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | class MD5DigestTest extends FlatSpec with Matchers { 8 | 9 | it should "digest block of data with MD5 algorithm" in { 10 | // given 11 | val block: Block = Array[Byte](10, 11, 18, 127, 0, -128) 12 | 13 | // when 14 | val digest = MerkleDigest.MD5.digest(block) 15 | 16 | // then 17 | digest.hash.deep shouldBe MessageDigest.getInstance("MD5").digest(block) 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/merkletrees/MerkleTreeTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.merkletrees 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class MerkleTreeTest extends FlatSpec with Matchers { 6 | 7 | behavior of "Merkle Tree" 8 | 9 | it should "have the same top hash" in { 10 | val blocks: Seq[Block] = Seq( 11 | Array[Byte](1,2,3), 12 | Array[Byte](4,5,6), 13 | Array[Byte](7,8,9), 14 | Array[Byte](10,11,12) 15 | ) 16 | val blocks2: Seq[Block] = Seq( 17 | Array[Byte](1,2,3), 18 | Array[Byte](4,5,6), 19 | Array[Byte](7,8,9), 20 | Array[Byte](10,11,12) 21 | ) 22 | 23 | val digest1 = MerkleTree.unapply(blocks)(MerkleDigest.CRC32).get.digest 24 | val digest2 = MerkleTree.unapply(blocks2)(MerkleDigest.CRC32).get.digest 25 | 26 | digest1.hash.deep shouldBe digest2.hash.deep 27 | } 28 | 29 | it should "have a different top hash" in { 30 | val blocks: Seq[Block] = Seq( 31 | Array[Byte](1,2,3), 32 | Array[Byte](4,5,6), 33 | Array[Byte](7,8,9), 34 | Array[Byte](10,11,12) 35 | ) 36 | val blocks2: Seq[Block] = Seq( 37 | Array[Byte](1,2,3), 38 | Array[Byte](4,5,6), 39 | Array[Byte](9,8,7), 40 | Array[Byte](12,11,10) 41 | ) 42 | 43 | val digest1 = MerkleTree.unapply(blocks)(MerkleDigest.CRC32).get.digest 44 | val digest2 = MerkleTree.unapply(blocks2)(MerkleDigest.CRC32).get.digest 45 | 46 | digest1.hash.deep should not be digest2.hash.deep 47 | } 48 | 49 | it should "use an all-zeros value to complete the pair" in { 50 | val oddBlocks: Seq[Block] = Seq( 51 | Array[Byte](1,2,3), 52 | Array[Byte](4,5,6), 53 | Array[Byte](7,8,9) 54 | ) 55 | 56 | val sameBlocksWithZeroed: Seq[Block] = Seq( 57 | Array[Byte](1,2,3), 58 | Array[Byte](4,5,6), 59 | Array[Byte](7,8,9), 60 | Array[Byte](0) 61 | ) 62 | 63 | val digest1 = MerkleTree.unapply(oddBlocks)(MerkleDigest.CRC32).get.digest 64 | val digest2 = MerkleTree.unapply(sameBlocksWithZeroed)(MerkleDigest.CRC32).get.digest 65 | 66 | digest1.hash.deep shouldBe digest2.hash.deep 67 | } 68 | 69 | it should "create missed zeroed byte blocks if initial blocks size is NOT power of two" in { 70 | val blocks = Array( 71 | Array[Byte](1,2,3), 72 | Array[Byte](4,5,6), 73 | Array[Byte](7,8,9), 74 | Array[Byte](7,8,9), 75 | Array[Byte](7,8,9) 76 | ) 77 | 78 | val expected = Array( 79 | Array[Byte](0), 80 | Array[Byte](0), 81 | Array[Byte](0) 82 | ) 83 | 84 | val zeroed = MerkleTree.zeroed(blocks) 85 | 86 | zeroed.deep shouldBe expected.deep 87 | } 88 | 89 | it should "NOT create missed zeroed bytes blocks if initial blocks size is power of two" in { 90 | val blocks = Array( 91 | Array[Byte](1,2,3), 92 | Array[Byte](4,5,6), 93 | Array[Byte](7,8,9), 94 | Array[Byte](10,11,12) 95 | ) 96 | 97 | val zeroed = MerkleTree.zeroed(blocks) 98 | 99 | zeroed.deep shouldBe empty 100 | } 101 | 102 | it should "start indexing of Merkle Tree nodes from 0" in { 103 | val blocks = Array( 104 | Array[Byte](1,2,3), 105 | Array[Byte](4,5,6), 106 | Array[Byte](7,8,9), 107 | Array[Byte](10,11,12) 108 | ) 109 | 110 | val firstNodeId = MerkleTree.unapply(blocks)(MerkleDigest.CRC32).get.nodeId.id 111 | 112 | firstNodeId shouldBe 0 113 | } 114 | 115 | it should "index Merkle Tree nodes in binary fashion" in { 116 | val blocks = Array( 117 | Array[Byte](1,2,3), 118 | Array[Byte](4,5,6), 119 | Array[Byte](7,8,9), 120 | Array[Byte](10,11,12) 121 | ) 122 | 123 | val tree = MerkleTree.unapply(blocks)(MerkleDigest.CRC32).get 124 | 125 | /** we should expect such tree indexing 126 | * 0 127 | * 1 4 128 | * 2 3 5 6 129 | */ 130 | tree.nodeId.id shouldBe 0 131 | tree.asInstanceOf[MerkleHashNode].left.nodeId.id shouldBe 1 132 | tree.asInstanceOf[MerkleHashNode].left.asInstanceOf[MerkleHashNode].left.nodeId.id shouldBe 2 133 | tree.asInstanceOf[MerkleHashNode].left.asInstanceOf[MerkleHashNode].right.nodeId.id shouldBe 3 134 | tree.asInstanceOf[MerkleHashNode].right.nodeId.id shouldBe 4 135 | tree.asInstanceOf[MerkleHashNode].right.asInstanceOf[MerkleHashNode].left.nodeId.id shouldBe 5 136 | tree.asInstanceOf[MerkleHashNode].right.asInstanceOf[MerkleHashNode].right.nodeId.id shouldBe 6 137 | } 138 | 139 | it should "find node with its id" in { 140 | val blocks = Array( 141 | Array[Byte](1,2,3), 142 | Array[Byte](4,5,6), 143 | Array[Byte](7,8,9), 144 | Array[Byte](10,11,12) 145 | ) 146 | val tree = MerkleTree.unapply(blocks)(MerkleDigest.CRC32).get 147 | 148 | MerkleTree.findNode(MerkleNodeId(0), tree).get.nodeId shouldBe MerkleNodeId(0) 149 | MerkleTree.findNode(MerkleNodeId(1), tree).get.nodeId shouldBe MerkleNodeId(1) 150 | MerkleTree.findNode(MerkleNodeId(2), tree).get.nodeId shouldBe MerkleNodeId(2) 151 | MerkleTree.findNode(MerkleNodeId(3), tree).get.nodeId shouldBe MerkleNodeId(3) 152 | MerkleTree.findNode(MerkleNodeId(4), tree).get.nodeId shouldBe MerkleNodeId(4) 153 | MerkleTree.findNode(MerkleNodeId(5), tree).get.nodeId shouldBe MerkleNodeId(5) 154 | MerkleTree.findNode(MerkleNodeId(6), tree).get.nodeId shouldBe MerkleNodeId(6) 155 | 156 | MerkleTree.findNode(MerkleNodeId(-1), tree) should not be defined 157 | MerkleTree.findNode(MerkleNodeId(7), tree) should not be defined 158 | } 159 | } 160 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/IsPrimaryOrReplicaTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.consistenthashing.{NodeId, Ring} 6 | import justin.db.storage.PluggableStorageProtocol.DataOriginality 7 | import org.scalatest.{FlatSpec, Matchers} 8 | 9 | class IsPrimaryOrReplicaTest extends FlatSpec with Matchers { 10 | 11 | behavior of "Data Originality Resolver" 12 | 13 | it should "reason exemplary data's id as a replica" in { 14 | // given 15 | val nodeId = NodeId(0) 16 | val ring = Ring.apply(nodesSize = 3, partitionsSize = 21) 17 | val resolver = new IsPrimaryOrReplica(nodeId, ring) 18 | val id = UUID.fromString("179d6eb0-681d-4277-9caf-3d6d60e9faf9") 19 | 20 | // when 21 | val originality = resolver.apply(id) 22 | 23 | // then 24 | originality shouldBe a[DataOriginality.Replica] 25 | } 26 | 27 | it should "reason exemplary data's id as a primary" in { 28 | // given 29 | val nodeId = NodeId(0) 30 | val ring = Ring.apply(nodesSize = 3, partitionsSize = 21) 31 | val resolver = new IsPrimaryOrReplica(nodeId, ring) 32 | val id = UUID.fromString("16ec44cd-5b4e-4b38-a647-206c1dc11b50") 33 | 34 | // when 35 | val originality = resolver.apply(id) 36 | 37 | // then 38 | originality shouldBe a[DataOriginality.Primary] 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/PreferenceListSpecification.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import justin.db.consistenthashing.Ring 4 | import org.scalacheck.Prop._ 5 | import org.scalacheck.{Gen, Properties} 6 | 7 | class PreferenceListSpecification extends Properties("PreferenceList") { 8 | 9 | property("head of preference-list is initial partitionId") = { 10 | val ring = Ring.apply(nodesSize = 5, partitionsSize = 64) 11 | val n = N(3) 12 | val partitionIdGen = Gen.choose(0, ring.size-1) 13 | 14 | forAll(partitionIdGen) { basePartitionId: Int => 15 | PreferenceList(basePartitionId, n, ring).right.get.primaryNodeId == ring.getNodeId(basePartitionId).get 16 | } 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/PreferenceListTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import justin.db.consistenthashing.{NodeId, Ring} 4 | import org.scalatest.{FlatSpec, Matchers} 5 | 6 | class PreferenceListTest extends FlatSpec with Matchers { 7 | 8 | behavior of "Preference List" 9 | 10 | it should "has size of defined nr of replicas" in { 11 | // given 12 | val n = N(3) // nr of replicas 13 | val ring = Ring(nodesSize = 5, partitionsSize = 64) 14 | val basePartitionId = 1 15 | 16 | // when 17 | val preferenceList = PreferenceList(basePartitionId, n, ring).right.get 18 | 19 | // then 20 | preferenceList.size shouldBe 3 21 | } 22 | 23 | it should "has defined first node in the list to be the one taken from Ring with initial partitionId" in { 24 | // given 25 | val n = N(3) // nr of replicas 26 | val ring = Ring(nodesSize = 5, partitionsSize = 64) 27 | val initialPartitionId = 1 28 | 29 | // when 30 | val coordinator = PreferenceList.apply(initialPartitionId, n, ring).right.get.primaryNodeId 31 | 32 | // then 33 | coordinator shouldBe ring.getNodeId(initialPartitionId).get 34 | } 35 | 36 | it should "has at least one member (coordinator) for none replicas" in { 37 | // given 38 | val n = N(0) // nr of replicas 39 | val ring = Ring(nodesSize = 5, partitionsSize = 64) 40 | val initialPartitionId = 1 41 | 42 | // when 43 | val preferenceList = PreferenceList.apply(initialPartitionId, n, ring).right.get 44 | 45 | // then 46 | preferenceList.size shouldBe 1 47 | preferenceList.primaryNodeId shouldBe ring.getNodeId(initialPartitionId).get 48 | } 49 | 50 | it should "check that selected nodes ids are continuous" in { 51 | // given 52 | val n = N(3) // nr of replicas 53 | val ring = Ring(nodesSize = 5, partitionsSize = 64) 54 | val initialPartitionId = 1 55 | 56 | // when 57 | val preferenceList = PreferenceList.apply(initialPartitionId, n, ring).right.get 58 | 59 | // then 60 | preferenceList shouldBe PreferenceList(NodeId(1), List(NodeId(2), NodeId(3))) 61 | } 62 | 63 | it should "fail to build PreferenceList if coordinator nodeId couldn't be found" in { 64 | // given 65 | val n = N(3) // nr of replicas 66 | val ring = Ring(nodesSize = 5, partitionsSize = 64) 67 | val notExistedPartitionId = -1 68 | 69 | // when 70 | val preferenceList = PreferenceList.apply(notExistedPartitionId, n, ring) 71 | 72 | // then 73 | preferenceList shouldBe Left(PreferenceList.LackOfCoordinator) 74 | } 75 | 76 | it should "fail to build PreferenceList if it has NOT expected size" in { 77 | // given 78 | val n = N(3) // nr of replicas 79 | val ring = Ring(nodesSize = 2, partitionsSize = 64) 80 | val initialPartitionId = 1 81 | 82 | // when 83 | val preferenceList = PreferenceList.apply(initialPartitionId, n, ring) 84 | 85 | // then 86 | preferenceList shouldBe Left(PreferenceList.NotSufficientSize(PreferenceList(NodeId(1), List(NodeId(0))))) 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/ResolveNodeAddressesTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica 2 | 3 | import akka.actor.ActorRef 4 | import justin.db.actors.StorageNodeActorRef 5 | import justin.db.cluster.ClusterMembers 6 | import justin.db.consistenthashing.NodeId 7 | import org.scalatest.{FlatSpec, Matchers} 8 | 9 | class ResolveNodeAddressesTest extends FlatSpec with Matchers { 10 | 11 | behavior of "Resolver of Node Addresses" 12 | 13 | it should "mark \"local\" value as \"true\" when node is placed in the preference list" in { 14 | // given 15 | val nodeId = NodeId(1) 16 | val preferenceList = PreferenceList(nodeId, Nil) 17 | val clusterMembers = ClusterMembers.empty 18 | 19 | // when 20 | val resolved = ResolveNodeAddresses(nodeId, preferenceList, clusterMembers) 21 | 22 | // then 23 | resolved.local shouldBe true 24 | } 25 | 26 | it should "mark \"local\" value as \"false\" when node is NOT placed in the preference list" in { 27 | // given 28 | val nodeId = NodeId(1) 29 | val preferenceList = PreferenceList(NodeId(2), Nil) 30 | val clusterMembers = ClusterMembers.empty 31 | 32 | // when 33 | val resolved = ResolveNodeAddresses(nodeId, preferenceList, clusterMembers) 34 | 35 | // then 36 | resolved.local shouldBe false 37 | } 38 | 39 | it should "not include node in remotes" in { 40 | // given 41 | val nodeId = NodeId(1) 42 | val preferenceList = PreferenceList(NodeId(1), List(NodeId(2), NodeId(3))) 43 | val clusterMembers = ClusterMembers(Map(NodeId(2) -> StorageNodeActorRef(ActorRef.noSender), NodeId(3) -> StorageNodeActorRef(ActorRef.noSender))) 44 | 45 | // when 46 | val resolved = ResolveNodeAddresses(nodeId, preferenceList, clusterMembers) 47 | 48 | // then 49 | resolved shouldBe ResolvedNodeAddresses(local = true, remotes = List(StorageNodeActorRef(ActorRef.noSender), StorageNodeActorRef(ActorRef.noSender))) 50 | } 51 | 52 | it should "flatten not existed nodes from preference list in cluster members" in { 53 | // given 54 | val nodeId = NodeId(1) 55 | val preferenceList = PreferenceList(NodeId(1), List(NodeId(2), NodeId(3))) 56 | val clusterMembers = ClusterMembers(Map(NodeId(2) -> StorageNodeActorRef(ActorRef.noSender))) 57 | 58 | // when 59 | val resolved = ResolveNodeAddresses(nodeId, preferenceList, clusterMembers) 60 | 61 | // then 62 | resolved shouldBe ResolvedNodeAddresses(local = true, remotes = List(StorageNodeActorRef(ActorRef.noSender))) 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/read/ReplicaLocalReaderTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.Data 6 | import justin.db.actors.protocol.{StorageNodeFailedRead, StorageNodeFoundRead, StorageNodeNotFoundRead} 7 | import justin.db.consistenthashing.NodeId 8 | import justin.db.storage.GetStorageProtocol 9 | import justin.db.storage.PluggableStorageProtocol.{DataOriginality, StorageGetData} 10 | import justin.db.vectorclocks.VectorClock 11 | import org.scalatest.concurrent.ScalaFutures 12 | import org.scalatest.{FlatSpec, Matchers} 13 | 14 | import scala.concurrent.ExecutionContext.Implicits.global 15 | import scala.concurrent.duration._ 16 | import scala.concurrent.Future 17 | 18 | class ReplicaLocalReaderTest extends FlatSpec with Matchers with ScalaFutures { 19 | 20 | behavior of "Replica Local Reader" 21 | 22 | override implicit def patienceConfig: PatienceConfig = PatienceConfig(10.seconds, 50.millis) 23 | 24 | it should "found data for existing key" in { 25 | // given 26 | val id = UUID.randomUUID() 27 | val data = Data(id, "value", VectorClock[NodeId]().increase(NodeId(1))) 28 | val service = new ReplicaLocalReader(new GetStorageProtocol { 29 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = { 30 | Future.successful(StorageGetData.Single(data)) 31 | } 32 | }) 33 | 34 | // when 35 | val result = service.apply(id, null) 36 | 37 | // then 38 | whenReady(result) { _ shouldBe StorageNodeFoundRead(data) } 39 | } 40 | 41 | it should "not found data for non-existing key" in { 42 | // given 43 | val id = UUID.randomUUID() 44 | val service = new ReplicaLocalReader(new GetStorageProtocol { 45 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = { 46 | Future.successful(StorageGetData.None) 47 | } 48 | }) 49 | 50 | // when 51 | val result = service.apply(id, null) 52 | 53 | // then 54 | whenReady(result) { _ shouldBe StorageNodeNotFoundRead(id) } 55 | } 56 | 57 | it should "recover failure reading" in { 58 | // given 59 | val id = UUID.randomUUID() 60 | val service = new ReplicaLocalReader(new GetStorageProtocol { 61 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.failed(new Exception) 62 | }) 63 | 64 | // when 65 | val result = service.apply(id, null) 66 | 67 | // then 68 | whenReady(result) { _ shouldBe StorageNodeFailedRead(id) } 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/read/ReplicaReadAgreementTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.Data 6 | import justin.db.actors.protocol.{StorageNodeFailedRead, StorageNodeFoundRead, StorageNodeNotFoundRead} 7 | import justin.db.consistenthashing.NodeId 8 | import justin.db.replica.R 9 | import justin.db.vectorclocks.VectorClock 10 | import justin.db.vectorclocks.VectorClockOps 11 | import org.scalatest.{FlatSpec, Matchers} 12 | 13 | class ReplicaReadAgreementTest extends FlatSpec with Matchers { 14 | 15 | behavior of "Reach Consensus of Replicated Reads" 16 | 17 | it should "agreed on \"AllNotFound\" when all searched data couldn't be found" in { 18 | // given 19 | val r = 1 20 | val searchedData = List(StorageNodeNotFoundRead(UUID.randomUUID()), StorageNodeNotFoundRead(UUID.randomUUID())) 21 | 22 | // when 23 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 24 | 25 | // then 26 | madeConsensus shouldBe ReadAgreement.AllNotFound 27 | } 28 | 29 | it should "agreed on \"AllFailed\" when all operations during search failed" in { 30 | // given 31 | val r = 1 32 | val searchedData = List(StorageNodeFailedRead(UUID.randomUUID()), StorageNodeFailedRead(UUID.randomUUID())) 33 | 34 | // when 35 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 36 | 37 | // then 38 | madeConsensus shouldBe ReadAgreement.AllFailed 39 | } 40 | 41 | it should "agreed on \"NotEnoughFound\" when number of found replica is smaller that what client expects" in { 42 | // given 43 | val r = 2 44 | val searchedData = List(StorageNodeNotFoundRead(UUID.randomUUID()), StorageNodeFailedRead(UUID.randomUUID()), StorageNodeFoundRead(Data(UUID.randomUUID(), "value"))) 45 | 46 | // when 47 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 48 | 49 | // then 50 | madeConsensus shouldBe ReadAgreement.NotEnoughFound 51 | } 52 | 53 | it should "agreed on \"Consequent\" scenario when client expectation is achieved and consequent data could be computed" in { 54 | // given 55 | val r = 3 56 | val searchedData = List( 57 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-1", "1:1")), 58 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-2", "1:2")), 59 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-3", "1:3")) 60 | ) 61 | 62 | // when 63 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 64 | 65 | // then 66 | madeConsensus shouldBe ReadAgreement.Consequent(searchedData.last.data) 67 | } 68 | 69 | it should "agreed on \"Conflict\" scenario when client expectation is achieved but consequent data could NOT be computed" in { 70 | // given 71 | val r = 3 72 | val searchedData = List( 73 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-1", "1:1")), 74 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-2", "1:2")), 75 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-3", "2:1")) 76 | ) 77 | 78 | // when 79 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 80 | 81 | // then 82 | madeConsensus shouldBe ReadAgreement.Conflicts(searchedData.map(_.data)) 83 | } 84 | 85 | it should "agreed on \"Found\" when exactly once data is found and client expects only one replica" in { 86 | // given 87 | val r = 1 88 | val foundData = StorageNodeFoundRead(Data(UUID.randomUUID(), "value")) 89 | val searchedData = List(StorageNodeNotFoundRead(UUID.randomUUID()), StorageNodeFailedRead(UUID.randomUUID()), foundData) 90 | 91 | // when 92 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 93 | 94 | // then 95 | madeConsensus shouldBe ReadAgreement.Found(foundData.data) 96 | } 97 | 98 | it should "agreed on \"Found\" scenario when client expectation is achieved and all replicas agreed on same value" in { 99 | // given 100 | val r = 3 101 | val searchedData = List( 102 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-1", "2:1")), 103 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-1", "2:1")), 104 | StorageNodeFoundRead(Data(UUID.randomUUID(), "value-1", "2:1")) 105 | ) 106 | 107 | // when 108 | val madeConsensus = new ReplicaReadAgreement().reach(R(r))(searchedData) 109 | 110 | // then 111 | madeConsensus shouldBe ReadAgreement.Found(searchedData.head.data) 112 | } 113 | 114 | implicit def nodeIdAsId(s: String): VectorClock[NodeId] = s.toVectorClock[NodeId](s => NodeId(s.toInt)) 115 | } -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/read/ReplicaRemoteReaderTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.read 2 | 3 | import java.util.UUID 4 | 5 | import akka.actor.{Actor, ActorSystem} 6 | import akka.testkit.{TestActorRef, TestKit} 7 | import justin.db.Data 8 | import justin.db.actors.StorageNodeActorRef 9 | import justin.db.actors.protocol._ 10 | import org.scalatest.concurrent.ScalaFutures 11 | import org.scalatest.{FlatSpecLike, Matchers} 12 | 13 | import scala.concurrent.duration._ 14 | 15 | class ReplicaRemoteReaderTest extends TestKit(ActorSystem("test-system")) 16 | with FlatSpecLike 17 | with Matchers 18 | with ScalaFutures { 19 | 20 | behavior of "Replica Remote Reader" 21 | 22 | override implicit def patienceConfig: PatienceConfig = PatienceConfig(10.seconds, 50.millis) 23 | 24 | it should "get info back that one of the value could be found and second one is obsolete" in { 25 | // given 26 | val service = new ReplicaRemoteReader()(system.dispatcher) 27 | val id = UUID.randomUUID() 28 | val foundData = Data(id, "value") 29 | val notFoundId = UUID.randomUUID() 30 | val storageNotFoundActorRef = testActorRef(msgBack = StorageNodeNotFoundRead(notFoundId)) 31 | val storageFoundActorRef = testActorRef(msgBack = StorageNodeFoundRead(foundData)) 32 | val storageNodeRefs = List(storageNotFoundActorRef, storageFoundActorRef).map(StorageNodeActorRef) 33 | 34 | // when 35 | val readingResult = service.apply(storageNodeRefs, id) 36 | 37 | // then 38 | whenReady(readingResult) { _ shouldBe List(StorageNodeNotFoundRead(notFoundId), StorageNodeFoundRead(foundData)) } 39 | } 40 | 41 | it should "recover failed behavior of actor" in { 42 | // given 43 | val service = new ReplicaRemoteReader()(system.dispatcher) 44 | val id = UUID.randomUUID() 45 | val storageActorRef = testActorRef(new Exception) 46 | val storageNodeRefs = List(StorageNodeActorRef(storageActorRef)) 47 | 48 | // when 49 | val readingResult = service.apply(storageNodeRefs, id) 50 | 51 | // then 52 | whenReady(readingResult) { _ shouldBe List(StorageNodeFailedRead(id)) } 53 | } 54 | 55 | private def testActorRef(msgBack: => Any) = { 56 | TestActorRef(new Actor { 57 | override def receive: Receive = { 58 | case StorageNodeLocalRead(id) => sender() ! msgBack 59 | } 60 | }) 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/write/ReplicaLocalWriterTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.Data 6 | import justin.db.actors.protocol.{StorageNodeConflictedWrite, StorageNodeFailedWrite, StorageNodeSuccessfulWrite} 7 | import justin.db.consistenthashing.NodeId 8 | import justin.db.storage.PluggableStorageProtocol.{Ack, DataOriginality, StorageGetData} 9 | import justin.db.storage.{GetStorageProtocol, JustinData, PutStorageProtocol} 10 | import justin.db.vectorclocks.{Counter, VectorClock} 11 | import org.scalatest.concurrent.ScalaFutures 12 | import org.scalatest.{FlatSpec, Matchers} 13 | 14 | import scala.concurrent.ExecutionContext.Implicits.global 15 | import scala.concurrent.Future 16 | import scala.concurrent.duration._ 17 | 18 | class ReplicaLocalWriterTest extends FlatSpec with Matchers with ScalaFutures { 19 | 20 | behavior of "Replica Local Writer" 21 | 22 | override implicit def patienceConfig: PatienceConfig = PatienceConfig(10.seconds, 50.millis) 23 | 24 | /** 25 | * ------------------- 26 | * NONE scenarios | 27 | * ------------------- 28 | */ 29 | it should "save successfully new data for not taken identificator" in { 30 | // given 31 | val notTakenId = UUID.randomUUID() 32 | val data = Data(notTakenId, "some-value") 33 | val writer = new ReplicaLocalWriter(new GetStorageProtocol with PutStorageProtocol { 34 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.successful(StorageGetData.None) 35 | override def put(data: JustinData)(resolveOriginality: (UUID) => DataOriginality): Future[Ack] = Ack.future 36 | }) 37 | 38 | // when 39 | val result = writer.apply(data, null) 40 | 41 | // then 42 | whenReady(result) { _ shouldBe StorageNodeSuccessfulWrite(notTakenId) } 43 | } 44 | 45 | /** 46 | * ------------------- 47 | * FAILURE scenarios | 48 | * ------------------- 49 | */ 50 | it should "recover failure situation" in { 51 | // given 52 | val notTakenId = UUID.randomUUID() 53 | val data = Data(notTakenId, "some-value") 54 | val writer = new ReplicaLocalWriter(new GetStorageProtocol with PutStorageProtocol { 55 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.successful(StorageGetData.None) 56 | override def put(data: JustinData)(resolveOriginality: (UUID) => DataOriginality): Future[Ack] = Future.failed(new Exception) 57 | }) 58 | 59 | // when 60 | val result = writer.apply(data, null) 61 | 62 | // then 63 | whenReady(result) { _ shouldBe StorageNodeFailedWrite(notTakenId) } 64 | } 65 | 66 | /** 67 | * ------------------- 68 | * SINGLE scenarios | 69 | * ------------------- 70 | */ 71 | it should "fail to write predecessor to already stored data" in { 72 | // given 73 | val id = UUID.randomUUID() 74 | val data = Data(id, "some-value", VectorClock(Map(NodeId(1) -> Counter(2)))) 75 | val newData = Data(id, "some-value-2", VectorClock(Map(NodeId(1) -> Counter(1)))) 76 | val writer = new ReplicaLocalWriter(new GetStorageProtocol with PutStorageProtocol { 77 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.successful(StorageGetData.Single(data)) 78 | override def put(data: JustinData)(resolveOriginality: (UUID) => DataOriginality): Future[Ack] = ??? 79 | }) 80 | 81 | // when 82 | val result = writer.apply(newData, null) 83 | 84 | // then 85 | whenReady(result) { _ shouldBe StorageNodeFailedWrite(id) } 86 | } 87 | 88 | it should "get conflicted write when trying to save new data with conflicted vector clock comparing to already existed one" in { 89 | // given 90 | val id = UUID.randomUUID() 91 | val data = Data(id, "some-value", VectorClock(Map(NodeId(1) -> Counter(1)))) 92 | val newData = Data(id, "some-value-2", VectorClock(Map(NodeId(2) -> Counter(1)))) 93 | val writer = new ReplicaLocalWriter(new GetStorageProtocol with PutStorageProtocol { 94 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.successful(StorageGetData.Single(data)) 95 | override def put(data: JustinData)(resolveOriginality: (UUID) => DataOriginality): Future[Ack] = Ack.future 96 | }) 97 | 98 | // when 99 | val result = writer.apply(newData, null) 100 | 101 | // then 102 | whenReady(result) { _ shouldBe StorageNodeConflictedWrite(data, newData) } 103 | } 104 | 105 | it should "get successful write when trying to save new data with consequent vector clock comparing to already existed one" in { 106 | // given 107 | val id = UUID.randomUUID() 108 | val data = Data(id, "some-value", VectorClock(Map(NodeId(1) -> Counter(1)))) 109 | val newData = Data(id, "some-value-2", VectorClock(Map(NodeId(1) -> Counter(2)))) 110 | val writer = new ReplicaLocalWriter(new GetStorageProtocol with PutStorageProtocol { 111 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.successful(StorageGetData.Single(data)) 112 | override def put(data: JustinData)(resolveOriginality: (UUID) => DataOriginality): Future[Ack] = Ack.future 113 | }) 114 | 115 | // when 116 | val result = writer.apply(newData, null) 117 | 118 | // then 119 | whenReady(result) { _ shouldBe StorageNodeSuccessfulWrite(id) } 120 | } 121 | } 122 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/write/ReplicaRemoteWriterTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import java.util.UUID 4 | 5 | import akka.actor.{Actor, ActorSystem} 6 | import akka.testkit.{TestActorRef, TestKit} 7 | import justin.db.Data 8 | import justin.db.actors.StorageNodeActorRef 9 | import justin.db.actors.protocol.{StorageNodeFailedWrite, StorageNodeSuccessfulWrite, StorageNodeWriteDataLocal} 10 | import org.scalatest.concurrent.ScalaFutures 11 | import org.scalatest.{FlatSpecLike, Matchers} 12 | 13 | import scala.concurrent.duration._ 14 | 15 | class ReplicaRemoteWriterTest extends TestKit(ActorSystem("test-system")) 16 | with FlatSpecLike 17 | with Matchers 18 | with ScalaFutures { 19 | 20 | behavior of "Replica Remote Writer" 21 | 22 | override implicit def patienceConfig: PatienceConfig = PatienceConfig(10.seconds, 50.millis) 23 | 24 | it should "get info back that one of the saving is successful and second one has failed" in { 25 | // given 26 | val service = new ReplicaRemoteWriter()(system.dispatcher) 27 | val data = Data(id = UUID.randomUUID(), value = "exemplary-value") 28 | val storageSuccessfulActorRef = testActorRef(msgBack = StorageNodeSuccessfulWrite(data.id)) 29 | val storageFailedActorRef = testActorRef(msgBack = StorageNodeFailedWrite(data.id)) 30 | val storageNodeRefs = List(storageSuccessfulActorRef, storageFailedActorRef).map(StorageNodeActorRef) 31 | 32 | // when 33 | val writingResult = service.apply(storageNodeRefs, data) 34 | 35 | // then 36 | whenReady(writingResult) { _ shouldBe List(StorageNodeSuccessfulWrite(data.id), StorageNodeFailedWrite(data.id)) } 37 | } 38 | 39 | it should "recover failed behavior of actor" in { 40 | // given 41 | val service = new ReplicaRemoteWriter()(system.dispatcher) 42 | val data = Data(id = UUID.randomUUID(), value = "exemplary-value") 43 | val storageActorRef = testActorRef(new Exception) 44 | val storageNodeRefs = List(StorageNodeActorRef(storageActorRef)) 45 | 46 | // when 47 | val writingResult = service.apply(storageNodeRefs, data) 48 | 49 | // then 50 | whenReady(writingResult) { _ shouldBe List(StorageNodeFailedWrite(data.id)) } 51 | } 52 | 53 | private def testActorRef(msgBack: => Any) = { 54 | TestActorRef(new Actor { 55 | override def receive: Receive = { 56 | case StorageNodeWriteDataLocal(id) => sender() ! msgBack 57 | } 58 | }) 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/replica/write/ReplicaWriteAgreementTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.replica.write 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.actors.protocol.{StorageNodeFailedWrite, StorageNodeSuccessfulWrite} 6 | import justin.db.replica.W 7 | import org.scalatest.{FlatSpec, Matchers} 8 | 9 | class ReplicaWriteAgreementTest extends FlatSpec with Matchers { 10 | 11 | behavior of "Reach Consensus of Replicated Writes" 12 | 13 | it should "agreed on \"SuccessfulWrite\" if number of successful write is not less than client expectations" in { 14 | // given 15 | val w = W(2) 16 | val writes = List(StorageNodeSuccessfulWrite(UUID.randomUUID()), StorageNodeSuccessfulWrite(UUID.randomUUID()), StorageNodeFailedWrite(UUID.randomUUID())) 17 | 18 | // when 19 | val result = new ReplicaWriteAgreement().reach(w)(writes) 20 | 21 | // then 22 | result shouldBe WriteAgreement.Ok 23 | } 24 | 25 | it should "agreed on \"NotEnoughWrites\" if number of successful write is less than client expectations" in { 26 | // given 27 | val w = W(3) 28 | val writes = List(StorageNodeSuccessfulWrite(UUID.randomUUID()), StorageNodeSuccessfulWrite(UUID.randomUUID()), StorageNodeFailedWrite(UUID.randomUUID())) 29 | 30 | // when 31 | val result = new ReplicaWriteAgreement().reach(w)(writes) 32 | 33 | // then 34 | result shouldBe WriteAgreement.NotEnoughWrites 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /justin-core/src/test/scala/justin/db/versioning/NodeIdVectorClockBase64Test.scala: -------------------------------------------------------------------------------- 1 | package justin.db.versioning 2 | 3 | import java.nio.charset.StandardCharsets 4 | 5 | import justin.db.consistenthashing.NodeId 6 | import justin.db.vectorclocks.{Counter, VectorClock} 7 | import org.scalatest.{FlatSpec, Matchers} 8 | 9 | class NodeIdVectorClockBase64Test extends FlatSpec with Matchers { 10 | 11 | behavior of "Base64 decoder/encoder of Justin Vector Clock" 12 | 13 | it should "encode vector clock to string and decode it back to same init vector clock" in { 14 | // given 15 | val vcBase64 = new NodeIdVectorClockBase64 16 | 17 | val initVClock = VectorClock(Map(NodeId(1) -> Counter(1), NodeId(2) -> Counter(2), NodeId(3) -> Counter(9))) 18 | 19 | // when 20 | val encoded: String = vcBase64.encode(initVClock).get 21 | val decoded: VectorClock[NodeId] = vcBase64.decode(encoded).get 22 | 23 | // then 24 | decoded shouldBe initVClock 25 | } 26 | 27 | it should "use UTF_8 charset when encoding/decoding" in { 28 | NodeIdVectorClockBase64.charset shouldBe StandardCharsets.UTF_8 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /justin-http-api/src/main/scala/justin/httpapi/BuildInfoRouter.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.model._ 4 | import akka.http.scaladsl.server.Directives._ 5 | import akka.http.scaladsl.server._ 6 | 7 | class BuildInfoRouter { 8 | 9 | def routes(buildInfoJson: String): Route = path("info") { 10 | get { 11 | complete(HttpResponse(entity = HttpEntity(ContentType(MediaTypes.`application/json`), buildInfoJson))) 12 | } 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /justin-http-api/src/main/scala/justin/httpapi/HealthCheckRouter.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.model.StatusCodes 4 | import akka.http.scaladsl.server.Directives._ 5 | import akka.http.scaladsl.server.Route 6 | 7 | class HealthCheckRouter { 8 | 9 | val routes: Route = path("health") { 10 | get { complete(StatusCodes.OK) } 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /justin-http-api/src/main/scala/justin/httpapi/HttpRouter.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import java.util.UUID 4 | 5 | import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._ 6 | import akka.http.scaladsl.marshalling.ToResponseMarshallable 7 | import akka.http.scaladsl.model.StatusCodes._ 8 | import akka.http.scaladsl.server.Directives._ 9 | import akka.http.scaladsl.server.Route 10 | import justin.db.Data 11 | import justin.db.client.{ActorRefStorageNodeClient, GetValueResponse, WriteValueResponse} 12 | import justin.db.replica.{R, W} 13 | import justin.db.storage.Base64 14 | import justin.db.versioning.NodeIdVectorClockBase64 15 | import justin.httpapi.JustinDirectives._ 16 | import justin.httpapi.Unmarshallers.UUIDUnmarshaller 17 | import spray.json.DefaultJsonProtocol._ 18 | import spray.json.RootJsonFormat 19 | 20 | import scala.concurrent.ExecutionContext 21 | import scala.util.{Failure, Success} 22 | 23 | object HttpRouter { 24 | import Unmarshallers.UuidFormat 25 | 26 | case class Result(value: String) 27 | implicit val valueFormat: RootJsonFormat[Result] = jsonFormat1(Result) 28 | 29 | case class ConflictedData(id: String, value: String, vclock: Base64) 30 | implicit val conflictedDataFormat: RootJsonFormat[ConflictedData] = jsonFormat3(ConflictedData) 31 | 32 | case class PutValue(id: UUID, value: String, w: Int) 33 | implicit val putValueFormat: RootJsonFormat[PutValue] = jsonFormat3(PutValue) 34 | } 35 | 36 | class HttpRouter(client: ActorRefStorageNodeClient)(implicit ec: ExecutionContext) { 37 | import HttpRouter._ 38 | 39 | private[this] def transformConflictedData(data: Data) = { 40 | val vcBase64 = new NodeIdVectorClockBase64().encode(data.vclock).get 41 | ConflictedData(data.id.toString, data.value, vcBase64) 42 | } 43 | 44 | def routes: Route = withVectorClockHeader { vClockHeader => 45 | { 46 | (get & path("get") & pathEndOrSingleSlash & parameters(('id.as(UUIDUnmarshaller), 'r.as[Int]))) { (uuid, r) => 47 | onComplete(client.get(uuid, R(r))) { 48 | case Success(GetValueResponse.Found(data)) => respondWithHeader(VectorClockHeader(data.vclock)) { complete(OK -> Result(data.value)) } 49 | case Success(GetValueResponse.Conflicts(data)) => complete(MultipleChoices -> data.map(transformConflictedData)) 50 | case Success(GetValueResponse.NotFound(id)) => complete(NotFound -> Result(s"Couldn't found value with id ${id.toString}")) 51 | case Success(GetValueResponse.Failure(err)) => complete(BadRequest -> Result(err)) 52 | case Failure(ex) => complete(InternalServerError -> Result(ex.getMessage)) 53 | } 54 | } 55 | } ~ 56 | (post & path("put") & pathEndOrSingleSlash & entity(as[PutValue])) { putValue => 57 | complete { 58 | client.write(Data(putValue.id, putValue.value, vClockHeader.vectorClock), W(putValue.w)).map[ToResponseMarshallable] { 59 | case WriteValueResponse.Success(id) => NoContent 60 | case WriteValueResponse.Conflict => MultipleChoices -> Result("Multiple Choices") 61 | case WriteValueResponse.Failure(err) => BadRequest -> Result(err) 62 | } 63 | } 64 | } 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /justin-http-api/src/main/scala/justin/httpapi/JustinDirectives.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.server.Directive1 4 | import akka.http.scaladsl.server.Directives.optionalHeaderValueByType 5 | import akka.http.scaladsl.server.Directives.provide 6 | 7 | trait JustinDirectives { 8 | 9 | def withVectorClockHeader: Directive1[VectorClockHeader] = { 10 | optionalHeaderValueByType[VectorClockHeader]((): Unit).flatMap { 11 | case Some(header) => provide(header) 12 | case None => provide(VectorClockHeader.empty) 13 | } 14 | } 15 | } 16 | 17 | object JustinDirectives extends JustinDirectives 18 | -------------------------------------------------------------------------------- /justin-http-api/src/main/scala/justin/httpapi/Unmarshallers.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import java.util.UUID 4 | 5 | import akka.http.scaladsl.unmarshalling._ 6 | import akka.stream.Materializer 7 | import spray.json.{JsString, JsValue, JsonFormat, _} 8 | 9 | import scala.concurrent.{ExecutionContext, Future} 10 | import scala.util.{Failure, Success, Try} 11 | 12 | object Unmarshallers { 13 | 14 | implicit val UuidFormat = new JsonFormat[UUID] { 15 | override def read(json: JsValue): UUID = { 16 | json match { 17 | case JsString(uuid) => Try(UUID.fromString(uuid)) match { 18 | case Success(parsedUuid) => parsedUuid 19 | case Failure(_) => deserializationError("UUID could not be created from given string") 20 | } 21 | case _ => deserializationError("UUID could not be converted to UUID object.") 22 | } 23 | } 24 | override def write(obj: UUID): JsValue = JsString(obj.toString) 25 | } 26 | 27 | object UUIDUnmarshaller extends FromStringUnmarshaller[UUID] { 28 | override def apply(value: String)(implicit ec: ExecutionContext, materializer: Materializer): Future[UUID] = { 29 | Future.apply(UUID.fromString(value)) 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /justin-http-api/src/main/scala/justin/httpapi/VectorClockHeader.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.model.headers.{ModeledCustomHeader, ModeledCustomHeaderCompanion} 4 | import justin.db.consistenthashing.NodeId 5 | import justin.db.vectorclocks.VectorClock 6 | import justin.db.versioning.NodeIdVectorClockBase64 7 | 8 | import scala.util.Try 9 | 10 | case class VectorClockHeaderException(msg: String) extends Exception(msg) 11 | 12 | case class VectorClockHeader(vectorClock: VectorClock[NodeId]) extends ModeledCustomHeader[VectorClockHeader] { 13 | override def companion: ModeledCustomHeaderCompanion[VectorClockHeader] = VectorClockHeader 14 | 15 | override def value(): String = new NodeIdVectorClockBase64().encode(vectorClock) match { 16 | case scala.util.Success(vclock) => vclock 17 | case scala.util.Failure(_) => throw VectorClockHeaderException("Couldn't encode vector clock of data") 18 | } 19 | 20 | override def renderInResponses(): Boolean = true 21 | override def renderInRequests(): Boolean = true 22 | } 23 | 24 | object VectorClockHeader extends ModeledCustomHeaderCompanion[VectorClockHeader] { 25 | override def name: String = "X-Vector-Clock" 26 | 27 | override def parse(value: String): Try[VectorClockHeader] = { 28 | new NodeIdVectorClockBase64() 29 | .decode(value) 30 | .map(VectorClockHeader(_)) 31 | } 32 | 33 | def empty: VectorClockHeader = VectorClockHeader(VectorClock.apply[NodeId]()) 34 | } 35 | -------------------------------------------------------------------------------- /justin-http-api/src/test/resources/test.conf: -------------------------------------------------------------------------------- 1 | akka { 2 | stdout-loglevel = "OFF" 3 | loglevel = "OFF" 4 | } -------------------------------------------------------------------------------- /justin-http-api/src/test/scala/justin/httpapi/BuildInfoRouterTest.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.model.StatusCodes 4 | import akka.http.scaladsl.server.Route 5 | import akka.http.scaladsl.testkit.ScalatestRouteTest 6 | import org.scalatest.{FlatSpec, Matchers} 7 | 8 | class BuildInfoRouterTest extends FlatSpec with Matchers with ScalatestRouteTest { 9 | 10 | behavior of "Build Info Router" 11 | 12 | it should "get 200 OK http status along with system build info" in { 13 | Get("/info") ~> Route.seal(new BuildInfoRouter().routes("""{ "build" : "ok" "}""")) ~> check { 14 | status shouldBe StatusCodes.OK 15 | responseAs[String] shouldBe """{ "build" : "ok" "}""" 16 | } 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /justin-http-api/src/test/scala/justin/httpapi/HealthCheckRouterTest.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.model.StatusCodes 4 | import akka.http.scaladsl.server.Route 5 | import akka.http.scaladsl.testkit.ScalatestRouteTest 6 | import org.scalatest.{FlatSpec, Matchers} 7 | 8 | class HealthCheckRouterTest extends FlatSpec with Matchers with ScalatestRouteTest { 9 | 10 | behavior of "Health Check Router" 11 | 12 | it should "get 200 OK http status" in { 13 | Get("/health") ~> Route.seal(new HealthCheckRouter().routes) ~> check { 14 | status shouldBe StatusCodes.OK 15 | responseAs[String] shouldBe "OK" 16 | } 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /justin-http-api/src/test/scala/justin/httpapi/JustinDirectivesTest.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import akka.http.scaladsl.server.directives._ 4 | import akka.http.scaladsl.testkit.ScalatestRouteTest 5 | import justin.db.consistenthashing.NodeId 6 | import justin.db.vectorclocks.{Counter, VectorClock} 7 | import org.scalatest.{FlatSpec, Matchers} 8 | 9 | class JustinDirectivesTest extends FlatSpec with Matchers with ScalatestRouteTest 10 | with RouteDirectives 11 | with JustinDirectives { 12 | 13 | behavior of "Justin Directives" 14 | 15 | it should "provide empty VectorClock instance when no header is passed" in { 16 | Get("/") ~> withVectorClockHeader(x => complete(x.vectorClock.toString)) ~> check { 17 | responseAs[String] shouldBe VectorClockHeader.empty.vectorClock.toString 18 | } 19 | } 20 | 21 | it should "provide instance of VectorClock build upon passed header" in { 22 | val vClock = VectorClock(Map(NodeId(1) -> Counter(1), NodeId(2) -> Counter(2), NodeId(3) -> Counter(9))) 23 | val header = VectorClockHeader(vClock) 24 | 25 | Get("/").addHeader(header) ~> withVectorClockHeader(x => complete(x.vectorClock.toString)) ~> check { 26 | responseAs[String] shouldBe vClock.toString 27 | } 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /justin-http-api/src/test/scala/justin/httpapi/UnmarshallersTest.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import java.util.UUID 4 | 5 | import org.scalatest.{FlatSpec, Matchers} 6 | import spray.json.{DeserializationException, JsNumber, JsString} 7 | 8 | class UnmarshallersTest extends FlatSpec with Matchers { 9 | 10 | behavior of "Unmarshaller" 11 | 12 | it should "encode JSON into UUID" in { 13 | val uuid = UUID.randomUUID() 14 | val jsString = JsString(uuid.toString) 15 | 16 | Unmarshallers.UuidFormat.read(jsString) shouldBe uuid 17 | } 18 | 19 | it should "decode UUID into JSON" in { 20 | val uuid = UUID.randomUUID() 21 | val expectedJSON = Unmarshallers.UuidFormat.write(uuid) 22 | 23 | expectedJSON shouldBe JsString(uuid.toString) 24 | } 25 | 26 | it should "handle not expected format of JSON" in { 27 | val jsNumber = JsNumber(1) 28 | 29 | intercept[DeserializationException] { 30 | Unmarshallers.UuidFormat.read(jsNumber) 31 | } 32 | } 33 | 34 | it should "handle wrong format of UUID" in { 35 | val fakeUUID = "1-2-3-4" 36 | val jsString = JsString(fakeUUID) 37 | 38 | intercept[DeserializationException] { 39 | Unmarshallers.UuidFormat.read(jsString) 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /justin-http-api/src/test/scala/justin/httpapi/VectorClockHeaderTest.scala: -------------------------------------------------------------------------------- 1 | package justin.httpapi 2 | 3 | import justin.db.consistenthashing.NodeId 4 | import justin.db.vectorclocks.{Counter, VectorClock} 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | class VectorClockHeaderTest extends FlatSpec with Matchers { 8 | 9 | behavior of "Vector Clock Header" 10 | 11 | it should "parse string and create Vector Clock instance upon it" in { 12 | // given 13 | val encoded = "W1siMSIsMV0sWyIyIiwyXSxbIjMiLDldXQ==" 14 | 15 | // when 16 | val vClockHeader = VectorClockHeader.parse(encoded).get 17 | 18 | // then 19 | vClockHeader.vectorClock shouldBe VectorClock(Map(NodeId(1) -> Counter(1), NodeId(2) -> Counter(2), NodeId(3) -> Counter(9))) 20 | } 21 | 22 | it should "stringify Vector Clock instance" in { 23 | // given 24 | val vClock = VectorClock(Map(NodeId(1) -> Counter(1), NodeId(2) -> Counter(2), NodeId(3) -> Counter(9))) 25 | 26 | // when 27 | val encoded = VectorClockHeader(vClock).value() 28 | 29 | // then 30 | encoded shouldBe "W1siMSIsMV0sWyIyIiwyXSxbIjMiLDldXQ==" 31 | } 32 | 33 | it should "throw an Exception for not parsable Vector Clock" in { 34 | val vClock = null 35 | 36 | intercept[VectorClockHeaderException] { 37 | val encoded = VectorClockHeader(vClock).value() 38 | } 39 | } 40 | 41 | it should "render header in response" in { 42 | VectorClockHeader(null).renderInResponses() shouldBe true 43 | } 44 | 45 | it should "render header in request" in { 46 | VectorClockHeader(null).renderInRequests() shouldBe true 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /justin-ring/src/main/scala/justin/db/consistenthashing/NodeId.scala: -------------------------------------------------------------------------------- 1 | package justin.db.consistenthashing 2 | 3 | case class NodeId(id: Int) extends AnyVal 4 | -------------------------------------------------------------------------------- /justin-ring/src/main/scala/justin/db/consistenthashing/Ring.scala: -------------------------------------------------------------------------------- 1 | package justin.db.consistenthashing 2 | 3 | import justin.db.consistenthashing.Ring.RingPartitionId 4 | 5 | class Ring(val ring: Map[RingPartitionId, NodeId]) { 6 | 7 | lazy val size: Int = ring.size 8 | 9 | lazy val nodesId: Set[NodeId] = ring.values.toSet 10 | 11 | lazy val swap: Map[NodeId, List[RingPartitionId]] = { 12 | ring.groupBy(_._2).mapValues(_.keys.toList.sorted) 13 | } 14 | 15 | def getNodeId(id: RingPartitionId): Option[NodeId] = ring.get(id) 16 | 17 | def updated(ringPartitionId: RingPartitionId, nodeId: NodeId): Ring = new Ring(ring.updated(ringPartitionId, nodeId)) 18 | 19 | def nextPartitionId(id: RingPartitionId): RingPartitionId = (id + 1) % ring.size 20 | 21 | override def toString: String = ring.toString() 22 | } 23 | 24 | object Ring { 25 | type RingPartitionId = Int 26 | 27 | sealed trait AddNodeResult 28 | case object AlreadyExistingNodeId extends AddNodeResult 29 | case class UpdatedRingWithTakenPartitions(ring: Ring, takeOverDataFrom: List[(RingPartitionId, NodeId)]) extends AddNodeResult 30 | 31 | def addNode(ring: Ring, nodeId: NodeId): Ring.AddNodeResult = { 32 | if(ring.nodesId.contains(nodeId)) { 33 | Ring.AlreadyExistingNodeId 34 | } else { 35 | // this could be improved e.g. we should rely on least taken resources 36 | val takeOverDataFrom = (0 until ring.size by (ring.nodesId.size + 1)) 37 | .flatMap { ringPartitionId => ring.getNodeId(ringPartitionId).map(nodeId => (ringPartitionId, nodeId)) } 38 | .toList 39 | 40 | val updatedRing = takeOverDataFrom.foldLeft(ring) { 41 | case (acc, (ringPartitionId, _)) => acc.updated(ringPartitionId, nodeId) 42 | } 43 | 44 | Ring.UpdatedRingWithTakenPartitions(updatedRing, takeOverDataFrom) 45 | } 46 | } 47 | 48 | def apply(nodesSize: Int, partitionsSize: Int): Ring = { 49 | val partitions2Nodes = for { 50 | id <- 0 until nodesSize 51 | partitionId <- id until partitionsSize by nodesSize 52 | } yield (partitionId, NodeId(id)) 53 | 54 | new Ring(partitions2Nodes.toMap) 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /justin-ring/src/main/scala/justin/db/consistenthashing/UUID2RingPartitionId.scala: -------------------------------------------------------------------------------- 1 | package justin.db.consistenthashing 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.consistenthashing.Ring.RingPartitionId 6 | 7 | object UUID2RingPartitionId extends ((UUID, Ring) => Ring.RingPartitionId) { 8 | override def apply(id: UUID, ring: Ring): RingPartitionId = scala.math.abs(id.hashCode()) % ring.size 9 | } 10 | -------------------------------------------------------------------------------- /justin-ring/src/test/scala/justin/db/consistenthashing/RingTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.consistenthashing 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class RingTest extends FlatSpec with Matchers { 6 | 7 | behavior of "Ring" 8 | 9 | it should "define its size by nr of partitions" in { 10 | val N = 3 // nr of nodes 11 | val S = 50 // nr of partitions 12 | 13 | val ring = Ring.apply(N, S) 14 | 15 | ring.size shouldBe S 16 | } 17 | 18 | it should "has expected set of node ids" in { 19 | val N = 3 // nr of nodes 20 | val S = 50 // nr of partitions 21 | 22 | val ring = Ring.apply(N, S) 23 | 24 | ring.nodesId shouldBe Set(NodeId(0), NodeId(1), NodeId(2)) 25 | } 26 | 27 | it should "initialize Ring with vnodes" in { 28 | val N = 5 // nr of nodes - recommended by Riak database team 29 | val S = 64 // nr of partitions - recommended by Riak database team 30 | 31 | val ring = Ring.apply(N, S) 32 | 33 | val expectedSwappedRing = Map( 34 | NodeId(0) -> List(0, 5, 10, 15, 20, 25, 30, 35, 40, 45, 50, 55, 60), 35 | NodeId(1) -> List(1, 6, 11, 16, 21, 26, 31, 36, 41, 46, 51, 56, 61), 36 | NodeId(2) -> List(2, 7, 12, 17, 22, 27, 32, 37, 42, 47, 52, 57, 62), 37 | NodeId(3) -> List(3, 8, 13, 18, 23, 28, 33, 38, 43, 48, 53, 58, 63), 38 | NodeId(4) -> List(4, 9, 14, 19, 24, 29, 34, 39, 44, 49, 54, 59) 39 | ) 40 | 41 | ring.swap shouldBe expectedSwappedRing 42 | } 43 | 44 | it should "start indexing of partitions from 0" in { 45 | val ring = Ring.apply(nodesSize = 5, partitionsSize = 64) 46 | 47 | ring.getNodeId(0) shouldBe defined 48 | } 49 | 50 | it should "end indexing of partitions with index that is minus one of ring's size" in { 51 | val ring = Ring.apply(nodesSize = 5, partitionsSize = 64) 52 | 53 | val lastIdx = ring.size - 1 54 | 55 | ring.getNodeId(lastIdx) shouldBe defined 56 | ring.getNodeId(lastIdx + 1) should not be defined 57 | } 58 | 59 | it should "stringify itself" in { 60 | val ring = Ring.apply(nodesSize = 2, partitionsSize = 2) 61 | 62 | ring.toString shouldBe "Map(0 -> NodeId(0), 1 -> NodeId(1))" 63 | } 64 | 65 | it should "update value for particular key" in { 66 | val ring = Ring.apply(nodesSize = 5, partitionsSize = 64) 67 | 68 | val updatedRing = ring.updated(ringPartitionId = 2, nodeId = NodeId(100)) 69 | 70 | updatedRing.getNodeId(id = 2) shouldBe Some(NodeId(100)) 71 | } 72 | 73 | behavior of "Ring Add Node" 74 | 75 | it should "end up with AlreadyExistingNodeId when trying to add reserved node id" in { 76 | val ring = Ring.apply(nodesSize = 5, partitionsSize = 64) 77 | 78 | Ring.addNode(ring, nodeId = NodeId(0)) shouldBe Ring.AlreadyExistingNodeId 79 | Ring.addNode(ring, nodeId = NodeId(1)) shouldBe Ring.AlreadyExistingNodeId 80 | Ring.addNode(ring, nodeId = NodeId(2)) shouldBe Ring.AlreadyExistingNodeId 81 | Ring.addNode(ring, nodeId = NodeId(3)) shouldBe Ring.AlreadyExistingNodeId 82 | Ring.addNode(ring, nodeId = NodeId(4)) shouldBe Ring.AlreadyExistingNodeId 83 | } 84 | 85 | it should "take over some partitions by added node" in { 86 | // given 87 | val nodesSize = 4 88 | val partitionsSize = 36 89 | val ring = Ring.apply(nodesSize, partitionsSize) 90 | 91 | // when 92 | val nodeId = NodeId(5) 93 | val updateResult = Ring.addNode(ring, nodeId).asInstanceOf[Ring.UpdatedRingWithTakenPartitions] 94 | val updatedRing = updateResult.ring 95 | val takenPartitions = updateResult.takeOverDataFrom 96 | 97 | // then 98 | updatedRing.ring.size shouldBe ring.size 99 | updatedRing.nodesId shouldBe (ring.nodesId + nodeId) 100 | 101 | takenPartitions should not be empty 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /justin-ring/src/test/scala/justin/db/consistenthashing/UUID2RingPartitionIdTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.consistenthashing 2 | 3 | import java.util.UUID 4 | 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | class UUID2RingPartitionIdTest extends FlatSpec with Matchers { 8 | 9 | behavior of "mapping function from UUID to Ring's PartitionId" 10 | 11 | it should "use inner hashCode with scala.math.abs on it" in { 12 | val uid = UUID.randomUUID() 13 | val uidHashCode = uid.hashCode() 14 | val ring = Ring(nodesSize = 1, partitionsSize = 2) 15 | 16 | val expectedPartitionId = scala.math.abs(uidHashCode) % ring.size 17 | 18 | UUID2RingPartitionId.apply(uid, ring) shouldBe expectedPartitionId 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /justin-storage-api/src/main/scala/justin/db/storage/JustinData.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.util.UUID 4 | 5 | case class JustinData(id: UUID, value: String, vclock: Base64, timestamp: Long) 6 | -------------------------------------------------------------------------------- /justin-storage-api/src/main/scala/justin/db/storage/PluggableStorageProtocol.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.storage.PluggableStorageProtocol.{Ack, DataOriginality, StorageGetData} 6 | 7 | import scala.concurrent.Future 8 | 9 | trait GetStorageProtocol { 10 | def get(id: UUID)(resolveOriginality: UUID => DataOriginality): Future[StorageGetData] 11 | } 12 | 13 | trait PutStorageProtocol { 14 | def put(data: JustinData)(resolveOriginality: UUID => DataOriginality): Future[Ack] 15 | } 16 | 17 | trait PluggableStorageProtocol extends GetStorageProtocol with PutStorageProtocol 18 | 19 | object PluggableStorageProtocol { 20 | 21 | sealed trait StorageGetData 22 | object StorageGetData { 23 | case class Single(data: JustinData) extends StorageGetData 24 | case object None extends StorageGetData 25 | } 26 | 27 | sealed trait Ack 28 | case object Ack extends Ack { 29 | val future: Future[Ack] = Future.successful(Ack) 30 | } 31 | 32 | sealed trait DataOriginality { def ringPartitionId: RingPartitionId } 33 | object DataOriginality { 34 | case class Primary(ringPartitionId: RingPartitionId) extends DataOriginality 35 | case class Replica(ringPartitionId: RingPartitionId) extends DataOriginality 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /justin-storage-api/src/main/scala/justin/db/storage/package.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | package object storage { 4 | type RingPartitionId = Int 5 | type Base64 = String 6 | } 7 | -------------------------------------------------------------------------------- /justin-storage-api/src/main/scala/justin/db/storage/provider/StorageProvider.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.provider 2 | 3 | import justin.db.storage.PluggableStorageProtocol 4 | 5 | trait StorageProvider { 6 | def name: String 7 | def init: PluggableStorageProtocol 8 | } 9 | 10 | object StorageProvider { 11 | def apply(clazz: String): StorageProvider = Class.forName(clazz).newInstance().asInstanceOf[StorageProvider] 12 | } 13 | -------------------------------------------------------------------------------- /justin-storage-in-mem/src/main/resources/justin.conf: -------------------------------------------------------------------------------- 1 | justin { 2 | storage { 3 | inmemory { 4 | name = In-Mem Storage 5 | } 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /justin-storage-in-mem/src/main/resources/reference.conf: -------------------------------------------------------------------------------- 1 | include "justin.conf" -------------------------------------------------------------------------------- /justin-storage-in-mem/src/main/scala/justin/db/storage/InMemStorage.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.storage.PluggableStorageProtocol.{Ack, DataOriginality, StorageGetData} 6 | 7 | import scala.collection.mutable 8 | import scala.concurrent.Future 9 | 10 | /** 11 | * NOT THREAD-SAFE! 12 | */ 13 | class InMemStorage extends PluggableStorageProtocol { 14 | 15 | private type MMap = mutable.Map[RingPartitionId, Map[UUID, JustinData]] 16 | private var primaries: MMap = mutable.Map.empty[RingPartitionId, Map[UUID, JustinData]] 17 | private var replicas: MMap = mutable.Map.empty[RingPartitionId, Map[UUID, JustinData]] 18 | 19 | override def get(id: UUID)(resolveOriginality: (UUID) => DataOriginality): Future[StorageGetData] = Future.successful { 20 | def get(mmap: MMap, partitionId: RingPartitionId) = { 21 | mmap.get(partitionId).fold[StorageGetData](StorageGetData.None) { _.get(id) match { 22 | case Some(data) => StorageGetData.Single(data) 23 | case None => StorageGetData.None 24 | }} 25 | } 26 | 27 | resolveOriginality(id) match { 28 | case DataOriginality.Primary(partitionId) => get(primaries, partitionId) 29 | case DataOriginality.Replica(partitionId) => get(replicas, partitionId) 30 | } 31 | } 32 | 33 | override def put(data: JustinData)(resolveOriginality: (UUID) => DataOriginality): Future[Ack] = { 34 | def update(mmap: MMap, partitionId: RingPartitionId, data: JustinData) = { 35 | mmap.get(partitionId) match { 36 | case Some(partitionMap) => mmap + (partitionId -> (partitionMap ++ Map(data.id -> data))) 37 | case None => mmap + (partitionId -> Map(data.id -> data)) 38 | } 39 | } 40 | 41 | resolveOriginality(data.id) match { 42 | case DataOriginality.Primary(partitionId) => primaries = update(primaries, partitionId, data) 43 | case DataOriginality.Replica(partitionId) => replicas = update(replicas, partitionId, data) 44 | } 45 | 46 | Ack.future 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /justin-storage-in-mem/src/main/scala/justin/db/storage/config/justin.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.config 2 | 3 | import com.wacai.config.annotation._ 4 | 5 | // $COVERAGE-OFF$ 6 | @conf 7 | trait justin { 8 | 9 | val storage = new { 10 | val inmemory = new { 11 | val name: String = "In-Mem Storage" 12 | } 13 | } 14 | } 15 | 16 | object StorageConfig extends justin 17 | // $COVERAGE-ON$ 18 | -------------------------------------------------------------------------------- /justin-storage-in-mem/src/main/scala/justin/db/storage/provider/InMemStorageProvider.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.provider 2 | 3 | import justin.db.storage.config.StorageConfig 4 | import justin.db.storage.{InMemStorage, PluggableStorageProtocol} 5 | 6 | class InMemStorageProvider extends StorageProvider { 7 | 8 | override def init: PluggableStorageProtocol = new InMemStorage 9 | 10 | override def name: String = StorageConfig.storage.inmemory.name 11 | } 12 | -------------------------------------------------------------------------------- /justin-storage-in-mem/src/test/scala/justin/db/storage/InMemStorageTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.util.UUID 4 | 5 | import justin.db.storage.PluggableStorageProtocol.{Ack, DataOriginality, StorageGetData} 6 | import org.scalatest.{FlatSpec, Matchers} 7 | 8 | import scala.concurrent.Await 9 | import scala.concurrent.duration._ 10 | import scala.language.postfixOps 11 | 12 | class InMemStorageTest extends FlatSpec with Matchers { 13 | 14 | behavior of "In-memory storage" 15 | 16 | it should "store single data" in { 17 | // given 18 | val data = prepareData(UUID.randomUUID(), "some-data") 19 | val inMemStorage = new InMemStorage 20 | val resolver = (id: UUID) => DataOriginality.Primary(ringPartitionId = 1) 21 | 22 | // when 23 | val result = Await.result(inMemStorage.put(data)(resolver), atMost = 5 seconds) 24 | 25 | // then 26 | result shouldBe Ack 27 | } 28 | 29 | it should "get single data for appropriate identifier" in { 30 | // given 31 | val data = prepareData(UUID.randomUUID(), "some-data") 32 | val inMemStorage = new InMemStorage 33 | val resolver = (id: UUID) => DataOriginality.Primary(ringPartitionId = 1) 34 | Await.result(inMemStorage.put(data)(resolver), atMost = 5 seconds) 35 | 36 | // when 37 | val result = Await.result(inMemStorage.get(data.id)(resolver), atMost = 5 seconds) 38 | 39 | // then 40 | result shouldBe StorageGetData.Single(data) 41 | } 42 | 43 | it should "get none data for not existing id in memory" in { 44 | // given 45 | val noExistingId = UUID.randomUUID() 46 | val inMemStorage = new InMemStorage 47 | val resolver = (id: UUID) => DataOriginality.Primary(ringPartitionId = 1) 48 | val otherData = prepareData(id = UUID.randomUUID(), "some-data") 49 | Await.result(inMemStorage.put(otherData)(resolver), atMost = 5 seconds) 50 | 51 | // when 52 | val result = Await.result(inMemStorage.get(noExistingId)(resolver), atMost = 5 seconds) 53 | 54 | // then 55 | result shouldBe StorageGetData.None 56 | } 57 | 58 | it should "get none data for not existing partitionId" in { 59 | // given 60 | val uid = UUID.randomUUID() 61 | val noExistingRingPartitionId = 1 62 | val inMemStorage = new InMemStorage 63 | val resolver = (id: UUID) => DataOriginality.Replica(ringPartitionId = noExistingRingPartitionId) 64 | 65 | // when 66 | val result = Await.result(inMemStorage.get(uid)(resolver), atMost = 5 seconds) 67 | 68 | // then 69 | result shouldBe StorageGetData.None 70 | } 71 | 72 | it should "store and merge many data within under single partitionId" in { 73 | // given 74 | val id1 = UUID.randomUUID() 75 | val id2 = UUID.randomUUID() 76 | val data1 = prepareData(id1, "some-data") 77 | val data2 = prepareData(id2, "some-data") 78 | val resolver = (id: UUID) => DataOriginality.Replica(ringPartitionId = 1) 79 | val inMemStorage = new InMemStorage 80 | 81 | Await.result(inMemStorage.put(data1)(resolver), atMost = 5 seconds) 82 | Await.result(inMemStorage.put(data2)(resolver), atMost = 5 seconds) 83 | 84 | // when 85 | val result1 = Await.result(inMemStorage.get(id1)(resolver), atMost = 5 seconds) 86 | val result2 = Await.result(inMemStorage.get(id2)(resolver), atMost = 5 seconds) 87 | 88 | // then 89 | result1 shouldBe StorageGetData.Single(data1) 90 | result2 shouldBe StorageGetData.Single(data2) 91 | } 92 | 93 | private def prepareData(id: UUID, value: String) = JustinData(id, value, "", 1L) 94 | } 95 | -------------------------------------------------------------------------------- /justin-storage-in-mem/src/test/scala/justin/db/storage/provider/InMemStorageProviderTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.provider 2 | 3 | import justin.db.storage.InMemStorage 4 | import org.scalatest.{FlatSpec, Matchers} 5 | 6 | class InMemStorageProviderTest extends FlatSpec with Matchers { 7 | 8 | it should "init In-Memory storage" in { 9 | val provider = StorageProvider.apply("justin.db.storage.provider.InMemStorageProvider").asInstanceOf[InMemStorageProvider] 10 | 11 | provider.name shouldBe "In-Mem Storage" 12 | provider.init shouldBe a[InMemStorage] 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/main/resources/justin.conf: -------------------------------------------------------------------------------- 1 | justin { 2 | storage { 3 | rocksdb { 4 | name = RocksDB storage 5 | journal-path = . 6 | } 7 | } 8 | } 9 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/main/resources/reference.conf: -------------------------------------------------------------------------------- 1 | include "justin.conf" -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/main/scala/justin/db/storage/RocksDBStorage.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.io.{Input, Output} 7 | import com.esotericsoftware.kryo.{Kryo, Serializer} 8 | import justin.db.storage.PluggableStorageProtocol.{Ack, StorageGetData} 9 | import org.rocksdb.{FlushOptions, Options, RocksDB} 10 | 11 | import scala.concurrent.Future 12 | 13 | // TODO: 14 | // Current version store every single data under one file (totally doesn't care about data originality). 15 | // Data should be eventually splitted by ring partitionId. 16 | // This might be an issue during possible data movements between nodes. 17 | final class RocksDBStorage(dir: File) extends PluggableStorageProtocol { 18 | import RocksDBStorage._ 19 | 20 | { 21 | RocksDB.loadLibrary() 22 | } 23 | 24 | private[this] val kryo = new Kryo() 25 | 26 | private[this] val db: RocksDB = { 27 | val options: Options = new Options().setCreateIfMissing(true) 28 | RocksDB.open(options, dir.getPath) 29 | } 30 | 31 | override def get(id: UUID)(resolveOriginality: (UUID) => PluggableStorageProtocol.DataOriginality): Future[PluggableStorageProtocol.StorageGetData] = { 32 | val key: Array[Byte] = uuid2bytes(kryo, id) 33 | val dataBytes: Array[Byte] = db.get(key) 34 | 35 | val justinDataOpt = Option(dataBytes).map { dataBytes => 36 | val input = new Input(new ByteArrayInputStream(dataBytes)) 37 | JustinDataSerializer.read(kryo, input, classOf[JustinData]) 38 | } 39 | 40 | Future.successful(justinDataOpt.map(StorageGetData.Single).getOrElse(StorageGetData.None)) 41 | } 42 | 43 | override def put(data: JustinData)(resolveOriginality: (UUID) => PluggableStorageProtocol.DataOriginality): Future[PluggableStorageProtocol.Ack] = { 44 | val key: Array[Byte] = uuid2bytes(kryo, data.id) 45 | val dataBytes: Array[Byte] = { 46 | val output = new Output(new ByteArrayOutputStream()) 47 | JustinDataSerializer.write(kryo, output, data) 48 | output.getBuffer 49 | } 50 | 51 | db.put(key, dataBytes) 52 | db.flush(new FlushOptions().setWaitForFlush(true)) 53 | 54 | Ack.future 55 | } 56 | } 57 | 58 | object RocksDBStorage { 59 | 60 | def uuid2bytes(kryo: Kryo, id: UUID): Array[Byte] = { 61 | val output = new Output(new ByteArrayOutputStream(), 16) 62 | UUIDSerializer.write(kryo, output, id) 63 | output.getBuffer 64 | } 65 | 66 | object UUIDSerializer extends Serializer[UUID] { 67 | override def read(kryo: Kryo, input: Input, `type`: Class[UUID]): UUID = { 68 | new UUID(input.readLong, input.readLong) 69 | } 70 | 71 | override def write(kryo: Kryo, output: Output, uuid: UUID): Unit = { 72 | output.writeLong(uuid.getMostSignificantBits) 73 | output.writeLong(uuid.getLeastSignificantBits) 74 | } 75 | } 76 | 77 | object JustinDataSerializer extends Serializer[JustinData] { 78 | override def read(kryo: Kryo, input: Input, `type`: Class[JustinData]): JustinData = { 79 | JustinData( 80 | id = UUIDSerializer.read(kryo, input, classOf[UUID]), 81 | value = input.readString(), 82 | vclock = input.readString(), 83 | timestamp = input.readLong() 84 | ) 85 | } 86 | 87 | override def write(kryo: Kryo, output: Output, data: JustinData): Unit = { 88 | UUIDSerializer.write(kryo, output, data.id) 89 | output.writeString(data.value) 90 | output.writeString(data.vclock) 91 | output.writeLong(data.timestamp) 92 | } 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/main/scala/justin/db/storage/config/justin.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.config 2 | 3 | import com.wacai.config.annotation._ 4 | 5 | // $COVERAGE-OFF$ 6 | @conf 7 | trait justin { 8 | 9 | val storage = new { 10 | val rocksdb = new { 11 | val name: String = "RocksDB storage" 12 | val `journal-path`: String = "." 13 | } 14 | } 15 | } 16 | 17 | object StorageConfig extends justin 18 | // $COVERAGE-ON$ 19 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/main/scala/justin/db/storage/provider/RocksDBStorageProvider.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.provider 2 | 3 | import java.io.File 4 | 5 | import justin.db.storage.{PluggableStorageProtocol, RocksDBStorage} 6 | import justin.db.storage.config.StorageConfig 7 | 8 | class RocksDBStorageProvider extends StorageProvider { 9 | override def init: PluggableStorageProtocol = { 10 | val dir = new File(StorageConfig.storage.rocksdb.`journal-path`) 11 | new RocksDBStorage(dir) 12 | } 13 | 14 | override def name: String = StorageConfig.storage.rocksdb.name 15 | } 16 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/test/scala/justin/db/storage/JustinDataSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.{Input, Output} 8 | import justin.db.storage.RocksDBStorage.JustinDataSerializer 9 | import org.scalatest.{FlatSpec, Matchers} 10 | 11 | class JustinDataSerializerTest extends FlatSpec with Matchers { 12 | 13 | behavior of "JustinDataSerializer" 14 | 15 | it should "serialize/deserialize JustinData with Kryo" in { 16 | val kryo = new Kryo() 17 | val data = JustinData( 18 | id = UUID.randomUUID, 19 | value = "to jest przykladowa wartość", 20 | vclock = "vclock-value", 21 | timestamp = 1234124L 22 | ) 23 | 24 | // serialize 25 | val output = new Output(new ByteArrayOutputStream()) 26 | JustinDataSerializer.write(kryo, output, data) 27 | val dataBytes = output.getBuffer 28 | 29 | // deserialize 30 | val input = new Input(new ByteArrayInputStream(dataBytes)) 31 | JustinDataSerializer.read(kryo, input, classOf[JustinData]) shouldBe data 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/test/scala/justin/db/storage/RocksDBStorageTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.nio.file.Files 4 | import java.util.UUID 5 | 6 | import justin.db.storage.PluggableStorageProtocol.{Ack, DataOriginality, StorageGetData} 7 | import org.scalatest.concurrent.ScalaFutures 8 | import org.scalatest.{FlatSpec, Matchers} 9 | 10 | import scala.concurrent.ExecutionContext.Implicits.global 11 | import scala.concurrent.duration._ 12 | 13 | class RocksDBStorageTest extends FlatSpec with Matchers with ScalaFutures { 14 | 15 | behavior of "RocksDBStorage" 16 | 17 | it should "save 3 payloads and read them" in { 18 | val journal = Files.createTempDirectory("rocksdb") 19 | val rocksdb = new RocksDBStorage(journal.toFile) 20 | val data1 = JustinData( 21 | id = UUID.randomUUID, 22 | value = "1", 23 | vclock = "vclock-value", 24 | timestamp = 1234124L 25 | ) 26 | val data2 = JustinData( 27 | id = UUID.randomUUID, 28 | value = "1", 29 | vclock = "vclock-value", 30 | timestamp = 1234124L 31 | ) 32 | val data3 = JustinData( 33 | id = UUID.randomUUID, 34 | value = "3", 35 | vclock = "vclock-value", 36 | timestamp = 1234124L 37 | ) 38 | val dataOriginality = DataOriginality.Primary(ringPartitionId = 1) 39 | 40 | // PUT 41 | rocksdb.put(data1)(_ => dataOriginality).futureValue shouldBe Ack 42 | rocksdb.put(data2)(_ => dataOriginality).futureValue shouldBe Ack 43 | rocksdb.put(data3)(_ => dataOriginality).futureValue shouldBe Ack 44 | 45 | // GET 46 | rocksdb.get(data3.id)(_ => dataOriginality).futureValue shouldBe StorageGetData.Single(data3) 47 | rocksdb.get(data2.id)(_ => dataOriginality).futureValue shouldBe StorageGetData.Single(data2) 48 | rocksdb.get(data1.id)(_ => dataOriginality).futureValue shouldBe StorageGetData.Single(data1) 49 | } 50 | 51 | override implicit def patienceConfig: PatienceConfig = PatienceConfig(10.seconds, 50.millis) 52 | } 53 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/test/scala/justin/db/storage/UUIDSerializerTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage 2 | 3 | import java.io.ByteArrayInputStream 4 | import java.util.UUID 5 | 6 | import com.esotericsoftware.kryo.Kryo 7 | import com.esotericsoftware.kryo.io.Input 8 | import justin.db.storage.RocksDBStorage.UUIDSerializer 9 | import org.scalatest.{FlatSpec, Matchers} 10 | 11 | class UUIDSerializerTest extends FlatSpec with Matchers { 12 | 13 | behavior of "UUIDSerializer" 14 | 15 | it should "serialize/deserialize UUID with Kryo" in { 16 | val uuid = UUID.randomUUID() 17 | val kryo = new Kryo() 18 | 19 | // serialize 20 | val bytes = RocksDBStorage.uuid2bytes(kryo, uuid) 21 | 22 | // deserialize 23 | val input = new Input(new ByteArrayInputStream(bytes)) 24 | val id = UUIDSerializer.read(kryo, input, classOf[UUID]) 25 | 26 | uuid shouldBe id 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /justin-storage-rocksdb/src/test/scala/justin/db/storage/provider/RocksDBStorageProviderTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.storage.provider 2 | 3 | import justin.db.storage.RocksDBStorage 4 | import org.scalatest.{FlatSpec, Matchers} 5 | 6 | class RocksDBStorageProviderTest extends FlatSpec with Matchers { 7 | 8 | it should "init RocksDB storage" in { 9 | val provider = StorageProvider.apply("justin.db.storage.provider.RocksDBStorageProvider").asInstanceOf[RocksDBStorageProvider] 10 | 11 | provider.name shouldBe "RocksDB storage" 12 | provider.init shouldBe a[RocksDBStorage] 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/main/scala/justin/db/vectorclocks/Counter.scala: -------------------------------------------------------------------------------- 1 | package justin.db.vectorclocks 2 | 3 | case class Counter(value: Int) extends AnyVal { 4 | def addOne: Counter = this.copy(value = value + 1) 5 | } 6 | 7 | object Counter { 8 | def max(c1: Counter, c2: Counter): Counter = { 9 | Counter(scala.math.max(c1.value, c2.value)) 10 | } 11 | 12 | def zero: Counter = Counter(0) 13 | } 14 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/main/scala/justin/db/vectorclocks/VectorClock.scala: -------------------------------------------------------------------------------- 1 | package justin.db.vectorclocks 2 | 3 | case class VectorClock[Id](private val clock: Map[Id, Counter]) { 4 | def get(id: Id): Option[Counter] = clock.get(id) 5 | 6 | def increase(id: Id): VectorClock[Id] = { 7 | val searchedCounter = clock.getOrElse(id, Counter.zero) 8 | val updatedCounter = searchedCounter.addOne 9 | 10 | VectorClock(clock + (id -> updatedCounter)) 11 | } 12 | 13 | def toList: List[(Id, Counter)] = clock.toList 14 | 15 | def keys: Set[Id] = clock.keys.toSet 16 | } 17 | 18 | object VectorClock { 19 | 20 | def apply[Id](): VectorClock[Id] = VectorClock(Map.empty[Id, Counter]) 21 | 22 | def empty[Id](id: Id): VectorClock[Id] = VectorClock(Map(id -> Counter.zero)) 23 | 24 | def merge[Id](receiverId: Id, vc1: VectorClock[Id], vc2: VectorClock[Id]): VectorClock[Id] = { 25 | val mergedClocks = vc1.clock ++ vc2.clock 26 | 27 | val mergedCounter = (vc1.clock.get(receiverId), vc2.clock.get(receiverId)) match { 28 | case (Some(counter1), Some(counter2)) => Counter.max(counter1, counter2) 29 | case (None, Some(counter2)) => counter2 30 | case (Some(counter1), None) => counter1 31 | case (None, None) => Counter.zero 32 | } 33 | 34 | val counter = mergedCounter.addOne 35 | 36 | VectorClock(mergedClocks + (receiverId -> counter)) 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/main/scala/justin/db/vectorclocks/VectorClockComparator.scala: -------------------------------------------------------------------------------- 1 | package justin.db.vectorclocks 2 | 3 | import justin.db.vectorclocks.VectorClockComparator.VectorClockRelation 4 | 5 | // DIRTY ! 6 | class VectorClockComparator[Id] extends ((VectorClock[Id], VectorClock[Id]) => VectorClockRelation) { 7 | 8 | override def apply(baseVC: VectorClock[Id], comparedVC: VectorClock[Id]): VectorClockRelation = { 9 | val vcKeys = baseVC.keys 10 | val vc2Keys = comparedVC.keys 11 | 12 | val vc2ContainsAllKeysOfVc = !comparedVC.keys.forall(vcKeys.contains) 13 | val vcContainsAllKeysOfVc2 = !vcKeys.forall(vc2Keys.contains) 14 | 15 | val (counter1, counter2) = vc2Keys.foldLeft((0,0)) { (counter, vc2Key) => 16 | val vc1Val = baseVC.get(vc2Key) 17 | val vc2Val = comparedVC.get(vc2Key) 18 | 19 | if(vc1Val.isEmpty || vc1Val.get.value == vc2Val.get.value) { 20 | counter 21 | } else { 22 | if(vc1Val.get.value > vc2Val.get.value) (counter._1 + 1, counter._2) 23 | else (counter._1, counter._2 + 1) 24 | } 25 | } 26 | 27 | if(isConflict(vc2ContainsAllKeysOfVc, vcContainsAllKeysOfVc2, counter1, counter2)) { 28 | VectorClockRelation.Conflict 29 | } else if(isConsequent(vc2ContainsAllKeysOfVc, counter1, counter2)) { 30 | VectorClockRelation.Consequent 31 | } else { 32 | VectorClockRelation.Predecessor 33 | } 34 | } 35 | 36 | private def isConflict(vc2ContainsAllKeysOfVc: Boolean, vcContainsAllKeysOfVc2: Boolean, counter1: Int, counter2: Int) = { 37 | (vc2ContainsAllKeysOfVc && vcContainsAllKeysOfVc2) || (vc2ContainsAllKeysOfVc && counter1 > 0) || (counter1 > 0 && counter2 > 0) 38 | } 39 | private def isConsequent(vc2ContainsAllKeysOfVc: Boolean, counter1: Int, counter2: Int) = { 40 | vc2ContainsAllKeysOfVc && counter1 >= 0 && counter2 >= 0 || !(counter1 >= 0 && counter2 == 0) 41 | } 42 | } 43 | 44 | object VectorClockComparator { 45 | 46 | sealed trait VectorClockRelation 47 | object VectorClockRelation { 48 | case object Predecessor extends VectorClockRelation 49 | case object Conflict extends VectorClockRelation 50 | case object Consequent extends VectorClockRelation 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/main/scala/justin/db/vectorclocks/package.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import scala.language.implicitConversions 4 | 5 | package object vectorclocks { 6 | 7 | /** 8 | * Create Vector Clock from plain string eg. "A:1, B:1, C:1" 9 | */ 10 | implicit class VectorClockOps(plain: String) { 11 | def toVectorClock[Id](implicit string2Id: String => Id): VectorClock[Id] = VectorClock.apply { 12 | plain.split(",").map { s => 13 | val Array(key, value) = s.trim.split(":") 14 | (string2Id(key), Counter(value.toInt)) 15 | }.toMap 16 | } 17 | } 18 | 19 | object VectorClockOps { 20 | implicit def stringAsId(s: String): VectorClock[String] = s.toVectorClock[String] 21 | implicit def intAsId(s: String): VectorClock[Int] = s.toVectorClock[Int](_.toInt) 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/test/scala/justin/db/vectorclocks/CounterTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.vectorclocks 2 | 3 | import org.scalatest.{FlatSpec, Matchers} 4 | 5 | class CounterTest extends FlatSpec with Matchers { 6 | 7 | behavior of "Counter" 8 | 9 | it should "create new Counter with increased by one when invoking \"addOne\" method on it" in { 10 | val counter = Counter(0) 11 | 12 | val increased = counter.addOne 13 | 14 | increased shouldBe Counter(1) 15 | } 16 | 17 | it should "choose Counter with bigger value" in { 18 | val counter1 = Counter(0) 19 | val counter2 = Counter(100) 20 | 21 | val max = Counter.max(counter1, counter2) 22 | 23 | max shouldBe counter2 24 | } 25 | 26 | it should "create new Counter with 0 value using zero method from companion object" in { 27 | Counter.zero shouldBe Counter(0) 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/test/scala/justin/db/vectorclocks/VectorClockComparatorTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.vectorclocks 2 | 3 | import justin.db.vectorclocks.VectorClockComparator.VectorClockRelation 4 | import justin.db.vectorclocks.VectorClockOps._ 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | class VectorClockComparatorTest extends FlatSpec with Matchers { 8 | 9 | behavior of "Vector Clock's comparator" 10 | 11 | private val comparator = new VectorClockComparator[String] 12 | 13 | /** 14 | * ------------------- 15 | * CONFLICT scenarios | 16 | * ------------------- 17 | */ 18 | it should "pass conflict scenario nr 1" in { 19 | val baseVC = "A:1, B:1" 20 | val comparedVC = "A:1, C:1" 21 | 22 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Conflict 23 | } 24 | 25 | it should "pass conflict scenario nr 2" in { 26 | val baseVC = "A:2" 27 | val comparedVC = "A:1, B:1" 28 | 29 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Conflict 30 | } 31 | 32 | it should "pass conflict scenario nr 3" in { 33 | val baseVC = "A:1" 34 | val comparedVC = "B:1" 35 | 36 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Conflict 37 | } 38 | 39 | it should "pass conflict scenario nr 4" in { 40 | val baseVC = "A:1, B:1, C:2" 41 | val comparedVC = "A:1, B:2, C:1" 42 | 43 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Conflict 44 | } 45 | 46 | it should "pass conflict scenario nr 5" in { 47 | val baseVC = "A:1, B:2" 48 | val comparedVC = "A:1, B:1, C:1" 49 | 50 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Conflict 51 | } 52 | 53 | /** 54 | * ---------------------- 55 | * PREDECESSOR scenarios | 56 | * ---------------------- 57 | */ 58 | it should "pass predecessor scenario nr 1" in { 59 | val baseVC = "A:2, B:1" 60 | val comparedVC = "A:1, B:1" 61 | 62 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Predecessor 63 | } 64 | 65 | it should "pass predecessor scenario nr 2" in { 66 | val baseVC = "A:2" 67 | val comparedVC = "A:1" 68 | 69 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Predecessor 70 | } 71 | 72 | it should "pass predecessor scenario nr 3" in { 73 | val baseVC = "A:1, B:1" 74 | val comparedVC = "A:1, B:1" 75 | 76 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Predecessor 77 | } 78 | 79 | /** 80 | * ---------------------- 81 | * CONSEQUENT scenarios | 82 | * ---------------------- 83 | */ 84 | it should "pass consequent scenario nr 1" in { 85 | val baseVC = "A:1" 86 | val comparedVC = "A:2" 87 | 88 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Consequent 89 | } 90 | 91 | it should "pass consequent scenario nr 2" in { 92 | val baseVC = "A:1, B:1" 93 | val comparedVC = "A:1, B:1, C:1" 94 | 95 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Consequent 96 | } 97 | 98 | it should "pass consequent scenario nr 3" in { 99 | val baseVC = "A:1, B:1, C:2" 100 | val comparedVC = "A:1, B:1, C:3" 101 | 102 | comparator.apply(baseVC, comparedVC) shouldBe VectorClockRelation.Consequent 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/test/scala/justin/db/vectorclocks/VectorClockOpsTest.scala: -------------------------------------------------------------------------------- 1 | 2 | package justin.db.vectorclocks 3 | 4 | import org.scalatest.{FlatSpec, Matchers} 5 | 6 | class VectorClockOpsTest extends FlatSpec with Matchers { 7 | 8 | behavior of "Vector Clock Ops" 9 | 10 | it should "create Vector Clock instance from plain string" in { 11 | "A:2".toVectorClock[String] shouldBe VectorClock(Map("A" -> Counter(2))) 12 | "A:1, B:2".toVectorClock[String] shouldBe VectorClock(Map("A" -> Counter(1), "B" -> Counter(2))) 13 | "A:1, B:1, C:1".toVectorClock[String] shouldBe VectorClock(Map("A" -> Counter(1), "B" -> Counter(1), "C" -> Counter(1))) 14 | } 15 | 16 | it should "create Vector Clock instance from plain string with numerical ids" in { 17 | import VectorClockOps.intAsId 18 | 19 | ("1:2": VectorClock[Int]) shouldBe VectorClock(Map(1 -> Counter(2))) 20 | ("1:2, 2:10": VectorClock[Int]) shouldBe VectorClock(Map(1 -> Counter(2), 2 -> Counter(10))) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /justin-vector-clocks/src/test/scala/justin/db/vectorclocks/VectorClockTest.scala: -------------------------------------------------------------------------------- 1 | package justin.db.vectorclocks 2 | 3 | import java.util.UUID 4 | 5 | import org.scalatest.{FlatSpec, Matchers} 6 | 7 | class VectorClockTest extends FlatSpec with Matchers { 8 | 9 | behavior of "Vector Clock" 10 | 11 | it should "initialize an empty Vector Clock with passed id" in { 12 | val id = UUID.randomUUID() 13 | 14 | val vc = VectorClock.empty(id) 15 | 16 | vc shouldBe VectorClock(Map(id -> Counter(0))) 17 | } 18 | 19 | it should "increase corresponding counter by one for particular id" in { 20 | val id = UUID.randomUUID() 21 | val vc = VectorClock.empty(id) 22 | 23 | val increased = vc.increase(id) 24 | 25 | increased shouldBe VectorClock(Map(id -> Counter(1))) 26 | } 27 | 28 | it should "merge two vector clocks" in { 29 | val id1 = UUID.randomUUID() 30 | val vc1 = VectorClock(Map(id1 -> Counter(109))) 31 | 32 | val id2 = UUID.randomUUID() 33 | val vc2 = VectorClock(Map( 34 | id1 -> Counter(1), 35 | id2 -> Counter(99) 36 | )) 37 | 38 | val receiverId = id1 39 | 40 | val merged = VectorClock.merge(receiverId, vc1, vc2) 41 | 42 | merged.get(id1).get shouldBe Counter(110) 43 | merged.get(id2).get shouldBe Counter(99) 44 | } 45 | 46 | it should "merge two vector clocks without having passed \"receiverId\" key" in { 47 | val id1 = UUID.randomUUID() 48 | val vc1 = VectorClock.empty(id1) 49 | 50 | val id2 = UUID.randomUUID() 51 | val vc2 = VectorClock.empty(id2) 52 | 53 | val receiverId = UUID.randomUUID() 54 | 55 | val merged = VectorClock.merge(receiverId, vc1, vc2) 56 | 57 | merged.get(id1).get shouldBe Counter(0) 58 | merged.get(id2).get shouldBe Counter(0) 59 | merged.get(receiverId).get shouldBe Counter(1) 60 | } 61 | 62 | it should "init an empty Vector Clock" in { 63 | type Id = Int 64 | val vc = VectorClock.apply[Id]() 65 | 66 | vc shouldBe VectorClock(Map.empty[Id, Counter]) 67 | } 68 | 69 | it should "list Vector Clock" in { 70 | val vc = VectorClock(Map(1 -> Counter(109))) 71 | 72 | val list = vc.toList 73 | 74 | list shouldBe List((1, Counter(109))) 75 | } 76 | 77 | it should "get keys" in { 78 | val vc = VectorClock(Map(1 -> Counter(109), 2 -> Counter(1))) 79 | 80 | val keys = vc.keys 81 | 82 | keys shouldBe Set(1,2) 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/justin-db/JustinDB/844a3f6f03192ff3e8248a15712fecd754e06fbc/logo.png -------------------------------------------------------------------------------- /project/Dependencies.scala: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | 3 | object Version { 4 | val scala = "2.12.3" 5 | val scalaBinary = scala.substring(0,4) 6 | 7 | val akka = "2.5.6" 8 | val akkaHttp = "10.0.10" 9 | val akkaClusterManager = "0.5" 10 | val akkaKryo = "0.5.2" 11 | val kryo = "4.0.0" 12 | val scalatest = "3.0.4" 13 | val scalacheck = "1.13.5" 14 | val sigarLoader = "1.6.6" 15 | val logback = "1.2.3" 16 | val scalaLogging = "3.7.2" 17 | val configAnnotation = "0.3.7" 18 | val macroParadise = "2.1.1" 19 | val rocksDB = "5.5.1" 20 | } 21 | 22 | object Library { 23 | val akkaActor = "com.typesafe.akka" %% "akka-actor" % Version.akka 24 | val akkaTestkit = "com.typesafe.akka" %% "akka-testkit" % Version.akka 25 | val akkaRemote = "com.typesafe.akka" %% "akka-remote" % Version.akka 26 | val akkaCluster = "com.typesafe.akka" %% "akka-cluster" % Version.akka 27 | val akkaClusterMetrics = "com.typesafe.akka" %% "akka-cluster-metrics" % Version.akka 28 | val akkaClusterTools = "com.typesafe.akka" %% "akka-cluster-tools" % Version.akka 29 | val akkaStream = "com.typesafe.akka" %% "akka-stream" % Version.akka 30 | val akkaMultiNodeTestkit = "com.typesafe.akka" %% "akka-multi-node-testkit" % Version.akka 31 | val akkaHttp = "com.typesafe.akka" %% "akka-http" % Version.akkaHttp 32 | val akkaHttpSprayJson = "com.typesafe.akka" %% "akka-http-spray-json" % Version.akkaHttp 33 | val akkaHttpTestkit = "com.typesafe.akka" %% "akka-http-testkit" % Version.akkaHttp 34 | val akkaKryo = "com.github.romix.akka" %% "akka-kryo-serialization" % Version.akkaKryo 35 | val akkaClusterManager = "com.lightbend.akka" %% "akka-management-cluster-http" % Version.akkaClusterManager 36 | val kamonSigar = "io.kamon" % "sigar-loader" % Version.sigarLoader 37 | 38 | // test libraries 39 | val scalactic = "org.scalactic" %% "scalactic" % Version.scalatest 40 | val scalatest = "org.scalatest" %% "scalatest" % Version.scalatest 41 | val scalacheck = "org.scalacheck" %% "scalacheck" % Version.scalacheck 42 | 43 | // logging 44 | val akkaSfl4j = "com.typesafe.akka" %% "akka-slf4j" % Version.akka 45 | val logback = "ch.qos.logback" % "logback-classic" % Version.logback 46 | val scalaLogging = "com.typesafe.scala-logging" %% "scala-logging" % Version.scalaLogging 47 | 48 | // config 49 | val configAnnotation = "com.wacai" %% "config-annotation" % Version.configAnnotation 50 | val macroParadise = "org.scalamacros" % "paradise" % Version.macroParadise 51 | 52 | // storage 53 | val rocksdb = "org.rocksdb" % "rocksdbjni" % Version.rocksDB 54 | val kryo = "com.esotericsoftware" % "kryo" % Version.kryo % "provided" 55 | } 56 | 57 | object Dependencies { 58 | import Library._ 59 | 60 | private val genericTest = Seq(scalactic, scalatest % "test") 61 | 62 | private val akkaCommon = Seq(akkaActor, akkaSfl4j, akkaTestkit, akkaKryo, akkaStream) 63 | private val akkaHttpCommon = Seq(akkaHttp, akkaHttpSprayJson, akkaHttpTestkit) 64 | private val akkaClusterCommon = Seq(akkaRemote, akkaMultiNodeTestkit % "multi-jvm", akkaCluster, akkaClusterMetrics, akkaClusterTools, kamonSigar, akkaClusterManager) 65 | 66 | val core = akkaCommon ++ akkaClusterCommon ++ genericTest ++ Seq(scalacheck % "test", logback, scalaLogging) ++ Seq(akkaHttpSprayJson) 67 | val ring = genericTest 68 | val vectorClocks = genericTest 69 | val httpApi = akkaCommon ++ akkaHttpCommon ++ genericTest 70 | 71 | val storageApi = genericTest 72 | val storageInMem = genericTest 73 | val storageLogDBExperimental = genericTest 74 | val storageRocksDB = Seq(rocksdb, rocksdb % "test", kryo) ++ genericTest 75 | 76 | val root = core ++ httpApi ++ storageApi 77 | } 78 | -------------------------------------------------------------------------------- /project/assembly.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.4") 2 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.13 -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers += Classpaths.typesafeResolver 2 | 3 | addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.3.11") 4 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.5.0") 5 | addSbtPlugin("io.spray" % "sbt-revolver" % "0.8.0") 6 | addSbtPlugin("com.timushev.sbt" % "sbt-updates" % "0.2.0") 7 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") 8 | addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.2.2") 9 | addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.8.5") 10 | addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.6.1") 11 | addSbtPlugin("io.get-coursier" % "sbt-coursier" % "1.0.0-RC10") 12 | 13 | libraryDependencies += "org.slf4j" % "slf4j-nop" % "1.7.21" 14 | -------------------------------------------------------------------------------- /project/travis-build.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | set -e 3 | 4 | cd `dirname $0`/.. 5 | 6 | if [ -z "$MAIN_SCALA_VERSION" ]; then 7 | >&2 echo "Environment MAIN_SCALA_VERSION is not set. Check .travis.yml." 8 | exit 1 9 | elif [ -z "$TRAVIS_SCALA_VERSION" ]; then 10 | >&2 echo "Environment TRAVIS_SCALA_VERSION is not set." 11 | exit 1 12 | else 13 | echo 14 | echo "TRAVIS_SCALA_VERSION=$TRAVIS_SCALA_VERSION" 15 | echo "MAIN_SCALA_VERSION=$MAIN_SCALA_VERSION" 16 | fi 17 | 18 | function buildJVM { 19 | INIT="clean" 20 | COMPILE="compile" 21 | COVERAGE="coverage" 22 | TEST="test;multi-jvm:test" 23 | 24 | COMMAND=";$INIT;$COVERAGE;$COMPILE;$TEST" 25 | echo 26 | echo "Executing JVM tests (with coverage): sbt -Dsbt.profile=coverage $COMMAND" 27 | echo 28 | sbt -Dsbt.profile=coverage "$COMMAND" 29 | } 30 | 31 | # Execute everything 32 | buildJVM -------------------------------------------------------------------------------- /project/travis-post-build.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -e 4 | 5 | cd `dirname $0`/.. 6 | 7 | if [ -z "$MAIN_SCALA_VERSION" ]; then 8 | >&2 echo "Environment MAIN_SCALA_VERSION is not set. Check .travis.yml." 9 | exit 1 10 | elif [ -z "$TRAVIS_SCALA_VERSION" ]; then 11 | >&2 echo "Environment TRAVIS_SCALA_VERSION is not set." 12 | exit 1 13 | else 14 | echo "TRAVIS_SCALA_VERSION=$TRAVIS_SCALA_VERSION" 15 | echo "MAIN_SCALA_VERSION=$MAIN_SCALA_VERSION" 16 | fi 17 | 18 | if [ "$TRAVIS_SCALA_VERSION" = "$MAIN_SCALA_VERSION" ]; then 19 | echo "Uploading coverage for Scala $TRAVIS_SCALA_VERSION" 20 | sbt -Dsbt.profile=coverage ";coverageAggregate;coverageReport" 21 | bash <(curl -s https://codecov.io/bash) 22 | else 23 | echo "Skipping uploading coverage for Scala $TRAVIS_SCALA_VERSION" 24 | fi -------------------------------------------------------------------------------- /scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 2 | Scalastyle standard configuration 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | -------------------------------------------------------------------------------- /src/main/resources/application.conf: -------------------------------------------------------------------------------- 1 | include "justin.conf" 2 | 3 | akka { 4 | extensions = [ 5 | "com.romix.akka.serialization.kryo.KryoSerializationExtension$" 6 | ] 7 | 8 | log-config-on-start = true 9 | loglevel = DEBUG 10 | loggers = ["akka.event.slf4j.Slf4jLogger"] 11 | logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" 12 | 13 | actor { 14 | serializers { 15 | kryo = "com.romix.akka.serialization.kryo.KryoSerializer" 16 | } 17 | serialization-bindings { 18 | "justin.db.actors.protocol.RegisterNode" = kryo 19 | 20 | "justin.db.actors.protocol.StorageNodeWriteDataLocal" = kryo 21 | 22 | "justin.db.actors.protocol.StorageNodeSuccessfulWrite" = kryo 23 | "justin.db.actors.protocol.StorageNodeFailedWrite" = kryo 24 | "justin.db.actors.protocol.StorageNodeConflictedWrite" = kryo 25 | 26 | "justin.db.actors.protocol.StorageNodeLocalRead" = kryo 27 | 28 | "justin.db.actors.protocol.StorageNodeFoundRead" = kryo 29 | "justin.db.actors.protocol.StorageNodeConflictedRead" = kryo 30 | "justin.db.actors.protocol.StorageNodeNotFoundRead" = kryo 31 | "justin.db.actors.protocol.StorageNodeFailedRead" = kryo 32 | } 33 | kryo { 34 | type = "nograph" 35 | idstrategy = "explicit" 36 | max-buffer-size = 10485760 # 10MB 37 | use-manifests = false 38 | use-unsafe = false 39 | implicit-registration-logging = true 40 | kryo-trace = false 41 | kryo-custom-serializer-init = "justin.db.kryo.SerializerInit" 42 | } 43 | allow-java-serialization = on 44 | provider = cluster 45 | } 46 | 47 | remote.log-remote-lifecycle-events = off 48 | remote.netty.tcp.hostname = ${justin.netty-tcp-hostname} 49 | remote.netty.tcp.port = ${justin.netty-tcp-port} 50 | 51 | # Disable legacy metrics in akka-cluster. 52 | cluster { 53 | seed-nodes = [] 54 | metrics.enabled = off 55 | role.storagenode.min-nr-of-members = ${justin.ring.members-count} 56 | http.management.hostname = "0.0.0.0" 57 | 58 | multi-data-center { 59 | cross-data-center-connections = ${justin.dc.cross-data-center-connections} 60 | self-data-center = ${justin.dc.self-data-center} 61 | } 62 | 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/resources/justin.conf: -------------------------------------------------------------------------------- 1 | justin { 2 | system = justindb 3 | kubernetes-hostname = justindb-0 4 | ring { 5 | members-count = 1 6 | partitions = 21 7 | } 8 | replication { 9 | N = 1 10 | } 11 | storage { 12 | provider = justin.db.storage.provider.InMemStorageProvider 13 | } 14 | http { 15 | interface = 0.0.0.0 16 | port = 9000 17 | } 18 | netty-tcp-hostname = localhost 19 | netty-tcp-port = 2551 20 | dc { 21 | cross-data-center-connections = 1 22 | self-data-center = dc1 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | %magenta(%d{yyyy-MM-dd HH:mm:ss.SSSX}) %highlight(%-5level) %green(%-35logger{0}): %msg%n 5 | 6 | ./logs/log.%d{yyyy-MM-dd}.log 7 | 30 8 | 9 | 10 | 11 | 12 | 13 | %magenta(%d{yyyy-MM-dd HH:mm:ss.SSSX}) %highlight(%-5level) %green(%-35logger{0}): %msg%n 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | -------------------------------------------------------------------------------- /src/main/scala/justin/db/JustinDB.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import akka.actor.ActorSystem 4 | import akka.cluster.Cluster 5 | import akka.cluster.http.management.ClusterHttpManagement 6 | import akka.http.scaladsl.Http 7 | import akka.http.scaladsl.server.Directives._ 8 | import akka.stream.{ActorMaterializer, Materializer} 9 | import buildinfo.BuildInfo 10 | import com.typesafe.scalalogging.StrictLogging 11 | import justin.db.actors.{StorageNodeActor, StorageNodeActorRef} 12 | import justin.db.client.ActorRefStorageNodeClient 13 | import justin.db.cluster.datacenter.Datacenter 14 | import justin.db.consistenthashing.{NodeId, Ring} 15 | import justin.db.replica.N 16 | import justin.db.storage.PluggableStorageProtocol 17 | import justin.db.storage.provider.StorageProvider 18 | import justin.httpapi.{BuildInfoRouter, HealthCheckRouter, HttpRouter} 19 | 20 | import scala.concurrent.duration._ 21 | import scala.concurrent.{Await, ExecutionContext, Promise} 22 | import scala.language.reflectiveCalls 23 | 24 | // $COVERAGE-OFF$ 25 | final class JustinDB 26 | 27 | object JustinDB extends StrictLogging { 28 | 29 | private[this] def validConfiguration(justinDBConfig: JustinDBConfig): Unit = { 30 | require(justinDBConfig.replication.N > 0, "replication N factor can't be smaller or equal 0") 31 | require(justinDBConfig.ring.`members-count` > 0, "members-counter can't be smaller or equal 0") 32 | require(justinDBConfig.ring.partitions > 0, "ring partitions can't be smaller or equal 0") 33 | require(justinDBConfig.ring.partitions >= justinDBConfig.ring.`members-count`, "number of ring partitions can't be smaller than number of members-count") 34 | require(justinDBConfig.replication.N <= justinDBConfig.ring.`members-count`, "replication N factor can't be bigger than defined members-count number") 35 | } 36 | 37 | private[this] def initStorage(justinConfig: JustinDBConfig) = { 38 | val provider = StorageProvider.apply(justinConfig.storage.provider) 39 | logger.info("Storage provider: " + provider.name) 40 | provider.init 41 | } 42 | 43 | def init(justinConfig: JustinDBConfig)(implicit actorSystem: ActorSystem): JustinDB = { 44 | validConfiguration(justinConfig) 45 | 46 | val processOrchestrator = Promise[JustinDB] 47 | 48 | implicit val executor: ExecutionContext = actorSystem.dispatcher 49 | implicit val materializer: Materializer = ActorMaterializer() 50 | 51 | val storage: PluggableStorageProtocol = initStorage(justinConfig) 52 | 53 | val cluster = Cluster(actorSystem) 54 | 55 | cluster.registerOnMemberUp { 56 | // STORAGE ACTOR 57 | val storageNodeActorRef = StorageNodeActorRef { 58 | val nodeId = NodeId(justinConfig.`kubernetes-hostname`.split("-").last.toInt) 59 | val ring = Ring(justinConfig.ring.`members-count`, justinConfig.ring.partitions) 60 | val n = N(justinConfig.replication.N) 61 | val datacenter = Datacenter(justinConfig.dc.`self-data-center`) 62 | 63 | actorSystem.actorOf( 64 | props = StorageNodeActor.props(nodeId, datacenter, storage, ring, n), 65 | name = StorageNodeActor.name(nodeId, datacenter) 66 | ) 67 | } 68 | 69 | // AKKA-MANAGEMENT 70 | ClusterHttpManagement(cluster).start().map { _ => 71 | logger.info("Cluster HTTP-Management is ready!") 72 | }.recover { case ex => processOrchestrator.failure(ex) } 73 | 74 | // HTTP API 75 | val routes = logRequestResult(actorSystem.name) { 76 | new HttpRouter(new ActorRefStorageNodeClient(storageNodeActorRef)).routes ~ 77 | new HealthCheckRouter().routes ~ 78 | new BuildInfoRouter().routes(BuildInfo.toJson) 79 | } 80 | Http() 81 | .bindAndHandle(routes, justinConfig.http.interface, justinConfig.http.port) 82 | .map { binding => logger.info(s"HTTP server started at ${binding.localAddress}"); processOrchestrator.trySuccess(new JustinDB) } 83 | .recover { case ex => logger.error("Could not start HTTP server", ex); processOrchestrator.failure(ex) } 84 | } 85 | 86 | Await.result(processOrchestrator.future, 2.minutes) 87 | } 88 | } 89 | // $COVERAGE-ON$ 90 | -------------------------------------------------------------------------------- /src/main/scala/justin/db/Main.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import akka.actor.ActorSystem 4 | import buildinfo.BuildInfo 5 | import com.typesafe.scalalogging.StrictLogging 6 | 7 | // $COVERAGE-OFF$ 8 | object Main extends App with StrictLogging { 9 | 10 | logger.info( 11 | """ 12 | | ___ _ _ ______ ______ 13 | | |_ | | | (_) | _ \| ___ \ 14 | | | | _ _ ___ | |_ _ _ __ | | | || |_/ / 15 | | | || | | |/ __|| __|| || '_ \ | | | || ___ \ 16 | |/\__/ /| |_| |\__ \| |_ | || | | || |/ / | |_/ / 17 | |\____/ \__,_||___/ \__||_||_| |_||___/ \____/ 18 | | 19 | """.stripMargin 20 | ) 21 | 22 | val justindbConfig = JustinDBConfig.init 23 | val actorSystem = ActorSystem(justindbConfig.system, justindbConfig.config) 24 | val justindb = JustinDB.init(justindbConfig)(actorSystem) 25 | 26 | logger.info("Build Info: " + BuildInfo.toString) 27 | } 28 | // $COVERAGE-ON$ 29 | -------------------------------------------------------------------------------- /src/main/scala/justin/db/justin.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import com.typesafe.config.{Config, ConfigFactory} 4 | import com.wacai.config.annotation._ 5 | import justin.db.actors.StorageNodeActor 6 | 7 | // $COVERAGE-OFF$ 8 | @conf 9 | trait justin extends Configurable { 10 | 11 | val system: String = "justindb" 12 | 13 | val `kubernetes-hostname`: String = "justindb-0" 14 | 15 | val ring = new { 16 | val `members-count`: Int = 1 17 | val partitions: Int = 21 18 | } 19 | 20 | val replication = new { 21 | val N: Int = 1 22 | } 23 | 24 | val storage = new { 25 | val provider = "justin.db.storage.provider.InMemStorageProvider" 26 | } 27 | 28 | val http = new { 29 | val interface: String = "0.0.0.0" 30 | val port: Int = 9000 31 | } 32 | 33 | val `netty-tcp-hostname`: String = "localhost" 34 | val `netty-tcp-port`: Int = 2551 35 | 36 | val dc = new { 37 | val `cross-data-center-connections`: Int = 1 38 | val `self-data-center` = "dc1" 39 | } 40 | } 41 | 42 | class JustinDBConfig(val config: Config) extends justin 43 | 44 | object JustinDBConfig { 45 | 46 | def init: JustinDBConfig = new JustinDBConfig(ConfigFactory 47 | .parseString(s"akka.cluster.roles = [${StorageNodeActor.role}]") 48 | .withFallback(ConfigFactory.load())) 49 | 50 | } 51 | // $COVERAGE-ON$ 52 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/justin/db/ConvergeJustinDBClusterSpec.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import akka.remote.testkit.{MultiNodeConfig, MultiNodeSpec} 4 | import com.typesafe.config.ConfigFactory 5 | 6 | final class ConvergeJustinDBClusterConfig extends MultiNodeConfig { 7 | val first = role("first") 8 | val second = role("second") 9 | val third = role("third") 10 | 11 | private[this] val allRoles = List(first, second, third) 12 | private[this] val clusterName = "ConvergeJustinDBClusterSpec" 13 | 14 | private[this] def commonNodeConfig(id: Int) = ConfigFactory.parseString( 15 | s""" 16 | |justin.system = $clusterName 17 | |justin.kubernetes-hostname = s"justindb-$id" 18 | |justin.http.port = ${9000 + id} 19 | |akka.cluster.role.storagenode.min-nr-of-members = ${allRoles.size} 20 | |akka.cluster.http.management.port = ${19999 + id} 21 | |akka.cluster.seed-nodes.0 = "akka.tcp://$clusterName@localhost:25551" 22 | |akka.remote.netty.tcp.port = ${25551 + id} 23 | |akka.remote.netty.tcp.hostname = "localhost" 24 | """.stripMargin 25 | ) 26 | 27 | commonConfig(MultiNodeClusterSpec.commonBaseConfig.withFallback(JustinDBConfig.init.config)) 28 | 29 | allRoles.zipWithIndex.foreach { case (roleName, id) => 30 | nodeConfig(roleName)(commonNodeConfig(id)) 31 | } 32 | } 33 | 34 | final class ConvergeJustinDBClusterSpecMultiJvmNode1 extends ConvergeJustinDBClusterSpec 35 | final class ConvergeJustinDBClusterSpecMultiJvmNode2 extends ConvergeJustinDBClusterSpec 36 | final class ConvergeJustinDBClusterSpecMultiJvmNode3 extends ConvergeJustinDBClusterSpec 37 | 38 | abstract class ConvergeJustinDBClusterSpec(config: ConvergeJustinDBClusterConfig) 39 | extends MultiNodeSpec(config) 40 | with MultiNodeClusterSpec { 41 | 42 | def this() = this(new ConvergeJustinDBClusterConfig()) 43 | 44 | "A cluster" must { 45 | "be able to form" in { 46 | val config = new JustinDBConfig(system.settings.config) 47 | val justinDB = JustinDB.init(config)(system) 48 | 49 | enterBarrier("justindb-cluster-up") 50 | } 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/justin/db/MultiDcSpec.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import akka.remote.testkit.{MultiNodeConfig, MultiNodeSpec} 4 | import com.typesafe.config.ConfigFactory 5 | 6 | final class MultiDcSpecConfig(crossDcConnections: Int = 1) extends MultiNodeConfig { 7 | val first = role("first") 8 | val second = role("second") 9 | 10 | private[this] val allRoles = List(first, second) 11 | private[this] val clusterName = "MultiDcSpec" 12 | 13 | private[this] def commonNodeConfig(id: Int) = ConfigFactory.parseString( 14 | s""" 15 | |justin.system = $clusterName 16 | |justin.kubernetes-hostname = s"justindb-$id" 17 | |justin.http.port = ${9000 + id} 18 | |akka.cluster.role.storagenode.min-nr-of-members = ${allRoles.size} 19 | |akka.cluster.http.management.port = ${19999 + id} 20 | |akka.cluster.seed-nodes.0 = "akka.tcp://$clusterName@localhost:25551" 21 | |akka.remote.netty.tcp.port = ${25551 + id} 22 | |akka.remote.netty.tcp.hostname = "localhost" 23 | |akka.cluster.multi-data-center.cross-data-center-connections = $crossDcConnections 24 | |akka.cluster.multi-data-center.self-data-center = "dc$id" 25 | """.stripMargin 26 | ) 27 | 28 | commonConfig(MultiNodeClusterSpec.commonBaseConfig.withFallback(JustinDBConfig.init.config)) 29 | 30 | allRoles.zipWithIndex.foreach { case (roleName, id) => 31 | nodeConfig(roleName)(commonNodeConfig(id)) 32 | } 33 | } 34 | 35 | final class MultiDcMultiJvm1 extends MultiDcSpec 36 | final class MultiDcMultiJvm2 extends MultiDcSpec 37 | 38 | abstract class MultiDcSpec(config: MultiDcSpecConfig) 39 | extends MultiNodeSpec(config) 40 | with MultiNodeClusterSpec { 41 | 42 | def this() = this(new MultiDcSpecConfig()) 43 | 44 | "A cluster with multiple data centers" must { 45 | "be able to form" in { 46 | val config = new JustinDBConfig(system.settings.config) 47 | val justinDB = JustinDB.init(config)(system) 48 | 49 | enterBarrier("justindb-cluster-up") 50 | } 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/justin/db/MultiNodeClusterSpec.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import akka.cluster.Cluster 4 | import akka.remote.testkit.MultiNodeSpec 5 | import com.typesafe.config.{Config, ConfigFactory} 6 | import org.scalatest.Suite 7 | 8 | object MultiNodeClusterSpec { 9 | 10 | val commonBaseConfig: Config = ConfigFactory.parseString( 11 | s""" 12 | |akka.loglevel = INFO 13 | |akka.log-config-on-start = false 14 | |akka.log-dead-letters = off 15 | |akka.log-dead-letters-during-shutdown = off 16 | |akka.remote.log-remote-lifecycle-events = off 17 | """.stripMargin 18 | ) 19 | } 20 | 21 | trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeSpec ⇒ 22 | 23 | /** 24 | * Get the cluster node to use. 25 | */ 26 | def cluster: Cluster = Cluster(system) 27 | 28 | def initialParticipants: Int = roles.size 29 | } 30 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/justin/db/STMultiNodeSpec.scala: -------------------------------------------------------------------------------- 1 | package justin.db 2 | 3 | import akka.remote.testkit.MultiNodeSpecCallbacks 4 | import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} 5 | 6 | trait STMultiNodeSpec extends MultiNodeSpecCallbacks 7 | with WordSpecLike 8 | with Matchers 9 | with BeforeAndAfterAll { 10 | 11 | override def beforeAll(): Unit = multiNodeSpecBeforeAll() 12 | override def afterAll(): Unit = multiNodeSpecAfterAll() 13 | } 14 | -------------------------------------------------------------------------------- /src/universal/cli: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/justin-db/JustinDB/844a3f6f03192ff3e8248a15712fecd754e06fbc/src/universal/cli -------------------------------------------------------------------------------- /src/universal/motd: -------------------------------------------------------------------------------- 1 | 2 | ___ _ _ ______ ______ 3 | |_ | | | (_) | _ \| ___ \ 4 | | | _ _ ___ | |_ _ _ __ | | | || |_/ / 5 | | || | | |/ __|| __|| || '_ \ | | | || ___ \ 6 | /\__/ /| |_| |\__ \| |_ | || | | || |/ / | |_/ / 7 | \____/ \__,_||___/ \__||_||_| |_||___/ \____/ 8 | 9 | Cli: 10 | cli help 11 | More documentation available at: 12 | https://github.com/speedcom/JustinDB 13 | https://speedcom.github.io/ 14 | 15 | --------------------------------------------------------------------------------