├── project ├── build.properties ├── protoc.sbt ├── plugins.sbt └── Dependencies.scala ├── version.sbt ├── .travis ├── pubring.gpg.enc ├── secring.gpg.enc ├── docs-requirements.txt ├── docs.sh ├── release.sh ├── settings.xml └── gpg.sh ├── docs ├── images │ ├── internals │ │ ├── ch_ring.png │ │ └── aggregation_viz.png │ └── developer │ │ └── release_workflow.png ├── Release_0.3.md ├── developer │ └── workflow.md ├── quick-start.md ├── internals │ ├── partitioner.md │ ├── replication.md │ ├── router.md │ └── aggregation.md ├── Release_0.2.md ├── Release_0.1.md ├── recipes │ ├── kvclient.md │ ├── rocksdb.md │ └── inmemorydb.md └── index.md ├── .scalafmt.conf ├── suuchi-core ├── src │ ├── test │ │ ├── resources │ │ │ └── META-INF │ │ │ │ └── services │ │ │ │ └── in.ashwanthkumar.suuchi.cluster.ClusterProvider │ │ ├── protobuf │ │ │ └── test.proto │ │ └── scala │ │ │ └── in │ │ │ └── ashwanthkumar │ │ │ └── suuchi │ │ │ ├── router │ │ │ ├── BooleanMarshallerSpec.scala │ │ │ ├── StringMarshallerSpec.scala │ │ │ ├── MemberAddressMarshallerSpec.scala │ │ │ ├── RoutingStrategySpec.scala │ │ │ ├── ListOfMemberAddressMarshallerSpec.scala │ │ │ ├── ParallelReplicatorSpec.scala │ │ │ ├── SequentialReplicatorSpec.scala │ │ │ ├── HandleOrForwardRouterSpec.scala │ │ │ ├── ReplicationRouterSpec.scala │ │ │ └── AggregationRouterSpec.scala │ │ │ ├── store │ │ │ ├── VersionsSpec.scala │ │ │ ├── PrimitivesSerDeUtilsSpec.scala │ │ │ ├── InMemoryStoreTest.scala │ │ │ ├── ShardedStoreSpec.scala │ │ │ └── VersionedStoreSpec.scala │ │ │ ├── cluster │ │ │ ├── TestStaticCluster.scala │ │ │ └── ClusterProviderSpec.scala │ │ │ ├── partitioner │ │ │ ├── ConsistentHashPartitionerTest.scala │ │ │ ├── RingStateSpec.scala │ │ │ └── ConsistentHashRingSpec.scala │ │ │ ├── utils │ │ │ └── ByteArrayUtilsSpec.scala │ │ │ └── rpc │ │ │ └── CachedChannelPoolSpec.scala │ └── main │ │ └── scala │ │ └── in │ │ └── ashwanthkumar │ │ └── suuchi │ │ ├── utils │ │ ├── DateUtils.scala │ │ ├── Logging.scala │ │ └── ByteArrayUtils.scala │ │ ├── cluster │ │ ├── MemberAddress.scala │ │ ├── SeedProvider.scala │ │ ├── ClusterProvider.scala │ │ └── Cluster.scala │ │ ├── store │ │ ├── PrimitivesSerDeUtils.scala │ │ ├── Store.scala │ │ ├── VersionedBy.scala │ │ ├── StoreUtils.scala │ │ ├── InMemoryStore.scala │ │ ├── ShardedStore.scala │ │ └── VersionedStore.scala │ │ ├── partitioner │ │ ├── Partitioner.scala │ │ └── ConsistentHashRing.scala │ │ ├── rpc │ │ └── CachedChannelPool.scala │ │ └── router │ │ ├── Headers.scala │ │ ├── Marshallers.scala │ │ ├── RoutingStrategy.scala │ │ ├── AggregationRouter.scala │ │ └── HandleOrForwardRouter.scala └── _pom.xml ├── suuchi-cluster-atomix ├── src │ ├── main │ │ ├── resources │ │ │ └── META-INF │ │ │ │ └── services │ │ │ │ └── in.ashwanthkumar.suuchi.cluster.ClusterProvider │ │ └── scala │ │ │ └── in │ │ │ └── ashwanthkumar │ │ │ └── suuchi │ │ │ └── cluster │ │ │ └── atomix │ │ │ ├── AtomixClusterProvider.scala │ │ │ └── AtomixCluster.scala │ └── test │ │ └── scala │ │ └── in │ │ └── ashwanthkumar │ │ └── suuchi │ │ └── cluster │ │ └── atomix │ │ └── AtomixClusterSpec.scala ├── README.md └── _pom.xml ├── suuchi-cluster-scalecube ├── src │ ├── main │ │ ├── resources │ │ │ └── META-INF │ │ │ │ └── services │ │ │ │ └── in.ashwanthkumar.suuchi.cluster.ClusterProvider │ │ └── scala │ │ │ └── in │ │ │ └── ashwanthkumar │ │ │ └── suuchi │ │ │ └── cluster │ │ │ └── scalecube │ │ │ ├── ScaleCubeProvider.scala │ │ │ └── ScaleCubeCluster.scala │ └── test │ │ └── scala │ │ └── in │ │ └── ashwanthkumar │ │ └── suuchi │ │ └── cluster │ │ └── scalecube │ │ ├── ScaleCubeConfigTest.scala │ │ └── ScaleCubeClusterIT.scala ├── README.md └── _pom.xml ├── .gitignore ├── sonatype.sbt ├── make-release.sh ├── .codecov.yml ├── suuchi-examples ├── src │ ├── main │ │ ├── scala │ │ │ └── in │ │ │ │ └── ashwanthkumar │ │ │ │ └── suuchi │ │ │ │ ├── rpc │ │ │ │ ├── PingService.scala │ │ │ │ ├── SuuchiPutService.scala │ │ │ │ ├── SuuchiReadService.scala │ │ │ │ ├── SuuchiShardService.scala │ │ │ │ ├── SuuchiAggregatorService.scala │ │ │ │ └── SuuchiScanService.scala │ │ │ │ ├── example │ │ │ │ ├── DistributedRocksDb.scala │ │ │ │ └── DistributedKVServer.scala │ │ │ │ └── client │ │ │ │ └── SuuchiClient.scala │ │ ├── resources │ │ │ └── log4j2.xml │ │ └── protobuf │ │ │ └── example.proto │ └── test │ │ └── scala │ │ └── in │ │ └── ashwanthkumar │ │ └── suuchi │ │ └── rpc │ │ ├── PingServiceTest.scala │ │ ├── SuuchiPutServiceTest.scala │ │ ├── SuuchiShardServiceTest.scala │ │ ├── SuuchiReadServiceTest.scala │ │ └── SuuchiScanServiceTest.scala └── _pom.xml ├── mkdocs.yml ├── suuchi-rocksdb ├── _pom.xml └── src │ ├── main │ └── scala │ │ └── in │ │ └── ashwanthkumar │ │ └── suuchi │ │ └── store │ │ └── rocksdb │ │ ├── RocksDbConfiguration.scala │ │ └── RocksDbStore.scala │ └── test │ └── scala │ └── in │ └── ashwanthkumar │ └── suuchi │ └── store │ └── rocksdb │ └── RocksDbStoreSpec.scala ├── .travis.yml └── README.md /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 0.13.11 2 | -------------------------------------------------------------------------------- /version.sbt: -------------------------------------------------------------------------------- 1 | version in ThisBuild := "0.4.0-SNAPSHOT" 2 | -------------------------------------------------------------------------------- /.travis/pubring.gpg.enc: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ashwanthkumar/suuchi/HEAD/.travis/pubring.gpg.enc -------------------------------------------------------------------------------- /.travis/secring.gpg.enc: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ashwanthkumar/suuchi/HEAD/.travis/secring.gpg.enc -------------------------------------------------------------------------------- /docs/images/internals/ch_ring.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ashwanthkumar/suuchi/HEAD/docs/images/internals/ch_ring.png -------------------------------------------------------------------------------- /docs/images/internals/aggregation_viz.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ashwanthkumar/suuchi/HEAD/docs/images/internals/aggregation_viz.png -------------------------------------------------------------------------------- /docs/images/developer/release_workflow.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ashwanthkumar/suuchi/HEAD/docs/images/developer/release_workflow.png -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | style = defaultWithAlign # For pretty alignment. 2 | maxColumn = 100 # For my wide 30" display. 3 | docstrings = JavaDoc 4 | -------------------------------------------------------------------------------- /suuchi-core/src/test/resources/META-INF/services/in.ashwanthkumar.suuchi.cluster.ClusterProvider: -------------------------------------------------------------------------------- 1 | in.ashwanthkumar.suuchi.cluster.TestStaticClusterProvider -------------------------------------------------------------------------------- /project/protoc.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.thesamet" % "sbt-protoc" % "0.99.12") 2 | 3 | libraryDependencies += "com.trueaccord.scalapb" %% "compilerplugin" % "0.6.6" 4 | -------------------------------------------------------------------------------- /suuchi-cluster-atomix/src/main/resources/META-INF/services/in.ashwanthkumar.suuchi.cluster.ClusterProvider: -------------------------------------------------------------------------------- 1 | in.ashwanthkumar.suuchi.cluster.atomix.AtomixClusterProvider -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/src/main/resources/META-INF/services/in.ashwanthkumar.suuchi.cluster.ClusterProvider: -------------------------------------------------------------------------------- 1 | in.ashwanthkumar.suuchi.cluster.scalecube.ScaleCubeProvider -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | *.iml 3 | target/ 4 | *.ipr 5 | *.iws 6 | site/ 7 | 8 | # mvn versions:set 9 | pom.xml.versionsBackup 10 | 11 | # Mac specific 12 | .DS_Store -------------------------------------------------------------------------------- /sonatype.sbt: -------------------------------------------------------------------------------- 1 | credentials += Credentials("Sonatype Nexus Repository Manager", 2 | "oss.sonatype.org", 3 | System.getenv("SONATYPE_USERNAME"), 4 | System.getenv("SONATYPE_PASSWORD")) 5 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/utils/DateUtils.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.utils 2 | 3 | import org.joda.time.DateTime 4 | 5 | trait DateUtils { 6 | def now = DateTime.now().getMillis 7 | } 8 | -------------------------------------------------------------------------------- /make-release.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | git stash --all 4 | # We create a special type of commit message to trigger a release workflow in travis-ci 5 | git commit --allow-empty -m "[Do Release]" 6 | git push 7 | git stash pop 8 | -------------------------------------------------------------------------------- /.codecov.yml: -------------------------------------------------------------------------------- 1 | comment: 2 | layout: "header, diff, changes, sunburst" 3 | behavior: default 4 | require_changes: false # if true: only post the comment if coverage changes 5 | branches: null 6 | flags: null 7 | paths: null 8 | 9 | ignore: 10 | - suuchi-examples -------------------------------------------------------------------------------- /.travis/docs-requirements.txt: -------------------------------------------------------------------------------- 1 | Jinja2==2.11.3 2 | Markdown==2.6.10 3 | MarkupSafe==1.0 4 | backports_abc==0.5 5 | backports.ssl_match_hostname==3.5.0.1 6 | certifi==2017.11.5 7 | click==6.7 8 | livereload==2.5.1 9 | mkdocs==0.15.3 10 | mkdocs-bootstrap==0.1.1 11 | mkdocs-bootswatch==0.4.0 12 | singledispatch==3.4.0.3 13 | tornado==4.5.2 14 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.github.sbt" % "sbt-jacoco" % "3.0.3") 2 | 3 | addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.5") 4 | 5 | addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.7.0") 6 | 7 | addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.6") 8 | 9 | addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "1.1") 10 | 11 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") 12 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/rpc/PingService.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{PingRequest, PingResponse, PingServiceGrpc} 4 | 5 | import scala.concurrent.Future 6 | 7 | class PingService extends PingServiceGrpc.PingService { 8 | override def ping(request: PingRequest) = Future.successful(PingResponse(status = true)) 9 | } 10 | -------------------------------------------------------------------------------- /docs/Release_0.3.md: -------------------------------------------------------------------------------- 1 | # Suuchi Release 0.3 2 | ## Bug Fixes 3 | - [x] Optimize scans in VersionedStore - #72 4 | - [x] Fix the VRecord.key in VersionedStore - #73 5 | 6 | List of all the issues and features 7 | - https://github.com/ashwanthkumar/suuchi/pulls?q=is%3Apr+milestone%3A0.3 8 | - https://github.com/ashwanthkumar/suuchi/issues?utf8=%E2%9C%93&q=is%3Aissue%20milestone%3A0.3%20 9 | 10 | ## Contributors 11 | - [Ashwanth Kumar](https://github.com/ashwanthkumar) 12 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/utils/Logging.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.utils 2 | 3 | import org.slf4j.LoggerFactory 4 | 5 | import scala.util.Try 6 | 7 | trait Logging { self => 8 | val log = LoggerFactory.getLogger(this.getClass) 9 | 10 | def logOnError[T](f: () => T): Try[T] = { 11 | Try { 12 | f() 13 | } recover { 14 | case e: Exception => 15 | log.error(e.getMessage, e) 16 | throw e 17 | } 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /suuchi-core/src/test/protobuf/test.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | option java_package = "in.ashwanthkumar.suuchi.core.tests"; 4 | option java_outer_classname = "SuuchiTestRPC"; 5 | 6 | message ReduceRequest { 7 | } 8 | 9 | message ReduceResponse { 10 | int64 output = 1; 11 | } 12 | 13 | service Aggregator { 14 | rpc Reduce (ReduceRequest) returns (ReduceResponse); 15 | } 16 | 17 | message FooRequest { 18 | } 19 | message FooResponse { 20 | } 21 | 22 | service Random { 23 | rpc Foo (FooRequest) returns (FooResponse); 24 | } 25 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/resources/log4j2.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/BooleanMarshallerSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import org.scalatest.FlatSpec 4 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 5 | 6 | class BooleanMarshallerSpec extends FlatSpec { 7 | "BooleanMarshaller" should "return the bool as string when serialised" in { 8 | BooleanMarshaller.toAsciiString(true) should be("true") 9 | } 10 | 11 | it should "return the bool when de-serialised" in { 12 | BooleanMarshaller.parseAsciiString("true") should be(true) 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/StringMarshallerSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import org.scalatest.FlatSpec 4 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 5 | 6 | class StringMarshallerSpec extends FlatSpec { 7 | "StringMarshaller" should "return the string as is when serialised" in { 8 | StringMarshaller.toAsciiString("suuchi") should be("suuchi") 9 | } 10 | 11 | it should "return the string as is when de-serialised" in { 12 | StringMarshaller.parseAsciiString("suuchi") should be("suuchi") 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/cluster/MemberAddress.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster 2 | 3 | case class MemberAddress(host: String, port: Int) { 4 | def toExternalForm = s"$host:$port" 5 | } 6 | 7 | object MemberAddress { 8 | 9 | /** 10 | * Constructs a MemberAddress from host:port string format 11 | * 12 | * @param hostPort Host:Port format of a node address 13 | * @return MemberAddress 14 | */ 15 | def apply(hostPort: String): MemberAddress = { 16 | val parts = hostPort.split(":") 17 | MemberAddress(parts(0), parts(1).toInt) 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/utils/ByteArrayUtils.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.utils 2 | 3 | import java.util.{Arrays => JArrays} 4 | 5 | import in.ashwanthkumar.suuchi.partitioner.Hash 6 | 7 | object ByteArrayUtils { 8 | def hasPrefix(bytes: Array[Byte], prefix: Array[Byte]): Boolean = { 9 | bytes.length >= prefix.length && JArrays.equals(bytes.take(prefix.length), prefix) 10 | } 11 | 12 | def isHashKeyWithinRange(start: Int, end: Int, key: Array[Byte], hashFn: Hash) = { 13 | val hash = hashFn.hash(key) 14 | 15 | start <= hash && hash <= end 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/PrimitivesSerDeUtils.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.nio.ByteBuffer 4 | 5 | object PrimitivesSerDeUtils { 6 | /* 7 | * FIXME: Not the most effective way to perform serde primitives. 8 | * */ 9 | def longToBytes(instance: Long) = ByteBuffer.allocate(8).putLong(instance).array() 10 | def intToBytes(instance: Int) = ByteBuffer.allocate(4).putInt(instance).array() 11 | def bytesToInt(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt 12 | def bytesToLong(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getLong 13 | } 14 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/store/VersionsSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import org.scalatest.FlatSpec 4 | import org.scalatest.Matchers.{convertToAnyShouldWrapper, be} 5 | 6 | import scala.util.Random 7 | 8 | class VersionsSpec extends FlatSpec { 9 | "Versions" should "do List[Long] SerDe properly" in { 10 | val versionTs = Random.nextLong() 11 | val writtenTs = Random.nextLong() 12 | 13 | val serialised = Versions.toBytes(List(Version(versionTs, writtenTs))) 14 | Versions.fromBytes(serialised) should be(List(Version(versionTs, writtenTs))) 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/rpc/SuuchiPutService.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{PutGrpc, PutRequest, PutResponse} 4 | import in.ashwanthkumar.suuchi.store.WriteStore 5 | 6 | import scala.concurrent.Future 7 | 8 | class SuuchiPutService(store: WriteStore) extends PutGrpc.Put { 9 | override def put(request: PutRequest) = Future.successful { 10 | val key = request.key.toByteArray 11 | val value = request.key.toByteArray 12 | 13 | val status = store.put(key, value) 14 | PutResponse(status = status) 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /docs/developer/workflow.md: -------------------------------------------------------------------------------- 1 | # Release Workflow 2 | In Suuchi and it's related modules we use the following mechanism of doing releases to sonatype. 3 | 4 | ## Steps to make a release 5 | 1. Make sure you've write access to the repository. 6 | 2. Run the `make-release.sh` from the root of the project. 7 | 3. It would create an empty commit with the message `"[Do Release]"`. 8 | 4. This commit message would trigger the release workflow using the build tool to build and publish the artifacts to sonatype, which later would get mirrored to maven central. 9 | 10 | ## Release Process 11 | 12 | ![Release Workflow](/images/developer/release_workflow.png) 13 | -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/src/main/scala/in/ashwanthkumar/suuchi/cluster/scalecube/ScaleCubeProvider.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.scalecube 2 | 3 | import com.typesafe.config.Config 4 | import in.ashwanthkumar.suuchi.cluster._ 5 | 6 | class ScaleCubeProvider extends ClusterProvider { 7 | 8 | /** 9 | * @inheritdoc 10 | */ 11 | override def createCluster(self: MemberAddress, 12 | config: Config, 13 | listeners: List[MemberListener]): Cluster = { 14 | new ScaleCubeCluster(config, listeners = listeners) 15 | } 16 | 17 | /** 18 | * @inheritdoc 19 | */ 20 | override def priority: Int = 4 21 | } 22 | -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/README.md: -------------------------------------------------------------------------------- 1 | # [ScaleCube](http://scalecube.io/) based Suuchi Cluster 2 | 3 | This module provides Cluster implementation using ScaleCube library. Use this library if you want to use Gossip style cluster member management. 4 | 5 | **Note** - This module needs Java8 to run. 6 | 7 | ## Configuration 8 | ``` 9 | ... 10 | cluster { 11 | scalecube { 12 | port = 9090 # port used by scalecube for cluster membership communication 13 | # Gossip protocol related settings 14 | gossip { 15 | fanout = 5 16 | interval = 3000 17 | } 18 | } 19 | } 20 | ... 21 | ``` 22 | 23 | ## License 24 | https://www.apache.org/licenses/LICENSE-2.0 25 | -------------------------------------------------------------------------------- /.travis/docs.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # Deploy Docs only for builds out of master and not PRs or tags. 4 | if ([ "$TRAVIS_BRANCH" == "master" ] || [ ! -z "$TRAVIS_TAG" ]) && 5 | [ "$TRAVIS_PULL_REQUEST" == "false" ]; then 6 | git config user.name "Ashwanth Kumar" 7 | git config user.email "ashwanthkumar@googlemail.com" 8 | git remote add gh-token "https://${GH_TOKEN}@github.com/ashwanthkumar/suuchi.git"; 9 | git fetch gh-token && git fetch gh-token gh-pages:gh-pages; 10 | 11 | virtualenv --system-site-packages ${HOME}/DENV 12 | source ${HOME}/DENV/bin/activate 13 | pip install -r .travis/docs-requirements.txt 14 | mkdocs gh-deploy -v --clean --remote-name gh-token; 15 | fi 16 | -------------------------------------------------------------------------------- /mkdocs.yml: -------------------------------------------------------------------------------- 1 | site_name: Suuchi 2 | repo_url: https://github.com/ashwanthkumar/suuchi/ 3 | site_description: Toolkit to build data systems 4 | theme: flatly 5 | 6 | pages: 7 | - 'Introduction': 'index.md' 8 | - 'Quick Start': 'quick-start.md' 9 | - Internals: 10 | - 'Aggregation': 'internals/aggregation.md' 11 | - 'Partitioner': 'internals/partitioner.md' 12 | - 'Replication': 'internals/replication.md' 13 | - 'Router': 'internals/router.md' 14 | - Recipes: 15 | - 'Distributed In Memory Database': 'recipes/inmemorydb.md' 16 | - 'Distributed RocksDB Database': 'recipes/rocksdb.md' 17 | - 'Distributed KVClient': 'recipes/kvclient.md' 18 | - Developer: 19 | - 'Release Workflow': 'developer/workflow.md' 20 | -------------------------------------------------------------------------------- /.travis/release.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | if ([ "$TRAVIS_COMMIT_MESSAGE" == "[Do Release]" ] && [ "$TRAVIS_PULL_REQUEST" == "false" ]); 4 | then 5 | echo "Triggering a versioned release of the project" 6 | echo "Attempting to publish signed jars" 7 | sbt +publishSigned 8 | echo "Published the signed jars" 9 | echo "Attempting to make a release of the sonatype staging" 10 | sbt sonatypeRelease 11 | echo "Released the sonatype staging setup" 12 | sbt release with-defaults 13 | echo "Versioned release of the project is now complete" 14 | else 15 | echo "Triggering a SNAPSHOT release of the project" 16 | sbt +publish 17 | echo "SNAPSHOT release of the project is now complete" 18 | fi 19 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/rpc/SuuchiReadService.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import com.google.protobuf.ByteString 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{GetRequest, GetResponse, ReadGrpc} 5 | import in.ashwanthkumar.suuchi.store.ReadStore 6 | 7 | import scala.concurrent.Future 8 | 9 | class SuuchiReadService(store: ReadStore) extends ReadGrpc.Read { 10 | 11 | override def get(request: GetRequest) = Future.successful { 12 | val key = request.key.toByteArray 13 | store.get(key) match { 14 | case Some(value) => 15 | GetResponse(key = ByteString.copyFrom(key), value = ByteString.copyFrom(value)) 16 | case None => 17 | GetResponse(key = ByteString.copyFrom(key)) 18 | } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/MemberAddressMarshallerSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import org.scalatest.FlatSpec 5 | import org.scalatest.Matchers.{be, contain, convertToAnyShouldWrapper, have} 6 | 7 | class MemberAddressMarshallerSpec extends FlatSpec { 8 | "MemberAddressMarshaller" should "convert a member to ascii string" in { 9 | MemberAddressMarshaller.toAsciiString(MemberAddress("localhost", 5051)) should be( 10 | "localhost:5051") 11 | } 12 | 13 | it should "convert the ascii string to actual member address" in { 14 | val member = MemberAddressMarshaller.parseAsciiString("localhost:5051") 15 | member should be(MemberAddress("localhost", 5051)) 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /suuchi-examples/src/test/scala/in/ashwanthkumar/suuchi/rpc/PingServiceTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{PingRequest, PingResponse} 4 | import io.grpc.stub.StreamObserver 5 | import org.mockito.Mockito._ 6 | import org.scalatest.FlatSpec 7 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 8 | import org.scalatest.concurrent.ScalaFutures.whenReady 9 | 10 | class PingServiceTest extends FlatSpec { 11 | val service = new PingService 12 | 13 | "PingService" should "return true when pinged" in { 14 | val request = PingRequest() 15 | val observer = mock(classOf[StreamObserver[PingResponse]]) 16 | whenReady(service.ping(request)) { response => 17 | response.status should be(true) 18 | } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /suuchi-cluster-atomix/README.md: -------------------------------------------------------------------------------- 1 | # [Atomix](http://atomix.io/atomix/) based SuuchiCluster 2 | 3 | This module provides Cluster implementation using [Raft](https://raft.github.io/) consensus algorithm. 4 | 5 | **Note** - This module needs Java8 to run. 6 | 7 | ## Configuration 8 | ``` 9 | ... 10 | cluster { 11 | atomix { 12 | port = 9090 # port used by atomix for cluster membership communication 13 | working-dir = "..." # location used for storing raft logs 14 | # cluster identifier to make sure all nodes are taking part in the right cluster. 15 | # You can also use environment specific identifiers to differentiate them. 16 | cluster-id = "..." 17 | rpc-port = "8080" # port used for gRPC communication 18 | } 19 | } 20 | ... 21 | ``` 22 | 23 | ## License 24 | https://www.apache.org/licenses/LICENSE-2.0 25 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/RoutingStrategySpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import com.google.protobuf.ByteString 4 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 5 | import org.scalatest.FlatSpec 6 | import org.scalatest.Matchers._ 7 | 8 | case class IntReq(i: Int) { 9 | def getKey: ByteString = ByteString.copyFrom(i.toString.getBytes) 10 | } 11 | class RoutingStrategySpec extends FlatSpec { 12 | "ConsistentHashingRoutingStrategy" should "route incoming requests WithKey to appropriate nodes" in { 13 | val routingStrategy = ConsistentHashingRouting( 14 | 2, 15 | 2, 16 | List(MemberAddress("host1:1"), MemberAddress("host2:2"), MemberAddress("host3:3")): _*) 17 | routingStrategy.route(IntReq(100)).size should be(2) 18 | routingStrategy.route(IntReq(100)).distinct.size should be(2) 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/Store.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.util 4 | 5 | trait ReadStore { 6 | def get(key: Array[Byte]): Option[Array[Byte]] 7 | } 8 | 9 | trait WriteStore { 10 | def put(key: Array[Byte], value: Array[Byte]): Boolean 11 | def remove(key: Array[Byte]): Boolean 12 | } 13 | 14 | case class KV(key: Array[Byte], value: Array[Byte]) { 15 | override def equals(obj: scala.Any): Boolean = obj match { 16 | case o: KV => util.Arrays.equals(key, o.key) && util.Arrays.equals(value, o.value) 17 | case _ => false 18 | } 19 | } 20 | trait Scannable { 21 | def scanner(): Scanner[KV] 22 | } 23 | 24 | trait Scanner[T] { 25 | def prepare(): Unit 26 | def scan(prefix: Array[Byte]): Iterator[T] 27 | def scan(): Iterator[T] 28 | def close(): Unit 29 | } 30 | 31 | trait Store extends ReadStore with WriteStore with Scannable 32 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/rpc/SuuchiShardService.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated._ 5 | import in.ashwanthkumar.suuchi.partitioner.ConsistentHashRing 6 | 7 | import scala.concurrent.Future 8 | 9 | class SuuchiShardService(ring: ConsistentHashRing, replicationFactor: Int) 10 | extends ShardsGrpc.Shards { 11 | private[rpc] def toNode(m: MemberAddress): Node = { 12 | Node(host = m.host, port = m.port) 13 | } 14 | 15 | override def info(request: ShardInfoRequest) = Future.successful { 16 | val shards = ring.ringState.withReplication(replicationFactor).map { 17 | case (token, replica) => 18 | Shard(start = token.start, end = token.end, nodes = replica.map(m => toNode(m))) 19 | }.toSeq 20 | ShardInfoResponse(shards) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /suuchi-examples/src/test/scala/in/ashwanthkumar/suuchi/rpc/SuuchiPutServiceTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import com.google.protobuf.ByteString 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated.PutRequest 5 | import in.ashwanthkumar.suuchi.store.InMemoryStore 6 | import org.scalatest.FlatSpec 7 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 8 | import org.scalatest.concurrent.ScalaFutures.whenReady 9 | 10 | class SuuchiPutServiceTest extends FlatSpec { 11 | val store = new InMemoryStore 12 | val service = new SuuchiPutService(store) 13 | 14 | "SuuchiPutService" should "support put for a Key-Value to the store" in { 15 | val request = PutRequest(key = ByteString.copyFrom("1".getBytes), value = ByteString.copyFrom("2".getBytes)) 16 | 17 | whenReady(service.put(request)) { response => 18 | response.status should be(true) 19 | } 20 | } 21 | 22 | } 23 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/VersionedBy.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import in.ashwanthkumar.suuchi.utils.DateUtils 4 | 5 | trait VersionedBy { 6 | val versionOrdering: Ordering[Long] 7 | def version(key: Array[Byte], value: Array[Byte]): Long 8 | 9 | /** 10 | * Choose either the versionTs or writtenTs for using it for purging. This is different 11 | * from [[VersionedBy#version]], only it it's way to store the actual version associated 12 | * with the key. 13 | * 14 | * @param version [[Version]] from which the implementation decides on what param it should sortOn 15 | * @return 16 | */ 17 | @inline 18 | def sortOn(version: Version): Long = version.versionTs 19 | } 20 | 21 | class ByWriteTimestamp extends VersionedBy with DateUtils { 22 | override def version(key: Array[Byte], value: Array[Byte]): Long = now 23 | override val versionOrdering: Ordering[Long] = Ordering.Long.reverse 24 | } 25 | -------------------------------------------------------------------------------- /docs/quick-start.md: -------------------------------------------------------------------------------- 1 | # Quick Start 2 | 3 | 1. Clone the repository [https://github.com/ashwanthkumar/suuchi-getting-started](https://github.com/ashwanthkumar/suuchi-getting-started) on your local machine. 4 | 5 | 2. Run `mvn clean compile` to generate the proto stubs for the project. 6 | 7 | 3. Import the project into your favorite IDE. 8 | 9 | 4. Create 3 Run configurations for `DistributedKVServer` main method with different arguments as 5051, 5052 and 5053 and start them all. 10 | 11 | 5. Open `SuuchiClient.scala` and run it to see them in action. 12 | 13 | 6. That's it! - you've now built a distributed, partitioned and replicated memory backed KVStore. 14 | 15 | ## See the Replication in Action 16 | 17 | 1. Change the port from `5051` to `5052` and stop the 5051 `DistributedKVServer` instance. 18 | 19 | 2. Remove the `client.put(...)` from the `SuuchiClient` to avoid writes into the cluster. 20 | 21 | 3. Now start the client's main method again, this time the reads should go through fine. 22 | -------------------------------------------------------------------------------- /docs/internals/partitioner.md: -------------------------------------------------------------------------------- 1 | # Partitioner 2 | 3 | Partitioners are defined by the following 4 | 5 | ```scala 6 | trait Partitioner { 7 | def find(key: Array[Byte], replicaCount: Int): List[MemberAddress] 8 | } 9 | ``` 10 | 11 | An implementation of Partitioner is supposed to return the list of nodes where the given key should be placed if we need `replicaCount` number of replicas. 12 | 13 | Suuchi by default comes with a ConsistentHashPartitioner which uses ConsistentHashRing underneath to partition the data. 14 | 15 | Interesting readings on Consistent Hash Ring 16 | 17 | - [http://blog.plasmaconduit.com/consistent-hashing/](http://blog.plasmaconduit.com/consistent-hashing/) 18 | - [http://www.paperplanes.de/2011/12/9/the-magic-of-consistent-hashing.html](http://www.paperplanes.de/2011/12/9/the-magic-of-consistent-hashing.html) 19 | 20 | An example of CH Ring during assignment or replication. 21 | 22 |
23 | CH Ring 24 |
25 | -------------------------------------------------------------------------------- /docs/Release_0.2.md: -------------------------------------------------------------------------------- 1 | # Suuchi Release 0.2 2 | ## Store 3 | - [x] Sharded Store - #53 4 | - [x] Versioned Store - #44 5 | - [x] Scan support in all stores - #61, #58 6 | 7 | ## Replication 8 | - [x] Pluggable replication - #45 9 | 10 | ## Routing 11 | - [x] Aggregator Support - #65 12 | 13 | ## Cluster 14 | - [x] Scalecube based clustering - #48 15 | - [x] Refactored Atomix + Scalecube with ClusterProvider abstractions - #68 16 | 17 | Atomix and scalecube modules need Java 8 to run. 18 | 19 | ## Bug Fixes 20 | - [x] Opening all stores in ShardedStore if they're not open already - #64 21 | - [x] Adding default of 10 minutes in replication, aggregation etc. - #62, #65 22 | 23 | ## Operations 24 | - [x] Moved away from SnapCI to Travis based build 25 | 26 | List of all the changes - https://github.com/ashwanthkumar/suuchi/pulls?q=is%3Apr+is%3Aclosed+milestone%3A0.2 27 | 28 | ## Contributors 29 | - [Ashwanth Kumar](https://github.com/ashwanthkumar) 30 | - [Sriram R](https://github.com/brewkode) 31 | - [Selvaram Ganesh](https://github.com/gsriram7) 32 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/cluster/SeedProvider.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster 2 | 3 | trait SeedProvider { 4 | 5 | /** 6 | * Returns a list of host that're used to connect to form a cluster. These nodes always represent the 7 | * initial seed nodes. While different [[Cluster]] implementations might have different guarantees, 8 | * It's generally a good practice to expose a single consistent copy of the nodes 9 | * as seed nodes. 10 | * 11 | * @return List[MemberAddress] that represents seed nodes to connect to 12 | */ 13 | def nodes: List[MemberAddress] 14 | } 15 | 16 | /** 17 | * Default in memory implementation of [[SeedProvider]] to be used in tests and static 18 | * configuration file based environments. 19 | * 20 | * @param nodes List[MemberAddress] that represents seed nodes to connect to 21 | */ 22 | case class InMemorySeedProvider(override val nodes: List[MemberAddress]) extends SeedProvider 23 | 24 | object InMemorySeedProvider { 25 | val EMPTY = InMemorySeedProvider(Nil) 26 | } 27 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/rpc/SuuchiAggregatorService.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import com.twitter.algebird.{Aggregator, LongRing, Semigroup} 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{AggregateRequest, AggregateResponse, AggregatorGrpc} 5 | import in.ashwanthkumar.suuchi.router.Aggregation 6 | 7 | import scala.concurrent.Future 8 | 9 | class SuuchiAggregatorService extends AggregatorGrpc.Aggregator { 10 | override def aggregate(request: AggregateRequest) = Future.successful(AggregateResponse(output = 1)) 11 | } 12 | 13 | class SumOfNumbers extends Aggregation { 14 | override def aggregator[ReqT, RespT] = { 15 | case AggregatorGrpc.METHOD_AGGREGATE => new Aggregator[AggregateResponse, Long, AggregateResponse] { 16 | override def prepare(input: AggregateResponse): Long = input.output 17 | override def semigroup: Semigroup[Long] = LongRing 18 | override def present(reduced: Long): AggregateResponse = AggregateResponse(output = reduced) 19 | }.asInstanceOf[Aggregator[RespT, Any, RespT]] 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /docs/Release_0.1.md: -------------------------------------------------------------------------------- 1 | # Suuchi Release 0.1 2 | ## What you get? 3 | - [ ] Ability for nodes to join a cluster - de-scoped - Tracked as part of [#20](https://github.com/ashwanthkumar/suuchi/issues/20) 4 | - [x] Ability to route traffic to different nodes based on CH strategy 5 | - [x] Ability to write data and read data from the cluster 6 | - [x] Pluggable store implementations 7 | - [x] Publish the project to maven for external consumption 8 | 9 | ### Membership 10 | - [x] Tests with members going up & down 11 | - [x] Ability to query any node and check for the available members 12 | 13 | ### Partitioner 14 | - [x] Publish Partitioner trait 15 | - [x] Implement CH Partitioner 16 | - [x] forwardOrHandle based on Partitioner trait 17 | 18 | ### Node Service 19 | - [x] gRPC <-> HTTP 20 | - [x] GET 21 | - [x] PUT 22 | - [x] HEALTH 23 | - [ ] SHARD_INFO (Good to have) - de-scoped - Tracked as part of [#20](https://github.com/ashwanthkumar/suuchi/issues/20) 24 | - what shards (key space) 25 | 26 | ### Store Implementations 27 | - [x] InMemoryStore - Implement an in-memory store that support get and put 28 | - [x] RocksDB Store 29 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/partitioner/Partitioner.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.partitioner 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | 5 | import scala.util.hashing.MurmurHash3 6 | 7 | trait Partitioner { 8 | def find(key: Array[Byte], replicaCount: Int): List[MemberAddress] 9 | def find(key: Array[Byte]): List[MemberAddress] = find(key, 1) 10 | } 11 | 12 | class ConsistentHashPartitioner(hashRing: ConsistentHashRing) extends Partitioner { 13 | override def find(key: Array[Byte], replicaCount: Int): List[MemberAddress] = { 14 | hashRing.findUnique(key, replicaCount) 15 | } 16 | } 17 | object ConsistentHashPartitioner { 18 | def apply(nodes: List[MemberAddress], partitionsPerNode: Int) = 19 | new ConsistentHashPartitioner(ConsistentHashRing(nodes, partitionsPerNode)) 20 | def apply(ring: ConsistentHashRing) = new ConsistentHashPartitioner(ring) 21 | } 22 | 23 | trait Hash { 24 | def hash(bytes: Array[Byte]): Integer 25 | } 26 | 27 | object SuuchiHash extends Hash { 28 | override def hash(bytes: Array[Byte]): Integer = MurmurHash3.bytesHash(bytes) 29 | } 30 | -------------------------------------------------------------------------------- /.travis/settings.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 10 | 11 | 12 | 13 | ossrh 14 | ${env.SONATYPE_USERNAME} 15 | ${env.SONATYPE_PASSWORD} 16 | 17 | 18 | 19 | 20 | ossrh 21 | 22 | true 23 | 24 | 25 | gpg 26 | ${env.GPG_PASSPHRASE} 27 | 28 | 29 | 30 | 31 | 32 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/rpc/CachedChannelPool.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import java.util.concurrent.ConcurrentHashMap 4 | 5 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 6 | import io.grpc.{ManagedChannel, ManagedChannelBuilder} 7 | 8 | import scala.language.existentials 9 | 10 | class CachedChannelPool(map: ConcurrentHashMap[String, ManagedChannel]) { 11 | def get(node: MemberAddress, insecure: Boolean = false): ManagedChannel = { 12 | val target = node.toExternalForm 13 | if (map.containsKey(target)) { 14 | map.get(target) 15 | } else { 16 | val builder = builderFrom(target) 17 | if (insecure) { 18 | builder.usePlaintext(true) 19 | } 20 | val channel = builder.build() 21 | map.put(target, channel) 22 | channel 23 | } 24 | } 25 | 26 | private[rpc] def builderFrom(key: String): ManagedChannelBuilder[_] = { 27 | ManagedChannelBuilder.forTarget(key) 28 | } 29 | } 30 | 31 | object CachedChannelPool { 32 | private val _default = new CachedChannelPool(new ConcurrentHashMap[String, ManagedChannel]()) 33 | 34 | def apply(): CachedChannelPool = _default 35 | } 36 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/router/Headers.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import io.grpc.{Context, Metadata} 4 | 5 | object Headers { 6 | val ELIGIBLE_NODES = "eligible_nodes" 7 | val PRIMARY_NODE = "primary_node" 8 | val REPLICATION_REQUEST = "replication_request" 9 | val BROADCAST_REQUEST = "broadcast_request" 10 | 11 | val REPLICATION_REQUEST_KEY = Metadata.Key.of(Headers.REPLICATION_REQUEST, StringMarshaller) 12 | 13 | /** 14 | * Context Key that's set to true if the node is processing a replication request is the primary node. 15 | * You might want to use this information 16 | * - to avoid doing double counts etc. 17 | * - to do certain tasks that should happen only at the primary replica etc. 18 | */ 19 | val PRIMARY_NODE_REQUEST_CTX: Context.Key[Boolean] = 20 | Context.keyWithDefault(Headers.PRIMARY_NODE, false) 21 | 22 | val ELIGIBLE_NODES_KEY = Metadata.Key.of(Headers.ELIGIBLE_NODES, ListOfMemberAddressMarshaller) 23 | val PRIMARY_NODE_KEY = Metadata.Key.of(Headers.PRIMARY_NODE, MemberAddressMarshaller) 24 | val BROADCAST_REQUEST_KEY = Metadata.Key.of(Headers.BROADCAST_REQUEST, BooleanMarshaller) 25 | } 26 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/ListOfMemberAddressMarshallerSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import org.scalatest.FlatSpec 5 | import org.scalatest.Matchers.{be, contain, convertToAnyShouldWrapper, have} 6 | 7 | class ListOfMemberAddressMarshallerSpec extends FlatSpec { 8 | "ListOfNodesMarshaller" should "convert list of members to ascii string" in { 9 | val members = List( 10 | MemberAddress("localhost", 5051), 11 | MemberAddress("localhost", 5052), 12 | MemberAddress("localhost", 5053) 13 | ) 14 | 15 | ListOfMemberAddressMarshaller.toAsciiString(members) should be( 16 | "localhost:5051|localhost:5052|localhost:5053") 17 | } 18 | 19 | it should "convert the ascii string to actual node objects" in { 20 | val members = 21 | ListOfMemberAddressMarshaller.parseAsciiString("localhost:5051|localhost:5052|localhost:5053") 22 | members should have size 3 23 | 24 | members should contain(MemberAddress("localhost", 5051)) 25 | members should contain(MemberAddress("localhost", 5052)) 26 | members should contain(MemberAddress("localhost", 5053)) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /suuchi-examples/src/test/scala/in/ashwanthkumar/suuchi/rpc/SuuchiShardServiceTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{ShardInfoRequest, ShardInfoResponse} 5 | import in.ashwanthkumar.suuchi.partitioner.ConsistentHashRing 6 | import io.grpc.stub.StreamObserver 7 | import org.mockito.Mockito._ 8 | import org.scalatest.FlatSpec 9 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 10 | import org.scalatest.concurrent.ScalaFutures.whenReady 11 | 12 | class SuuchiShardServiceTest extends FlatSpec { 13 | "SuuchiShardService" should "return shardInfo details for a given CHRing" in { 14 | val nodes = 15 | List(MemberAddress("host1", 1), MemberAddress("host2", 2), MemberAddress("host3", 3)) 16 | val ring = ConsistentHashRing(nodes, partitionsPerNode = 3) 17 | 18 | val observer = mock(classOf[StreamObserver[ShardInfoResponse]]) 19 | 20 | val service: SuuchiShardService = new SuuchiShardService(ring, replicationFactor = 2) 21 | whenReady(service.info(ShardInfoRequest())) { response => 22 | response.shard.size should be(9) // 3 nodes * 3 partitions per node 23 | } 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /docs/internals/replication.md: -------------------------------------------------------------------------------- 1 | # Replication Internals 2 | 3 | Suuchi out of the box comes with _Synchronous Sequential Replication_ during writes. It's fairly easy to build custom replicators. 4 | 5 | Refer [#27](https://github.com/ashwanthkumar/suuchi/pull/27) and [#23](https://github.com/ashwanthkumar/suuchi/pull/23) on how Replication is implemented. 6 | 7 | ## Types 8 | - Synchronous SequentialReplication (default available) 9 | - Synchronous ParallelReplication - (default available) 10 | - Synchronous ChainedReplication - ([#31](https://github.com/ashwanthkumar/suuchi/issues/31)) 11 | 12 | ## Working with Methods that handle Replication 13 | When implementing a service sometimes as a developer we would like to know if a particular invocation is a replication invocation or not. May be you might want to send a metric for every write and don't want to send the same metric multiple times. Sometimes you would also want to do a specific operation only at the primary shard and not others. You can use the following snippet 14 | 15 | ```scala 16 | class WriteService extends ... { 17 | def write(...) = { 18 | if(Headers.PRIMARY_NODE_REQUEST_CTX.get()) { 19 | // do something if this is a primary replica 20 | } 21 | } 22 | } 23 | ``` 24 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/store/PrimitivesSerDeUtilsSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import org.scalatest.FlatSpec 4 | import org.scalatest.Matchers._ 5 | 6 | class PrimitivesSerDeUtilsSpec extends FlatSpec { 7 | "PrimitivesSerDe" should "perform serde on long as expected" in { 8 | PrimitivesSerDeUtils.bytesToLong(PrimitivesSerDeUtils.longToBytes(100l)) should be(100l) 9 | PrimitivesSerDeUtils.bytesToLong(PrimitivesSerDeUtils.longToBytes(-100l)) should be(-100l) 10 | PrimitivesSerDeUtils.bytesToLong(PrimitivesSerDeUtils.longToBytes(Long.MaxValue)) should be( 11 | Long.MaxValue) 12 | PrimitivesSerDeUtils.bytesToLong(PrimitivesSerDeUtils.longToBytes(Long.MinValue)) should be( 13 | Long.MinValue) 14 | } 15 | 16 | it should "perform serde on int as expected" in { 17 | PrimitivesSerDeUtils.bytesToInt(PrimitivesSerDeUtils.intToBytes(100)) should be(100) 18 | PrimitivesSerDeUtils.bytesToInt(PrimitivesSerDeUtils.intToBytes(-100)) should be(-100) 19 | PrimitivesSerDeUtils.bytesToInt(PrimitivesSerDeUtils.intToBytes(Int.MaxValue)) should be( 20 | Int.MaxValue) 21 | PrimitivesSerDeUtils.bytesToInt(PrimitivesSerDeUtils.intToBytes(Int.MinValue)) should be( 22 | Int.MinValue) 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/cluster/TestStaticCluster.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster 2 | 3 | import com.typesafe.config.Config 4 | 5 | class TestStaticCluster(self: MemberAddress, config: Config, listeners: List[MemberListener]) 6 | extends Cluster(config, listeners) { 7 | var members: Set[MemberAddress] = _ 8 | 9 | override def start(seedProvider: SeedProvider): Cluster = { 10 | members = seedProvider.nodes.toSet 11 | this 12 | } 13 | override def nodes: scala.Iterable[MemberAddress] = members 14 | override def whoami: MemberAddress = self 15 | override def stop(): Unit = {} 16 | 17 | def addNode(node: MemberAddress): Unit = { 18 | members ++= Set(node) 19 | this.onJoin.apply(node) 20 | } 21 | 22 | def removeNode(node: MemberAddress): Unit = { 23 | members --= Set(node) 24 | this.onLeave.apply(node) 25 | } 26 | } 27 | 28 | class TestStaticClusterProvider extends ClusterProvider { 29 | override def createCluster(self: MemberAddress, 30 | config: Config, 31 | listeners: scala.List[MemberListener]): Cluster = { 32 | new TestStaticCluster(self, config, listeners) 33 | } 34 | override def priority: Int = 1 35 | } 36 | -------------------------------------------------------------------------------- /suuchi-cluster-atomix/src/main/scala/in/ashwanthkumar/suuchi/cluster/atomix/AtomixClusterProvider.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.atomix 2 | 3 | import com.typesafe.config.Config 4 | import in.ashwanthkumar.suuchi.cluster.{Cluster, ClusterProvider, MemberAddress, MemberListener} 5 | 6 | /** 7 | * [[ClusterProvider]] wrapper for [[AtomixCluster]]. Please note the config passed to `createCluster` 8 | * should have a sub-configuration called "atomix". That'll be used for getting atomix specific settings. 9 | */ 10 | class AtomixClusterProvider extends ClusterProvider { 11 | 12 | /** 13 | * @inheritdoc 14 | */ 15 | override def createCluster(self: MemberAddress, 16 | config: Config, 17 | listeners: List[MemberListener]) = { 18 | val atomixConfig = config.getConfig("atomix") 19 | new AtomixCluster( 20 | host = self.host, 21 | atomixPort = atomixConfig.getInt("port"), 22 | rpcPort = atomixConfig.getInt("rpc-port"), 23 | workDir = atomixConfig.getString("working-dir"), 24 | config = config, 25 | clusterIdentifier = atomixConfig.getString("cluster-id"), 26 | listeners = listeners 27 | ) 28 | } 29 | 30 | /** 31 | * @inheritdoc 32 | */ 33 | override def priority = 5 34 | } 35 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/StoreUtils.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | trait CloseableIterator[T] extends Iterator[T] with AutoCloseable 4 | 5 | object StoreUtils { 6 | 7 | def scan[T](scanner: Scanner[T]): CloseableIterator[T] = { 8 | scanner.prepare() 9 | val it = scanner.scan() 10 | new CloseableIterator[T] { 11 | var closed = false 12 | 13 | override def hasNext: Boolean = { 14 | val result = !closed && it.hasNext 15 | if(!closed && !result) close() 16 | result 17 | } 18 | override def next(): T = it.next() 19 | override def close(): Unit = { 20 | scanner.close() 21 | closed = true 22 | } 23 | } 24 | } 25 | 26 | def scan[T](prefix: Array[Byte], scanner: Scanner[T]): CloseableIterator[T] = { 27 | scanner.prepare() 28 | val it = scanner.scan(prefix) 29 | new CloseableIterator[T] { 30 | var closed = false 31 | 32 | override def hasNext: Boolean = { 33 | val result = !closed && it.hasNext 34 | if(!closed && !result) close() 35 | result 36 | } 37 | override def next(): T = it.next() 38 | override def close(): Unit = { 39 | scanner.close() 40 | closed = true 41 | } 42 | } 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /docs/recipes/kvclient.md: -------------------------------------------------------------------------------- 1 | # Distributed KVClient 2 | 3 | In the either of [InMemory KV Store](inmemorydb.md) or [RocksDB based KV Store](rocksdb.md) recipes we only started a server. We need some way to access the server. This recipe is about a simple gRPC client that does a PUT followed by a GET of the same key in the cluster and verify that they're the same. 4 | 5 | ``` 6 | package in.ashwanthkumar.suuchi 7 | 8 | import java.nio.ByteBuffer 9 | 10 | import in.ashwanthkumar.suuchi.client.SuuchiClient 11 | 12 | object DistributedKVClient extends App { 13 | val client = new SuuchiClient("localhost", 5051) 14 | val putResponse = client.put(Array(65.toByte), Array(65.toByte)) // puts k=v as A=A (in bytes) 15 | require(putResponse, "client should have responded successfully") 16 | 17 | val getResponse = client.get(Array(65.toByte)) 18 | require(getResponse.isDefined, "server should return a valid response") // gets k=A 19 | require(ByteBuffer.wrap(getResponse.get) == ByteBuffer.wrap(Array(65.toByte)), "response seems invalid - it should be A") 20 | 21 | println("Client has been validated") 22 | } 23 | ``` 24 | 25 | While running with this client you can find information about how the node in the point of contact with the client automatically 26 | - forwarded requests to the right node based on Input Key and ConsistentHash Ring 27 | - replicates the given message across multiple nodes again based on ConsistentHashing Ring. 28 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/partitioner/ConsistentHashPartitionerTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.partitioner 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import org.scalatest.FlatSpec 5 | import org.scalatest.Matchers.{be, contain, convertToAnyShouldWrapper, have} 6 | 7 | class ConsistentHashPartitionerTest extends FlatSpec { 8 | 9 | "ConsistentHashPartitioner" should "not return anything when underlying CHR has 0 nodes" in { 10 | val partitioner = ConsistentHashPartitioner(Nil, 3) 11 | partitioner.find("1".getBytes) should be(Nil) 12 | } 13 | 14 | it should "return a node when underlying CHR has a node" in { 15 | val partitioner = 16 | ConsistentHashPartitioner(ConsistentHashRing(List(MemberAddress("host1", 1)), 1)) 17 | partitioner.find("1".getBytes) should be(List(MemberAddress("host1", 1))) 18 | } 19 | 20 | it should "always return unique set of nodes for replication" in { 21 | val members = (1 to 5).map { index => 22 | MemberAddress(s"host$index", index) 23 | }.toList 24 | val partitioner = ConsistentHashPartitioner(ConsistentHashRing(members, 3)) 25 | val nodes = partitioner.find("1".getBytes, 3) 26 | nodes should have size 3 27 | nodes should contain(MemberAddress("host3", 3)) 28 | nodes should contain(MemberAddress("host4", 4)) 29 | nodes should contain(MemberAddress("host1", 1)) 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/cluster/ClusterProviderSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster 2 | 3 | import com.typesafe.config.Config 4 | import org.mockito.Mockito.{times, verify, when} 5 | import org.scalatest.FlatSpec 6 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper, have} 7 | import org.scalatest.mockito.MockitoSugar._ 8 | 9 | class ClusterProviderSpec extends FlatSpec { 10 | "ClusterProvider" should "return TestStaticCluster instance by doing service loading" in { 11 | val emptyFn: (MemberAddress) => Unit = (m: MemberAddress) => {} 12 | val listener = mock[MemberListener] 13 | when(listener.onJoin).thenReturn(emptyFn) 14 | when(listener.onLeave).thenReturn(emptyFn) 15 | 16 | val config = mock[Config] 17 | val cluster = ClusterProvider(MemberAddress("host1", 1), config, List(listener)) 18 | cluster.start(InMemorySeedProvider(List(MemberAddress("host1", 1), MemberAddress("host2", 2)))) 19 | 20 | cluster.nodes should have size 2 21 | cluster.whoami should be(MemberAddress("host1", 1)) 22 | 23 | cluster.asInstanceOf[TestStaticCluster].addNode(MemberAddress("host3", 3)) 24 | cluster.nodes should have size 3 25 | verify(listener, times(1)).onJoin 26 | 27 | cluster.asInstanceOf[TestStaticCluster].removeNode(MemberAddress("host3", 3)) 28 | cluster.nodes should have size 2 29 | verify(listener, times(1)).onLeave 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/src/test/scala/in/ashwanthkumar/suuchi/cluster/scalecube/ScaleCubeConfigTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.scalecube 2 | 3 | import com.typesafe.config.ConfigFactory 4 | import io.scalecube.cluster.gossip.GossipConfig 5 | import org.scalatest.FlatSpec 6 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 7 | 8 | class ScaleCubeConfigTest extends FlatSpec { 9 | "ScalecubeConfig" should "parse the config correctly" in { 10 | val scaleCubeConfig = 11 | """ 12 | |scalecube { 13 | | port = 9000 14 | |} 15 | """.stripMargin 16 | 17 | val config = ScaleCubeConfig(ConfigFactory.parseString(scaleCubeConfig)) 18 | config.port should be(9000) 19 | config.gossipConfig should be(None) 20 | } 21 | 22 | it should "parse the gossip settings as well from the config" in { 23 | val scaleCubeConfig = 24 | """ 25 | |scalecube { 26 | | port = 9000 27 | | gossip { 28 | | interval = 3000 29 | | fanout = 5 30 | | } 31 | |} 32 | """.stripMargin 33 | 34 | val config = ScaleCubeConfig(ConfigFactory.parseString(scaleCubeConfig)) 35 | val expectedGossipConfig = GossipConfig.builder().gossipFanout(5).gossipInterval(3000).build() 36 | config.port should be(9000) 37 | config.gossipConfig.get.getGossipFanout should be(5) 38 | config.gossipConfig.get.getGossipInterval should be(3000) 39 | } 40 | 41 | } 42 | -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/_pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | suuchi-cluster-scalecube 4 | 0.3.6-SNAPSHOT 5 | Suuchi Scalecube Membership 6 | Suuchi Scalecube based Membership 7 | jar 8 | 9 | 10 | in.ashwanthkumar 11 | suuchi 12 | 0.3.6-SNAPSHOT 13 | 14 | 15 | 16 | 2.6.2 17 | 18 | 19 | 20 | 21 | in.ashwanthkumar 22 | suuchi-core 23 | ${project.version} 24 | 25 | 26 | 27 | io.scalecube 28 | scalecube-cluster 29 | 0.9.0 30 | 31 | 32 | io.reactivex 33 | rxscala_${scala.lib.version} 34 | 0.26.2 35 | 36 | 37 | 38 | 39 | 40 | -------------------------------------------------------------------------------- /suuchi-rocksdb/_pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | suuchi-rocksdb 4 | 0.3.6-SNAPSHOT 5 | Suuchi Rocksdb 6 | RocksDB Store implementation for Suuchi 7 | jar 8 | 9 | 10 | in.ashwanthkumar 11 | suuchi 12 | 0.3.6-SNAPSHOT 13 | 14 | 15 | 16 | 5.8.6 17 | 18 | 19 | 20 | 21 | in.ashwanthkumar 22 | suuchi-core 23 | ${project.version} 24 | 25 | 26 | 27 | org.rocksdb 28 | rocksdbjni 29 | ${rocksdb.version} 30 | 31 | 32 | 33 | org.mockito 34 | mockito-core 35 | 1.10.19 36 | test 37 | 38 | 39 | 40 | -------------------------------------------------------------------------------- /suuchi-examples/src/test/scala/in/ashwanthkumar/suuchi/rpc/SuuchiReadServiceTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import com.google.protobuf.ByteString 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{GetRequest, GetResponse} 5 | import in.ashwanthkumar.suuchi.store.InMemoryStore 6 | import io.grpc.stub.StreamObserver 7 | import org.mockito.Mockito._ 8 | import org.scalatest.FlatSpec 9 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 10 | import org.scalatest.concurrent.ScalaFutures.whenReady 11 | 12 | class SuuchiReadServiceTest extends FlatSpec { 13 | val store = new InMemoryStore 14 | val service = new SuuchiReadService(store) 15 | 16 | "SuuchiReadService" should "support get on a value that exists in the store" in { 17 | store.put("1".getBytes, "2".getBytes) should be(true) 18 | val request = GetRequest(key = ByteString.copyFrom("1".getBytes)) 19 | 20 | val observer = mock(classOf[StreamObserver[GetResponse]]) 21 | whenReady(service.get(request)) { response => 22 | response.key should be(ByteString.copyFrom("1".getBytes)) 23 | response.value should be(ByteString.copyFrom("2".getBytes)) 24 | } 25 | } 26 | 27 | it should "return response with empty value with key does not exist in store" in { 28 | val request = GetRequest(key = ByteString.copyFrom("2".getBytes)) 29 | whenReady(service.get(request)) { response => 30 | response.key should be(ByteString.copyFrom("2".getBytes)) 31 | response.value.isEmpty should be(true) 32 | } 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /.travis/gpg.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | ## Shamelessly copied from https://github.com/idealo/logback-redis/wiki/Release-Process 4 | 5 | set -e 6 | 7 | # create a random passphrase 8 | export GPG_PASSPHRASE=$(echo "$RANDOM$(date)" | md5sum | cut -d\ -f1) 9 | 10 | # configuration to generate gpg keys 11 | cat >gen-key-script < 34 | # ssb 4096R/CC1613B2 2016-09-08 35 | # ssb 4096R/55B7CAA2 2016-09-08 36 | export GPG_KEYNAME=$(gpg -K | grep ^sec | cut -d/ -f2 | cut -d\ -f1 | head -n1) 37 | 38 | # cleanup local configuration 39 | shred --remove gen-key-script 40 | 41 | # publish the gpg key 42 | # (use keyserver.ubuntu.com as travis request keys from this server, 43 | # we avoid synchronization issues, while releasing) 44 | gpg --keyserver keyserver.ubuntu.com --send-keys ${GPG_KEYNAME} 45 | 46 | # wait for the key beeing accessible 47 | while(true); do 48 | date 49 | gpg --keyserver keyserver.ubuntu.com --recv-keys ${GPG_KEYNAME} && break || sleep 30 50 | done 51 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/router/Marshallers.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import io.grpc.Metadata.AsciiMarshaller 5 | 6 | /** 7 | * Send a string value using AsciiMarshaller 8 | */ 9 | case object StringMarshaller extends AsciiMarshaller[String] { 10 | override def toAsciiString(value: String): String = value 11 | override def parseAsciiString(serialized: String): String = serialized 12 | } 13 | 14 | /** 15 | * Send a boolean value using AsciiMarshaller 16 | */ 17 | case object BooleanMarshaller extends AsciiMarshaller[Boolean] { 18 | override def toAsciiString(value: Boolean): String = value.toString 19 | override def parseAsciiString(serialized: String): Boolean = serialized.toBoolean 20 | } 21 | 22 | /** 23 | * Converts a collection of [[MemberAddress]] to it's external form separated by `|` 24 | */ 25 | case object ListOfMemberAddressMarshaller extends AsciiMarshaller[List[MemberAddress]] { 26 | override def parseAsciiString(serialized: String): List[MemberAddress] = 27 | serialized.split('|').map(MemberAddress.apply).toList 28 | override def toAsciiString(value: List[MemberAddress]): String = 29 | value.map(_.toExternalForm).mkString("|") 30 | } 31 | 32 | /** 33 | * Converts a [[MemberAddress]] to it's external form 34 | */ 35 | case object MemberAddressMarshaller extends AsciiMarshaller[MemberAddress] { 36 | override def parseAsciiString(serialized: String): MemberAddress = 37 | MemberAddress.apply(serialized) 38 | override def toAsciiString(value: MemberAddress): String = 39 | value.toExternalForm 40 | } 41 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/utils/ByteArrayUtilsSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.utils 2 | 3 | import in.ashwanthkumar.suuchi.partitioner.{Hash, SuuchiHash} 4 | import org.scalatest.FlatSpec 5 | import org.scalatest.Matchers._ 6 | import org.mockito.Mockito._ 7 | 8 | class ByteArrayUtilsSpec extends FlatSpec { 9 | 10 | "ByteArrayUtils" should "say whether a given byte array starts with a specified byte array prefix" in { 11 | ByteArrayUtils.hasPrefix("string".getBytes, prefix = "char".getBytes) should be(false) 12 | ByteArrayUtils.hasPrefix("string".getBytes, prefix = "str".getBytes) should be(true) 13 | ByteArrayUtils.hasPrefix("string".getBytes, prefix = "string".getBytes) should be(true) 14 | } 15 | 16 | it should "return false when the given prefix is longer than the key" in { 17 | ByteArrayUtils.hasPrefix("string".getBytes, prefix = "longerString".getBytes) should be(false) 18 | } 19 | 20 | it should "return true if hash of the given key within start, end range" in { 21 | val hashFn = mock(classOf[Hash]) 22 | val key = "1".getBytes 23 | 24 | when(hashFn.hash(key)).thenReturn(10) 25 | 26 | ByteArrayUtils.isHashKeyWithinRange(1, 50, key, hashFn) should be(true) 27 | } 28 | 29 | it should "return false if hash of the given key is not within start, end range" in { 30 | val hashFn = mock(classOf[Hash]) 31 | val key = "1".getBytes 32 | 33 | when(hashFn.hash(key)).thenReturn(1) 34 | 35 | ByteArrayUtils.isHashKeyWithinRange(10, 100, key, hashFn) should be(false) 36 | 37 | when(hashFn.hash(key)).thenReturn(100) 38 | 39 | ByteArrayUtils.isHashKeyWithinRange(1, 10, key, hashFn) should be(false) 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/example/DistributedRocksDb.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.example 2 | 3 | import java.nio.file.Files 4 | import java.util.concurrent.Executors 5 | 6 | import in.ashwanthkumar.suuchi.example.DistributedKVServer.REPLICATION_FACTOR 7 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{PutGrpc, ReadGrpc} 8 | import in.ashwanthkumar.suuchi.router.ConsistentHashingRouting 9 | import in.ashwanthkumar.suuchi.rpc.Server._ 10 | import in.ashwanthkumar.suuchi.rpc.{Server, SuuchiPutService, SuuchiReadService} 11 | import in.ashwanthkumar.suuchi.store.rocksdb.{RocksDbConfiguration, RocksDbStore} 12 | import io.grpc.netty.NettyServerBuilder 13 | 14 | import scala.concurrent.ExecutionContext 15 | 16 | object DistributedRocksDb extends App { 17 | val port = args(0).toInt 18 | 19 | val REPLICATION_COUNT = 2 20 | val PARTITIONS_PER_NODE = 50 21 | val routingStrategy = 22 | ConsistentHashingRouting(REPLICATION_COUNT, PARTITIONS_PER_NODE, whoami(5051), whoami(5052)) 23 | 24 | val path = Files.createTempDirectory("distributed-rocksdb").toFile 25 | println(s"Using ${path.getAbsolutePath} for RocksDB") 26 | 27 | val cachedThreadPool = Executors.newCachedThreadPool() 28 | val executionContext = ExecutionContext.fromExecutor(cachedThreadPool) 29 | 30 | val store = new RocksDbStore(RocksDbConfiguration(path.getAbsolutePath)) 31 | val server = Server(NettyServerBuilder.forPort(port), whoami(port)) 32 | .routeUsing(ReadGrpc.bindService(new SuuchiReadService(store), executionContext), routingStrategy) 33 | .withParallelReplication(PutGrpc.bindService(new SuuchiPutService(store), executionContext), REPLICATION_FACTOR, routingStrategy) 34 | server.start() 35 | server.blockUntilShutdown() 36 | 37 | } 38 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/example/DistributedKVServer.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.example 2 | 3 | import java.util.concurrent.Executors 4 | 5 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{AggregatorGrpc, PutGrpc, ReadGrpc, ScanGrpc} 6 | import in.ashwanthkumar.suuchi.router.ConsistentHashingRouting 7 | import in.ashwanthkumar.suuchi.rpc.Server.whoami 8 | import in.ashwanthkumar.suuchi.rpc._ 9 | import in.ashwanthkumar.suuchi.store.InMemoryStore 10 | import io.grpc.netty.NettyServerBuilder 11 | 12 | import scala.concurrent.ExecutionContext 13 | 14 | // Start the app with either / one each of 5051, 5052 or/and 5053 port numbers 15 | object DistributedKVServer extends App { 16 | 17 | val port = args(0).toInt 18 | val PARTITIONS_PER_NODE = 100 19 | val REPLICATION_FACTOR = 2 20 | 21 | val allNodes = List(whoami(5051), whoami(5052), whoami(5053)) 22 | val routingStrategy = 23 | ConsistentHashingRouting(REPLICATION_FACTOR, PARTITIONS_PER_NODE, allNodes: _*) 24 | 25 | val cachedThreadPool = Executors.newCachedThreadPool() 26 | val executionContext = ExecutionContext.fromExecutor(cachedThreadPool) 27 | 28 | val store = new InMemoryStore 29 | val server = Server(NettyServerBuilder.forPort(port), whoami(port)) 30 | .routeUsing(ReadGrpc.bindService(new SuuchiReadService(store), executionContext), routingStrategy) 31 | .withParallelReplication(PutGrpc.bindService(new SuuchiPutService(store), executionContext), REPLICATION_FACTOR, routingStrategy) 32 | .withService(ScanGrpc.bindService(new SuuchiScanService(store), executionContext)) 33 | .aggregate(allNodes, AggregatorGrpc.bindService(new SuuchiAggregatorService(), executionContext), new SumOfNumbers) 34 | 35 | server.start() 36 | 37 | server.blockUntilShutdown() 38 | } 39 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/InMemoryStore.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.nio.ByteBuffer 4 | import java.util.concurrent.ConcurrentSkipListMap 5 | import in.ashwanthkumar.suuchi.utils.ByteArrayUtils 6 | import org.slf4j.LoggerFactory 7 | 8 | import scala.collection.JavaConversions._ 9 | 10 | class InMemoryStore extends Store { 11 | private val log = LoggerFactory.getLogger(getClass) 12 | private val store = new ConcurrentSkipListMap[ByteBuffer, Array[Byte]]() 13 | 14 | override def put(key: Array[Byte], value: Array[Byte]): Boolean = { 15 | log.trace(s"Put with key=${new String(key)}, value=${new String(value)}") 16 | store.put(ByteBuffer.wrap(key), value) 17 | true 18 | } 19 | 20 | override def get(key: Array[Byte]): Option[Array[Byte]] = { 21 | log.trace(s"Get with key=${new String(key)}") 22 | val value = Option(store.get(ByteBuffer.wrap(key))) 23 | log.trace(s"GetResult for key=${new String(key)}, value=${value.map(b => new String(b))}") 24 | value 25 | } 26 | 27 | override def remove(key: Array[Byte]): Boolean = { 28 | log.trace(s"Remove for key=${new String(key)}") 29 | store.remove(ByteBuffer.wrap(key)) 30 | true 31 | } 32 | 33 | override def scanner(): Scanner[KV] = new Scanner[KV] { 34 | override def prepare(): Unit = {} 35 | 36 | override def scan(prefix: Array[Byte]): Iterator[KV] = { 37 | store 38 | .tailMap(ByteBuffer.wrap(prefix)) 39 | .takeWhile { case (k, v) => ByteArrayUtils.hasPrefix(k.array(), prefix) } 40 | .map { case (k, v) => KV(k.array(), v) } 41 | .iterator 42 | } 43 | 44 | override def scan(): Iterator[KV] = 45 | store.entrySet().map(kv => KV(kv.getKey.array(), kv.getValue)).iterator 46 | 47 | override def close(): Unit = {} 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/rpc/SuuchiScanService.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import com.google.protobuf.ByteString 4 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{KV, ScanGrpc, ScanRequest, ScanResponse} 5 | import in.ashwanthkumar.suuchi.partitioner.SuuchiHash 6 | import in.ashwanthkumar.suuchi.store.Store 7 | import in.ashwanthkumar.suuchi.store.{KV => StoreKV} 8 | import in.ashwanthkumar.suuchi.utils.ByteArrayUtils 9 | import io.grpc.stub.{ServerCallStreamObserver, StreamObserver} 10 | 11 | class SuuchiScanService(store: Store) extends ScanGrpc.Scan { 12 | 13 | override def scan(request: ScanRequest, responseObserver: StreamObserver[ScanResponse]): Unit = { 14 | val observer = responseObserver.asInstanceOf[ServerCallStreamObserver[ScanResponse]] 15 | val start = request.start 16 | val end = request.end 17 | 18 | val scanner = store.scanner() 19 | scanner.prepare() 20 | val it = scanner 21 | .scan() 22 | .filter(kv => ByteArrayUtils.isHashKeyWithinRange(start, end, kv.key, SuuchiHash)) 23 | .map(buildResponse) 24 | 25 | observer.setOnCancelHandler(new Runnable() { 26 | override def run() = { 27 | scanner.close() 28 | } 29 | }) 30 | observer.setOnReadyHandler(new Runnable() { 31 | override def run() = { 32 | while (observer.isReady && it.hasNext) { 33 | observer.onNext(it.next) 34 | } 35 | 36 | if (!it.hasNext) { 37 | observer.onCompleted() 38 | scanner.close() 39 | } 40 | } 41 | }) 42 | } 43 | 44 | private def buildKV(kv: StoreKV) = { 45 | KV(key = ByteString.copyFrom(kv.key), value = ByteString.copyFrom(kv.value)) 46 | } 47 | 48 | private def buildResponse(response: StoreKV): ScanResponse = { 49 | ScanResponse(kv = Option(buildKV(response))) 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/cluster/ClusterProvider.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster 2 | 3 | import java.util.ServiceLoader 4 | 5 | import com.typesafe.config.Config 6 | 7 | abstract class ClusterProvider { 8 | 9 | /** 10 | * Create Membership implementation using a [[MemberAddress]] for cluster communication and [[MemberListener]] 11 | * 12 | * @param self [[MemberAddress]] of this node to use for cluster management 13 | * @param config [[Config]] instance that's used for configuring the Cluster implementation. This also acts 14 | * as a way of passing additional information / configuration parameters required for the Cluster 15 | * implementation 16 | * @param listeners List of [[MemberListener]] to hook into as part of the returned 17 | * cluster implementation 18 | * @return A Cluster implementation that's configured 19 | */ 20 | def createCluster(self: MemberAddress, config: Config, listeners: List[MemberListener]): Cluster 21 | 22 | /** 23 | * We use this method to sort when multiple providers are found. We'll pick the provider with highest value. 24 | * @return 25 | */ 26 | def priority: Int 27 | } 28 | 29 | object ClusterProvider { 30 | def apply(self: MemberAddress, clusterConfig: Config, listeners: List[MemberListener]) = { 31 | import scala.collection.JavaConversions._ 32 | 33 | val providers = ServiceLoader 34 | .load(classOf[ClusterProvider]) 35 | .iterator() 36 | .toList 37 | .sortBy(_.priority)(Ordering[Int].reverse) 38 | 39 | providers.headOption 40 | .map(_.createCluster(self, clusterConfig, listeners)) 41 | .getOrElse( 42 | throw new RuntimeException( 43 | "No Cluster implementations found. Consider adding suuchi-cluster-atomix or suuchi-cluster-scalecube modules to your dependencies") 44 | ) 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/store/InMemoryStoreTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import org.scalatest.FlatSpec 4 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper, have, startWith} 5 | 6 | class InMemoryStoreTest extends FlatSpec { 7 | "InMemoryStore" should "support get and put on a KV" in { 8 | val store = new InMemoryStore 9 | store.put("1".getBytes, "2".getBytes) should be(true) 10 | store.get("1".getBytes).map(v => new String(v)) should be(Some("2")) 11 | } 12 | 13 | it should "support full store scan" in { 14 | val store = new InMemoryStore 15 | store.put("1".getBytes, "one".getBytes) 16 | store.put("2".getBytes, "two".getBytes) 17 | store.put("3".getBytes, "three".getBytes) 18 | store.put("4".getBytes, "four".getBytes) 19 | store.put("5".getBytes, "five".getBytes) 20 | 21 | val scanner = store.scanner() 22 | scanner.prepare() 23 | val kVs = scanner.scan().toList 24 | scanner.close() 25 | kVs should have size 5 26 | kVs.sortBy(kv => new String(kv.key)) should be( 27 | List(kv("1", "one"), kv("2", "two"), kv("3", "three"), kv("4", "four"), kv("5", "five"))) 28 | } 29 | 30 | it should "support prefix scan" in { 31 | val store = new InMemoryStore 32 | store.put("prefix1/1".getBytes, "one".getBytes) 33 | store.put("prefix1/2".getBytes, "two".getBytes) 34 | store.put("prefix1/3".getBytes, "three".getBytes) 35 | store.put("prefix2/1".getBytes, "eleven".getBytes) 36 | store.put("prefix2/2".getBytes, "twelve".getBytes) 37 | store.put("prefix2/3".getBytes, "thirteen".getBytes) 38 | 39 | val scanner = store.scanner() 40 | scanner.prepare() 41 | val kVs = scanner.scan("prefix1".getBytes).toList 42 | scanner.close() 43 | 44 | kVs.foreach { kv => 45 | new String(kv.key) should startWith("prefix1") 46 | } 47 | } 48 | 49 | def kv(key: String, value: String) = KV(key.getBytes, value.getBytes) 50 | } 51 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/partitioner/RingStateSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.partitioner 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import org.scalatest.FlatSpec 5 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 6 | import org.mockito.Mockito._ 7 | 8 | class RingStateSpec extends FlatSpec { 9 | 10 | "RingState" should "return true if a key is within start - end combination" in { 11 | val key: Array[Byte] = "key".getBytes() 12 | 13 | RingState.contains(key, Int.MinValue, Int.MaxValue, SuuchiHash) should be(true) 14 | RingState.contains(key, -838523459, 0, SuuchiHash) should be(true) 15 | RingState.contains(key, -838523456, -838523458, SuuchiHash) should be(false) 16 | 17 | val mockHash = mock(classOf[Hash]) 18 | when(mockHash.hash(key)).thenReturn(1) 19 | RingState.contains(key, 1, 9, mockHash) should be(true) 20 | RingState.contains(key, 0, 10, mockHash) should be(true) 21 | RingState.contains(key, -10, 10, mockHash) should be(true) 22 | RingState.contains(key, -10, -1, mockHash) should be(false) 23 | } 24 | 25 | it should "return true if a key is within TokenRange" in { 26 | val key: Array[Byte] = "key".getBytes() 27 | RingState.contains(key, rangeOf(Int.MinValue, Int.MaxValue), SuuchiHash) should be(true) 28 | RingState.contains(key, rangeOf(-838523459, 0), SuuchiHash) should be(true) 29 | } 30 | 31 | it should "find the tokenRange that encapsulates the given key" in { 32 | val ring = new ConsistentHashRing(SuuchiHash, 2, 2) 33 | ring 34 | .add(MemberAddress("host1", 1)) 35 | .add(MemberAddress("host2", 2)) 36 | 37 | val ringState = ring.ringState 38 | RingState.find("key".getBytes, ringState, SuuchiHash) should be( 39 | Some( 40 | TokenRange(-1758288377, 267687071, VNode(MemberAddress("host1", 1), 1)) 41 | )) 42 | } 43 | 44 | def rangeOf(start: Int, end: Int): TokenRange = { 45 | TokenRange(start, end, VNode(MemberAddress("host1", 1), 1)) 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /suuchi-cluster-atomix/_pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | suuchi-cluster-atomix 4 | 0.3.6-SNAPSHOT 5 | Suuchi Atomix Membership 6 | Suuchi Atomix based Membership 7 | jar 8 | 9 | 10 | in.ashwanthkumar 11 | suuchi 12 | 0.3.6-SNAPSHOT 13 | 14 | 15 | 16 | 2.6.2 17 | 1.0.6 18 | 1.2.0 19 | 1.8 20 | 21 | 22 | 23 | 24 | in.ashwanthkumar 25 | suuchi-core 26 | ${project.version} 27 | 28 | 29 | 30 | io.atomix 31 | atomix 32 | ${atomix.version} 33 | 34 | 35 | io.atomix 36 | atomix-resource 37 | ${atomix.version} 38 | 39 | 40 | io.atomix.catalyst 41 | catalyst-transport 42 | ${catalyst.version} 43 | 44 | 45 | io.atomix.catalyst 46 | catalyst-netty 47 | ${catalyst.version} 48 | 49 | 50 | 51 | 52 | 53 | -------------------------------------------------------------------------------- /suuchi-rocksdb/src/main/scala/in/ashwanthkumar/suuchi/store/rocksdb/RocksDbConfiguration.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store.rocksdb 2 | 3 | import org.rocksdb.Options 4 | 5 | case class RocksDbConfiguration(location: String, 6 | batchSize: Int, 7 | readOnly: Boolean, 8 | memTableSize: Int, 9 | numFilesAtBase: Int, 10 | maxBytesForBaseLevel: Long, 11 | maxWriteBufferNumber: Int, 12 | maxBackgroundCompactions: Int, 13 | baseOptions: Options) { 14 | val perFileSizeAtBase = maxBytesForBaseLevel / numFilesAtBase 15 | def toOptions = { 16 | baseOptions 17 | .setMaxBytesForLevelBase(maxBytesForBaseLevel) 18 | .setTargetFileSizeBase(perFileSizeAtBase) 19 | .setWriteBufferSize(memTableSize) 20 | .setMaxWriteBufferNumber(maxWriteBufferNumber) 21 | .setMaxBackgroundCompactions(maxBackgroundCompactions) 22 | } 23 | } 24 | 25 | object RocksDbConfiguration { 26 | val BATCH_SIZE = 100000 27 | val DESIRED_NUM_FILES_AT_BASE_LEVEL = 10 28 | val MEMTABLE_SIZE = 128 * 1024 * 1024 29 | val MAX_BYTES_FOR_BASE_LEVEL = 4l * 1024 * 1024 * 1024 30 | val MAX_WRITE_BUFFER_NUMBER = 2 31 | val MAX_BG_COMPACTIONS = 2 32 | 33 | def apply(location: String): RocksDbConfiguration = 34 | apply(location, new Options().setCreateIfMissing(true)) 35 | 36 | def apply(location: String, options: Options): RocksDbConfiguration = 37 | new RocksDbConfiguration(location, 38 | BATCH_SIZE, 39 | false, 40 | MEMTABLE_SIZE, 41 | DESIRED_NUM_FILES_AT_BASE_LEVEL, 42 | MAX_BYTES_FOR_BASE_LEVEL, 43 | MAX_WRITE_BUFFER_NUMBER, 44 | MAX_BG_COMPACTIONS, 45 | options) 46 | } 47 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/protobuf/example.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | option java_package = "in.ashwanthkumar.suuchi.examples.rpc.generated"; 4 | option java_outer_classname = "SuuchiRPC"; 5 | 6 | // The Ping service definition. 7 | service PingService { 8 | rpc Ping (PingRequest) returns (PingResponse); 9 | } 10 | message PingRequest { 11 | } 12 | message PingResponse { 13 | bool status = 1; 14 | } 15 | 16 | // Read service which can be used for serving read-only datasets 17 | service Read { 18 | rpc Get (GetRequest) returns (GetResponse); 19 | } 20 | message GetRequest { 21 | bytes key = 1; 22 | } 23 | message GetResponse { 24 | bytes key = 1; 25 | bytes value = 2; 26 | } 27 | 28 | // Put service which can be used for taking in writes 29 | service Put { 30 | rpc Put (PutRequest) returns (PutResponse); 31 | } 32 | message PutRequest { 33 | bytes key = 1; 34 | bytes value = 2; 35 | } 36 | message PutResponse { 37 | bool status = 1; 38 | } 39 | 40 | // Shards service which can be used for getting all Shard details 41 | service Shards { 42 | rpc Info (ShardInfoRequest) returns (ShardInfoResponse); 43 | } 44 | 45 | message ShardInfoRequest { 46 | } 47 | message ShardInfoResponse { 48 | repeated Shard shard = 1; 49 | } 50 | message Shard { 51 | int32 start = 1; 52 | int32 end = 2; 53 | 54 | repeated Node nodes = 3; 55 | } 56 | message Node { 57 | string host = 1; 58 | int32 port = 2; 59 | } 60 | 61 | // Scan service is used for doing streaming Reads over a range of data 62 | service Scan { 63 | rpc Scan (ScanRequest) returns (stream ScanResponse); 64 | } 65 | message ScanRequest { 66 | int32 start = 1; 67 | int32 end = 2; 68 | } 69 | message ScanResponse { 70 | KV kv = 1; 71 | } 72 | message KV { 73 | bytes key = 1; 74 | bytes value = 2; 75 | } 76 | 77 | // Aggregator service is used for doing reduce and re-reduce operations; otherwise also known as scatter-gather 78 | // request paradigm. 79 | service Aggregator { 80 | rpc aggregate (AggregateRequest) returns (AggregateResponse); 81 | } 82 | message AggregateRequest { 83 | } 84 | message AggregateResponse { 85 | int64 output = 1; 86 | } 87 | -------------------------------------------------------------------------------- /docs/recipes/rocksdb.md: -------------------------------------------------------------------------------- 1 | # Distributed RocksDB backed KV 2 | 3 | ### Dependencies 4 | ```xml 5 | 6 | in.ashwanthkumar 7 | suuchi-core 8 | ${suuchi.version} 9 | 10 | 11 | in.ashwanthkumar 12 | suuchi-rocksdb 13 | ${suuchi.version} 14 | 15 | ``` 16 | 17 | ### Code 18 | 19 | Following code builds a consistent hashing based Get/Put requests backed by [RocksDB](https://github.com/facebook/rocksdb). It also does replication for Put requests to `REPLICATION_COUNT` number of nodes in the cluster. 20 | 21 | ```scala 22 | package in.ashwanthkumar.suuchi.example 23 | 24 | import java.nio.file.Files 25 | 26 | import in.ashwanthkumar.suuchi.router.ConsistentHashingRouting 27 | import in.ashwanthkumar.suuchi.rpc.Server._ 28 | import in.ashwanthkumar.suuchi.rpc.{Server, SuuchiPutService, SuuchiReadService} 29 | import in.ashwanthkumar.suuchi.store.rocksdb.{RocksDbConfiguration, RocksDbStore} 30 | import io.grpc.netty.NettyServerBuilder 31 | 32 | object DistributedRocksDb extends App { 33 | val port = args(0).toInt 34 | 35 | val REPLICATION_COUNT = 2 36 | val PARTITIONS_PER_NODE = 50 37 | val routingStrategy = ConsistentHashingRouting(REPLICATION_COUNT, PARTITIONS_PER_NODE, whoami(5051), whoami(5052)) 38 | 39 | val path = Files.createTempDirectory("distributed-rocksdb").toFile 40 | println(s"Using ${path.getAbsolutePath} for RocksDB") 41 | val store = new RocksDbStore(RocksDbConfiguration(path.getAbsolutePath)) 42 | val server = Server(NettyServerBuilder.forPort(port), whoami(port)) 43 | .routeUsing(new SuuchiReadService(store), routingStrategy) 44 | .withParallelReplication(new SuuchiPutService(store), REPLICATION_COUNT, routingStrategy) 45 | server.start() 46 | server.blockUntilShutdown() 47 | 48 | } 49 | ``` 50 | 51 | This code is available as part of [`suuchi-examples`](https://github.com/ashwanthkumar/suuchi/tree/master/suuchi-examples) module in the repo. 52 | 53 | To see this recipe in action, you might also want to look into the client which can talk to this service - [DistributedKVClient](kvclient.md). 54 | -------------------------------------------------------------------------------- /suuchi-rocksdb/src/main/scala/in/ashwanthkumar/suuchi/store/rocksdb/RocksDbStore.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store.rocksdb 2 | 3 | import in.ashwanthkumar.suuchi.store.{KV, Scanner, Store} 4 | import in.ashwanthkumar.suuchi.utils.{ByteArrayUtils, Logging} 5 | import org.rocksdb._ 6 | 7 | import scala.language.postfixOps 8 | 9 | class RocksDbStore(config: RocksDbConfiguration) extends Store with Logging { 10 | lazy val db = { 11 | if (config.readOnly) RocksDB.openReadOnly(config.toOptions, config.location) 12 | else RocksDB.open(config.toOptions, config.location) 13 | } 14 | 15 | RocksDB.loadLibrary() 16 | lazy val writeOptions = new WriteOptions().setDisableWAL(false).setSync(true) 17 | 18 | override def get(key: Array[Byte]) = this.synchronized { 19 | Option(db.get(key)) 20 | } 21 | 22 | override def put(key: Array[Byte], value: Array[Byte]) = this.synchronized { 23 | logOnError(() => db.put(writeOptions, key, value)) isSuccess 24 | } 25 | 26 | def close() = { 27 | log.info(s"[Closing RocksDb]") 28 | db.close() 29 | } 30 | 31 | override def remove(key: Array[Byte]): Boolean = { 32 | logOnError(() => db.remove(key)) isSuccess 33 | } 34 | 35 | override def scanner(): Scanner[KV] = new RocksDBScanner(db) 36 | } 37 | 38 | class RocksDBScanner(db: RocksDB) extends Scanner[KV] { 39 | 40 | private[this] lazy val snapshot = db.getSnapshot 41 | private[this] var rocksIterator: RocksIterator = _ 42 | 43 | override def prepare(): Unit = { 44 | rocksIterator = db.newIterator() 45 | } 46 | 47 | override def scan(prefix: Array[Byte]): Iterator[KV] = { 48 | rocksIterator.seek(prefix) 49 | 50 | new Iterator[KV] { 51 | override def hasNext: Boolean = 52 | rocksIterator.isValid && ByteArrayUtils.hasPrefix(rocksIterator.key(), prefix) 53 | 54 | override def next(): KV = { 55 | val kv = KV(rocksIterator.key(), rocksIterator.value()) 56 | rocksIterator.next() 57 | kv 58 | } 59 | } 60 | } 61 | 62 | override def scan(): Iterator[KV] = scan(Array.ofDim[Byte](0)) 63 | 64 | override def close(): Unit = { 65 | rocksIterator.close() 66 | db.releaseSnapshot(snapshot) 67 | snapshot.close() 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /docs/internals/router.md: -------------------------------------------------------------------------------- 1 | # HandleOrForward Router 2 | 3 | HandleOrForward Router is the entry point of a request in your Suuchi based application. It uses a _RoutingStrategy_ implementation to decide which nodes in the cluster are eligible for handling the current request. It also takes care of forwarding the request to that particular node and returning the response back to the client. 4 | 5 | Since there isn't any _SPOC_ (Single Point of Contact) in the system, any node in the cluster can handle or forward any request automatically. This makes the whole operations of the systems very easy. You can setup a load balancer as an entry point to your app 6 | backed by all the nodes in the cluster. 7 | 8 | Refer [#23](https://github.com/ashwanthkumar/suuchi/pull/23), [#11](https://github.com/ashwanthkumar/suuchi/pull/11) and [#2](https://github.com/ashwanthkumar/suuchi/pull/2) on how HandleOrForward Router is implemented. TBD - Explain with pictures on how it works. 9 | 10 | ## RoutingStrategy 11 | RoutingStrategy forms the heart of HandleOrForward router. Out of the box Suuchi comes with the following routing strategies 12 | 13 | - [ConsistentHashingRouting](https://en.wikipedia.org/wiki/Consistent_hashing) 14 | 15 | ## Custom Routers 16 | [_RoutingStrategy_](https://github.com/ashwanthkumar/suuchi/blob/master/suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/router/RoutingStrategy.scala#L10) trait is defined as follows 17 | 18 | ```scala 19 | trait RoutingStrategy { 20 | /** 21 | * Decides if the incoming message should be forwarded or handled by the current node. 22 | * 23 | * @tparam ReqT Type of the input Message 24 | * @return Some(MemberAddress) - if the request is meant to be forwarded 25 | *

None - if the request can be handled by the current node 26 | */ 27 | def route[ReqT]: PartialFunction[ReqT, Option[MemberAddress]] 28 | } 29 | ``` 30 | 31 | Any implementations of that trait can be passed to HandleOrForward Router. 32 | 33 | ## Notes 34 | 35 | - HandleOrForward Router is implemented internally as a ServerInterceptor. What this means is, when you're handling a streaming request every message that's sent in the stream goes through HandleOrForward backed by a RoutingStrategy to decide which nodes the request should go to. 36 | -------------------------------------------------------------------------------- /docs/recipes/inmemorydb.md: -------------------------------------------------------------------------------- 1 | # Distributed InMemory Database 2 | 3 | Following code builds a consistent hashing based Get/Put requests backed by an ConcurrentHashMap in memory. 4 | 5 | ```scala 6 | package in.ashwanthkumar.suuchi.example 7 | 8 | import java.nio.ByteBuffer 9 | 10 | import in.ashwanthkumar.suuchi.client.SuuchiClient 11 | import in.ashwanthkumar.suuchi.router.ConsistentHashingRouting 12 | import in.ashwanthkumar.suuchi.rpc.Server.whoami 13 | import in.ashwanthkumar.suuchi.rpc.{Server, SuuchiPutService, SuuchiReadService} 14 | import in.ashwanthkumar.suuchi.store.InMemoryStore 15 | import io.grpc.netty.NettyServerBuilder 16 | 17 | object DistributedKVServer extends App { 18 | val port = args(0).toInt 19 | val PARTITIONS_PER_NODE = 100 20 | val REPLICATION_FACTOR = 2 21 | 22 | val routingStrategy = ConsistentHashingRouting(REPLICATION_FACTOR, PARTITIONS_PER_NODE, whoami(5051), whoami(5052), whoami(5053)) 23 | 24 | val store = new InMemoryStore 25 | val server = Server(NettyServerBuilder.forPort(port), whoami(port)) 26 | .routeUsing(new SuuchiReadService(store), routingStrategy) 27 | .withParallelReplication(new SuuchiPutService(store), REPLICATION_FACTOR, routingStrategy) 28 | server.start() 29 | 30 | server.blockUntilShutdown() 31 | } 32 | ``` 33 | 34 | Let's break down the above code step by step. 35 | 36 | - `ConsistentHashingRouting` is a [_Routing Strategy_](../internals/router.md#routingstrategy) that does routing between all the nodes using a ConsistentHashRing underneath with default vnode factor of 3. 37 | - `NettyServerBuilder.forPort(5051)` creates a NettyServer on `5051` port. 38 | - `server.routeUsing()` adds a new protobuf rpc using a custom routing strategy behind [_HandleOrForward_](../internals/router.md) router. 39 | - `server.withParallelReplication()` adds a new protobuf rpc using the ReplicationRouter. By default it wraps both [_HandleOrForward_](../internals/router.md) and [_Replicator_](../internals/replication.md) routers. 40 | - `server1.start()` starts the underlying gRPC server. 41 | - `server1.blockUntilShutdown()` waits until the server is stopped. 42 | 43 | To see this recipe in action, you might also want to look into the client which can talk to this service - [DistributedKVClient](kvclient.md). 44 | -------------------------------------------------------------------------------- /suuchi-rocksdb/src/test/scala/in/ashwanthkumar/suuchi/store/rocksdb/RocksDbStoreSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store.rocksdb 2 | 3 | import java.nio.file.Files 4 | import java.util.UUID 5 | 6 | import in.ashwanthkumar.suuchi.store.{KV, StoreUtils} 7 | import org.apache.commons.io.FileUtils 8 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper, have, startWith} 9 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FlatSpec} 10 | 11 | import scala.language.postfixOps 12 | 13 | class RocksDbStoreSpec extends FlatSpec with BeforeAndAfter with BeforeAndAfterAll { 14 | 15 | val dir = Files.createTempDirectory("suuchi-rocks-test").toFile 16 | 17 | override def afterAll() = { 18 | FileUtils.deleteDirectory(dir.getAbsoluteFile) 19 | } 20 | 21 | "RocksDb" should "store & retrieve results properly" in { 22 | val db = createDB() 23 | (1 to 100).foreach { i => 24 | db.put(Array(i toByte), Array(i * 2 toByte)) 25 | } 26 | 27 | (1 to 100).foreach { i => 28 | db.get(Array(i toByte)).get.head should be(i * 2 toByte) 29 | } 30 | } 31 | 32 | it should "support full db scan" in { 33 | val db = createDB() 34 | val inputKVs = (1 to 100).map(i => (Array(i toByte), Array(i * 2 toByte))) 35 | 36 | inputKVs.foreach { case (k, v) => db.put(k, v) } 37 | val scannedResult = StoreUtils.scan(db.scanner()).toList 38 | 39 | scannedResult should have size 100 40 | scannedResult.sortBy(kv => new String(kv.key)) should be(inputKVs.map { 41 | case (k, v) => KV(k, v) 42 | }.toList) 43 | } 44 | 45 | it should "support prefix scan" in { 46 | val db = createDB() 47 | val kVs = (1 to 100).flatMap( 48 | i => 49 | List((s"prefix1/$i".getBytes, Array(i toByte)), 50 | (s"prefix2/$i".getBytes, Array(i * 2 toByte)))) 51 | 52 | kVs.foreach { case (k, v) => db.put(k, v) } 53 | val scannedResult = StoreUtils.scan("prefix1".getBytes, db.scanner()).toList 54 | 55 | scannedResult should have size 100 56 | scannedResult.foreach { kv => 57 | new String(kv.key) should startWith("prefix1") 58 | } 59 | } 60 | 61 | def createDB() = { 62 | val location = dir.getAbsolutePath + "/" + UUID.randomUUID() 63 | new RocksDbStore(RocksDbConfiguration(location)) 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | scala: 3 | - 2.11.12 4 | sudo: false 5 | jdk: 6 | - openjdk8 7 | env: 8 | global: 9 | - secure: MmyMAHnhCNXJIclWXIZIIb2ZpJZknvaqZ2HUpxf+LIsr0dSwg3OhULc6hOEsktusio4rf7/mK5FihOEz4Lb4FFAzTuTn23YyPjjNY4BHIoqF+v5sgy0z8EGUl+DymoJExQ02CRwy6gyIk0MUYbcuZMbUEg5S8v/JvLL1hgEhv+8eAN3dGrMoog8NsDuYlhzZlFXXm8DN83PZ/9boagWq+WGHajNCUC7Ra7y4TfhRUKEDZfoln39xeazQwdUFq9iZ0urGM+QhZxL6wf2G68DAHfTmor/ahVZlOMcbmhakVqxVYz17f0ovKlvD2ybBDzaqae5j/AiTkEmHTEvXzXeSJT0VHE0/JuSHH+r+1tJxnrT+zg6s9YSOUDaZRkIWmLFe/jWV3X8AZ9MwETZDQptyTgxHqIYM8HBQA8j3CkjzgLMFkZfqwVgJRc2QjUwKKttmLEM7mJlxuyNoqD5fNHhFJCJBnJtJQAV3J6CEse4a1qOQRJoFBGIp8u5l/ej9Bwdf+WVp+mXt3uQOfr6L6PgYE+Ll9gy/gAPwn8IgllYN9q/ypGc9BO2fe77uWfDgQsTjwNfNnA0wKf3YYYunNnMN58oFBjukpZZKiyUOjgEdrrcCX/I6zMRDaA2YrW4s+JG1Idr2przEouPTbNjanw0hqQzz96Gslsv6FNhRYJaJicw= 10 | - secure: rUkD/vXLmdUB/hDBkregZz4YmJoG7Ro91xJbADumf4Lzfm8wO4bK1mpbsGqBDOXvp/the76MPOdu6halDw8njktfXcrwUCPQaXtn8A7+bQW1Kc1L/SpKs5G+ycGVKF8AAYUMMBRs6RX1QHQTwxSOVkOTYD6me9rYZTX6PdcW9fCw7EgIzBc8X2I8tQckCj5DIoLECxUo2bwrkL7KKkNmFPLK7Cm6jddNASqzqZo3pybeGTFuGbAsXN11b2CfINl2srjJ/gH3DDXKFlcAKpPuH7YW+3YIA/15S6xoimUFnLaUILFpNTTwxGmQwZ/h235K6kh0kWHdPdeUlGzl0yN8KgSwS1DFKLfe+VrKSD0441XyW/snwtrlC9jl/jX2A5uxaalVDD27MZj3Z8kpdSHtuaYIZ86d7yNRWqOJMcwkI7K9W4JiocBGZ373EttxVM7dywQuPPu/bwyEI2fq44/JUQD+3AJ2Exgoh1NFrHT+H+NCDXY+x7W8S+lD9adlrmvHZfNZ1I401V8u9mEApw0XWaQCIvKfqXogk4UDtzxAvJKv0J7A1k70xH59OtnU5kHWF5j1805hkVhp/1+AC6UNVaiNLAoRNSH5PQFRtfJz4WyfMmonR+ULtBsFTdw6ON1gxbHAHiAY0WQmcBwUsF3gOn8GZrXKGxizKb6QutH64wI= 11 | - SONATYPE_USERNAME: ashwanthkumar 12 | cache: 13 | directories: 14 | - $HOME/.m2 15 | - $HOME/DENV 16 | before_install: 17 | - openssl aes-256-cbc -pass pass:$SONATYPE_PASSWORD -in .travis/secring.gpg.enc -out local.secring.gpg -d 18 | - openssl aes-256-cbc -pass pass:$SONATYPE_PASSWORD -in .travis/pubring.gpg.enc -out local.pubring.gpg -d 19 | after_success: 20 | - bash <(curl -s https://codecov.io/bash) 21 | 22 | addons: 23 | apt: 24 | packages: 25 | - python-virtualenv 26 | 27 | deploy: 28 | - provider: script 29 | script: .travis/release.sh 30 | skip_cleanup: true 31 | on: 32 | repo: ashwanthkumar/suuchi 33 | branch: master 34 | jdk: openjdk8 35 | 36 | - provider: script 37 | script: .travis/docs.sh 38 | on: 39 | repo: ashwanthkumar/suuchi 40 | branch: master 41 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/router/RoutingStrategy.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import com.google.protobuf.ByteString 4 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 5 | import in.ashwanthkumar.suuchi.partitioner.{ConsistentHashRing, ConsistentHashPartitioner} 6 | import org.slf4j.LoggerFactory 7 | 8 | import scala.language.reflectiveCalls 9 | 10 | trait RoutingStrategy { 11 | 12 | /** 13 | * Decides if the incoming message should be forwarded or handled by the current node itself. 14 | * 15 | * @tparam ReqT Type of the input Message 16 | * @return List[MemberAddress] 17 | *

> 1 when there are multiple nodes to which this needs to be replicated to including the current node 18 | *

= 1 when replication factor is 1 19 | *

Nil - should never happen unless there's a bug 20 | */ 21 | def route[ReqT]: PartialFunction[ReqT, List[MemberAddress]] 22 | } 23 | object RoutingStrategy { 24 | type WithKey = { def getKey: ByteString } 25 | } 26 | 27 | /** 28 | * Always forward the requests to a given node - useful in tests or while debugging 29 | * @param members 30 | */ 31 | class AlwaysRouteTo(members: MemberAddress*) extends RoutingStrategy { 32 | private val log = LoggerFactory.getLogger(getClass) 33 | 34 | /** 35 | * @inheritdoc 36 | */ 37 | override def route[ReqT]: PartialFunction[ReqT, List[MemberAddress]] = { 38 | case msg: RoutingStrategy.WithKey => 39 | members.toList 40 | } 41 | } 42 | 43 | /** 44 | * Uses a ConsistentHash based Partitioner to find the right node for the incoming message. 45 | * @param partitioner - which is an implementation of ConsistentHashPartitioner 46 | **/ 47 | class ConsistentHashingRouting(partitioner: ConsistentHashPartitioner, nrReplicas: Int) 48 | extends RoutingStrategy { 49 | override def route[ReqT]: PartialFunction[ReqT, List[MemberAddress]] = { 50 | case msg: RoutingStrategy.WithKey => partitioner.find(msg.getKey.toByteArray, nrReplicas) 51 | } 52 | } 53 | 54 | object ConsistentHashingRouting { 55 | def apply(nrReplicas: Int, partitionsPerNode: Int, nodes: MemberAddress*) = { 56 | new ConsistentHashingRouting(ConsistentHashPartitioner(nodes.toList, partitionsPerNode), 57 | nrReplicas) 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /suuchi-cluster-atomix/src/test/scala/in/ashwanthkumar/suuchi/cluster/atomix/AtomixClusterSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.atomix 2 | 3 | import java.nio.file.Files 4 | 5 | import com.typesafe.config.ConfigFactory 6 | import in.ashwanthkumar.suuchi.cluster.{ 7 | Cluster, 8 | ClusterProvider, 9 | InMemorySeedProvider, 10 | MemberAddress 11 | } 12 | import org.apache.commons.io.FileUtils 13 | import org.scalatest.{BeforeAndAfter, FlatSpec} 14 | import org.scalatest.Matchers.{convertToAnyShouldWrapper, have} 15 | 16 | class AtomixClusterSpec extends FlatSpec with BeforeAndAfter { 17 | 18 | val BASE_PORT = 60000 19 | val raftDir = Files.createTempDirectory("suuchi-membership-it") 20 | 21 | var members: List[Cluster] = List() 22 | 23 | after { 24 | members.foreach(_.stop()) 25 | FileUtils.deleteDirectory(raftDir.toFile) 26 | } 27 | 28 | def atomixConfig(port: Int) = 29 | ConfigFactory.parseString(s""" 30 | |atomix { 31 | | port = $port # port used by atomix for cluster membership communication 32 | | working-dir = "${raftDir.toString}" # location used for storing raft logs 33 | | # cluster identifier to make sure all nodes are taking part in the right cluster. 34 | | # You can also use environment specific identifiers to differentiate them. 35 | | cluster-id = "suuchi-atomix-test-group" 36 | | rpc-port = 8080 # port used for gRPC communication 37 | |} 38 | """.stripMargin) 39 | 40 | "Membership" should "launch 5 nodes and say they have 5 nodes" in { 41 | val bootstrapper = InMemorySeedProvider(List(MemberAddress("localhost", BASE_PORT + 1))) 42 | (1 to 5).foreach { i => 43 | val memberPort = BASE_PORT + i 44 | // val member = new AtomixCluster("localhost", memberPort, memberPort, raftDir.toString, "succhi-test-group", ConfigFactory.load()) 45 | val member = ClusterProvider.apply(MemberAddress("localhost", memberPort), 46 | atomixConfig(memberPort), 47 | Nil) 48 | if (i > 1) { 49 | members = members ++ List(member.start(bootstrapper)) 50 | } else { 51 | members = members ++ List(member.start(InMemorySeedProvider(List()))) 52 | } 53 | } 54 | members.map(m => (m.nodes, m.whoami)).foreach(println) 55 | val totalNodes = members.head.nodes 56 | totalNodes should have size 5 57 | } 58 | 59 | } 60 | -------------------------------------------------------------------------------- /suuchi-examples/src/main/scala/in/ashwanthkumar/suuchi/client/SuuchiClient.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.client 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import com.google.protobuf.ByteString 6 | import in.ashwanthkumar.suuchi.examples.rpc.generated._ 7 | import io.grpc.netty.NettyChannelBuilder 8 | import org.slf4j.LoggerFactory 9 | 10 | class SuuchiClient(host: String, port: Int) { 11 | private val log = LoggerFactory.getLogger(getClass) 12 | 13 | private val channel = NettyChannelBuilder 14 | .forAddress(host, port) 15 | .usePlaintext(true) 16 | .build() 17 | 18 | private val writeStub = PutGrpc.blockingStub(channel) 19 | private val readStub = ReadGrpc.blockingStub(channel) 20 | private val scanStub = ScanGrpc.blockingStub(channel) 21 | private val aggStub = AggregatorGrpc.blockingStub(channel) 22 | 23 | def shutdown() = { 24 | channel.awaitTermination(5, TimeUnit.SECONDS) 25 | } 26 | 27 | def put(key: Array[Byte], value: Array[Byte]): Boolean = { 28 | log.info(s"Doing Put with key=${new String(key)} value=${new String(value)}") 29 | val request = PutRequest(key = ByteString.copyFrom(key), value = ByteString.copyFrom(value)) 30 | writeStub.put(request).status 31 | } 32 | 33 | def get(key: Array[Byte]): Option[Array[Byte]] = { 34 | log.info("Doing Get with key={}", new String(key)) 35 | val request = GetRequest(key = ByteString.copyFrom(key)) 36 | 37 | val response = readStub.get(request) 38 | if (response.value.isEmpty) { 39 | None 40 | } else { 41 | Some(response.value.toByteArray) 42 | } 43 | } 44 | 45 | def scan() = { 46 | scanStub.scan(ScanRequest(start = Int.MinValue, end = Int.MaxValue)) 47 | } 48 | 49 | def sumOfNumbers() = { 50 | aggStub.aggregate(AggregateRequest()) 51 | } 52 | } 53 | 54 | object SuuchiClient extends App { 55 | private val log = LoggerFactory.getLogger(getClass) 56 | val client = new SuuchiClient("localhost", 5051) 57 | 58 | (0 until 5).foreach { index => 59 | val status = client.put(Array((65 + index).toByte), Array((65 + index).toByte)) 60 | log.info("Put Status={}", status) 61 | } 62 | 63 | (0 until 5).foreach { index => 64 | val value = client.get(Array((65 + index).toByte)) 65 | log.info("Got value={}", new String(value.get)) 66 | } 67 | 68 | (0 to 5).foreach { i => 69 | client.put(s"prefix/$i".getBytes, s"$i".getBytes) 70 | } 71 | 72 | val iterator = client.scan() 73 | 74 | iterator.foreach { response => 75 | println(new String(response.getKv.key.toByteArray)) 76 | } 77 | 78 | println(client.sumOfNumbers) 79 | 80 | client.shutdown() 81 | } 82 | -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/src/test/scala/in/ashwanthkumar/suuchi/cluster/scalecube/ScaleCubeClusterIT.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.scalecube 2 | 3 | import java.util.concurrent._ 4 | 5 | import com.typesafe.config.ConfigFactory 6 | import in.ashwanthkumar.suuchi.cluster._ 7 | import org.scalatest.Matchers.{convertToAnyShouldWrapper, have} 8 | import org.scalatest.{BeforeAndAfter, FlatSpec} 9 | 10 | case class ExpectedMemberCount(joinLatch: Option[CountDownLatch] = None, 11 | removeLatch: Option[CountDownLatch] = None) 12 | extends MemberListener { 13 | 14 | /** 15 | * Triggered when a node represented by [[MemberAddress]] is added to the cluster 16 | */ 17 | override def onJoin: (MemberAddress) => Unit = _ => joinLatch.foreach(_.countDown()) 18 | 19 | /** 20 | * Triggered when a node represented by [[MemberAddress]] is removed from the cluster 21 | */ 22 | override def onLeave: (MemberAddress) => Unit = _ => removeLatch.foreach(_.countDown()) 23 | } 24 | 25 | class ScaleCubeClusterIT extends FlatSpec with BeforeAndAfter { 26 | val BASE_PORT = 20000 27 | var members: List[Cluster] = List() 28 | val latch = new CountDownLatch(4) // at least 4 members should have joined 29 | 30 | after { 31 | members.foreach(_.stop()) 32 | } 33 | 34 | "ScaleCubeCluster" should "launch 5 nodes and say they have 5 nodes" in { 35 | val seedNode = new ScaleCubeCluster(configWithPort(BASE_PORT + 1), 36 | List( 37 | ExpectedMemberCount(joinLatch = Some(latch)) 38 | )) 39 | members = List(seedNode.start(InMemorySeedProvider(List()))) 40 | 41 | (2 to 5).foreach { i => 42 | val bootstrapper = InMemorySeedProvider(List(seedNode.whoami)) 43 | val member = ClusterProvider.apply(MemberAddress("localhost", BASE_PORT + i), 44 | configWithPort(BASE_PORT + i), 45 | Nil) 46 | members = members ++ List(member.start(bootstrapper)) 47 | } 48 | latch.await(10, TimeUnit.SECONDS) // wait until all nodes have contacted with the seed node 49 | 50 | members.map(m => m.nodes).foreach(println) 51 | val totalNodes = members.head.nodes 52 | totalNodes should have size 5 53 | } 54 | 55 | def configWithPort(port: Int = BASE_PORT) = 56 | ConfigFactory.parseString(s""" 57 | |scalecube { 58 | | port = $port 59 | | gossip { 60 | | interval = 200 61 | | fanout = 5 62 | | } 63 | |} 64 | """.stripMargin) 65 | 66 | } 67 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | [![Build Status](https://travis-ci.org/ashwanthkumar/suuchi.svg?branch=master)](https://travis-ci.org/ashwanthkumar/suuchi) 2 | [![codecov](https://codecov.io/gh/ashwanthkumar/suuchi/branch/master/graph/badge.svg)](https://codecov.io/gh/ashwanthkumar/suuchi) 3 | 4 | # Suuchi - सूचि 5 | 6 | Having inspired from tools like [Uber's Ringpop](https://ringpop.readthedocs.io/) and a strong desire to understand how distributed systems work - Suuchi was born. 7 | 8 | Suuchi is toolkit to build distributed data systems, that uses [gRPC](http://www.grpc.io/) under the hood as the communication medium. The overall goal of this project is to build pluggable components that can be easily composed by the developer to build a data system of desired characteristics. 9 | 10 | > This project is in beta quality and it's currently running couple of systems in production setting [@indix](https://twitter.com/indix). We welcome all kinds of feedback to help improve the library. 11 | 12 | Read the Documentation at [http://ashwanthkumar.github.io/suuchi](http://ashwanthkumar.github.io/suuchi). 13 | 14 | Suuchi in sanskrit means an Index[1](http://spokensanskrit.de/index.php?tinput=sUci&direction=SE&script=HK&link=yes&beginning=0). 15 | 16 | ## Presentations 17 | Following presentations / videos explain motivation behind Suuchi 18 | 19 | - [Why we built a distributed system](https://speakerdeck.com/ashwanthkumar/why-we-built-a-distributed-system-dsconf-2018) at DSConf 2018. 20 | - Video by [@brewkode](https://twitter.com/brewkode) on [Suuchi - Toolkit to build distributed systems](https://www.youtube.com/watch?v=GK0-ICFvIGw) at Fifth Elephant, 2017. 21 | - [Suuchi - Distributed Systems Primitives](https://speakerdeck.com/ashwanthkumar/suuchi-distributed-system-primitives) 22 | - [Suuchi - Application Layer Sharding](https://speakerdeck.com/ashwanthkumar/suuchi-application-layer-sharding) 23 | - [Suuchi - Distributed Data Systems Toolkit](https://speakerdeck.com/ashwanthkumar/suuchi-distributed-data-systems-toolkit/) 24 | 25 | ## Notes 26 | If you're getting `ClassNotFound` exception, please run `mvn clean compile` once to generate from the java classes from protoc files. Also, if you're using IntelliJ it helps to close the project when running the above command. It seems to auto-detect sources in `target/` at startup but not afterwards. 27 | 28 | ## Release workflow 29 | Suuchi and it's modules follow a git commit message based release workflow. Use the script `make-release.sh` to push an empty commit to the repository which would trigger a release workflow on travis-ci. More information can be found at [docs](https://ashwanthkumar.github.io/suuchi/developer/workflow/). 30 | 31 | ## License 32 | https://www.apache.org/licenses/LICENSE-2.0 33 | -------------------------------------------------------------------------------- /suuchi-examples/src/test/scala/in/ashwanthkumar/suuchi/rpc/SuuchiScanServiceTest.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import in.ashwanthkumar.suuchi.examples.rpc.generated.{ScanRequest, ScanResponse} 4 | import in.ashwanthkumar.suuchi.store.InMemoryStore 5 | import io.grpc.stub.ServerCallStreamObserver 6 | import org.mockito.ArgumentCaptor 7 | import org.mockito.Mockito._ 8 | import org.scalatest.FlatSpec 9 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper, have} 10 | 11 | import scala.collection.JavaConversions._ 12 | 13 | class SuuchiScanServiceTest extends FlatSpec { 14 | 15 | "SuuchiScanService" should "support scan for a given token range" in { 16 | 17 | val service = new SuuchiScanService(getPopulatedStore(10)) 18 | 19 | val request = ScanRequest(start = Integer.MIN_VALUE, end = Integer.MAX_VALUE) 20 | 21 | val observer = mock(classOf[ServerCallStreamObserver[ScanResponse]]) 22 | val runnable = ArgumentCaptor.forClass(classOf[Runnable]) 23 | when(observer.isReady).thenReturn(true) 24 | service.scan(request, observer) 25 | verify(observer, times(1)).setOnReadyHandler(runnable.capture()) 26 | runnable.getValue.run() // run the stream observer 27 | 28 | val captor = ArgumentCaptor.forClass(classOf[ScanResponse]) 29 | val values = captor.getAllValues 30 | verify(observer, times(10)).onNext(captor.capture()) 31 | verify(observer, times(1)).onCompleted() 32 | 33 | values should have size 10 34 | values.toList.map(extractKey).toSet should be(1 to 10 toSet) 35 | } 36 | 37 | it should "not include key which are out of the given token range" in { 38 | val service = new SuuchiScanService(getPopulatedStore(10)) 39 | val request = ScanRequest(start = 1, end = 10) 40 | 41 | val observer = mock(classOf[ServerCallStreamObserver[ScanResponse]]) 42 | val runnable = ArgumentCaptor.forClass(classOf[Runnable]) 43 | when(observer.isReady).thenReturn(true) 44 | service.scan(request, observer) 45 | verify(observer, times(1)).setOnReadyHandler(runnable.capture()) 46 | runnable.getValue.run() // run the stream observer 47 | 48 | val captor = ArgumentCaptor.forClass(classOf[ScanResponse]) 49 | val values = captor.getAllValues 50 | when(observer.isReady).thenReturn(true) 51 | 52 | verify(observer, times(0)).onNext(captor.capture()) 53 | verify(observer, times(1)).onCompleted() 54 | values should have size 0 55 | } 56 | 57 | private def getPopulatedStore(num: Int) = { 58 | val store = new InMemoryStore 59 | 1 to num foreach (i => store.put(i.toString.getBytes, (i * 100).toString.getBytes)) 60 | store 61 | } 62 | 63 | private def extractKey(response: ScanResponse) = 64 | new String(response.getKv.key.toByteArray).toInt 65 | } 66 | -------------------------------------------------------------------------------- /docs/internals/aggregation.md: -------------------------------------------------------------------------------- 1 | # Aggregation 2 | 3 | _Available since version 0.2.21 onwards_ 4 | 5 | `Aggregation` is a special type of router used to perform aggregation type of operation across all the nodes in the cluster. 6 | It is used to fan-out requests to all the nodes in the cluster, collect the response and aggregate their the responses. 7 | 8 | Aggregations can be used with all functions that exhibit both associative and commutative property like Sum / Product / 9 | TopK etc. It's conceptually similar to doing reduce individually on all the nodes and doing a global reduction on those 10 | reduced results. 11 | 12 | ## Implementation Details 13 | 14 | Aggregation in Suuchi makes use of Twitter [Algebird's Aggregator](https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Aggregator.scala) 15 | to represent how we can aggregate the results of all service calls. In short Aggregation is presented as a `PartialFunction[MethodDescriptor[ReqT, RespT], Aggregator[RespT, Any, RespT]]`. 16 | 17 | Example of an Aggregation that represent SumOfNumbers can be defined as follows 18 | 19 | ```scala 20 | class SumOfNumbers extends Aggregation { 21 | override def aggregator[ReqT, RespT] = { 22 | case AggregatorGrpc.METHOD_AGGREGATE => new Aggregator[Response, Long, Response] { 23 | override def prepare(input: Response): Long = input.getOutput 24 | override def semigroup: Semigroup[Long] = LongRing 25 | override def present(reduced: Long): Response = Response.newBuilder().setOutput(reduced).build() 26 | }.asInstanceOf[Aggregator[RespT, Any, RespT]] 27 | } 28 | } 29 | ``` 30 | 31 | We compose this `Aggregation` with `Server` abstraction as follows 32 | 33 | ```scala 34 | Server(...) 35 | .aggregate(allNodes, new SuuchiAggregatorService(new SumOfNumbers), new SumOfNumbers) 36 | .start() 37 | ``` 38 | 39 | `SuuchiAggregatorService` filters all even numbers from the store and does a local aggregation of the sum. These sums 40 | are then globally summed again at the co-ordinator (node that recieves the request for aggregation) node and the result 41 | is sent back as a response back to the client. 42 | 43 | ## Distributed Sum Example 44 | 45 | Let's consider an example where we would like to find a sum of all even numbers we have on each node. The entire flow 46 | of data on each node and the co-ordinator node is depicted below in the diagram. 47 | 48 | 49 | ![Distributed Sum Visualization](/images/internals/aggregation_viz.png) 50 | 51 | Assume we've 4 nodes Node A - D, and each of them contain a set of numbers with them. The cost of doing filter and sum 52 | on each node if very efficient then returning all the numbers to a single node, filtering them and then computing the sum. 53 | This in traditional computer architectural terms is called as function shipping paradigm, very similar to stored procedures 54 | in RDBMS. 55 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/rpc/CachedChannelPoolSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.rpc 2 | 3 | import java.util.concurrent.{TimeUnit, ConcurrentHashMap} 4 | 5 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 6 | import io.grpc.{Channel, ManagedChannelBuilder, ManagedChannel} 7 | import org.scalatest.FlatSpec 8 | import org.scalatest.Matchers.{convertToAnyShouldWrapper, be, have, size} 9 | import org.mockito.Mockito._ 10 | import org.mockito.Matchers._ 11 | 12 | class MockCachedPool(map: ConcurrentHashMap[String, ManagedChannel], 13 | mockBuilder: ManagedChannelBuilder[_ <: ManagedChannelBuilder[_]]) 14 | extends CachedChannelPool(map) { 15 | override private[rpc] def builderFrom(key: String): ManagedChannelBuilder[_] = mockBuilder 16 | } 17 | 18 | class CachedChannelPoolSpec extends FlatSpec { 19 | "CachedChannelPool" should "build a new channel if no cache is available" in { 20 | val mockChannel = mock(classOf[ManagedChannel]) 21 | val builder = mock(classOf[ManagedChannelBuilder[_ <: ManagedChannelBuilder[_]]]) 22 | when(builder.build()).thenReturn(mockChannel) 23 | 24 | val map = new ConcurrentHashMap[String, ManagedChannel]() 25 | val pool = new MockCachedPool(map, builder) 26 | map should have size 0 27 | val channel = pool.get(MemberAddress("host1", 1)) 28 | map should have size 1 29 | channel.shutdown() 30 | 31 | verify(mockChannel, times(1)).shutdown() 32 | verify[ManagedChannelBuilder[_]](builder, times(1)).build() 33 | } 34 | 35 | it should "build a new channel using plainText if insecure=true" in { 36 | val mockChannel = mock(classOf[ManagedChannel]) 37 | val builder = mock(classOf[ManagedChannelBuilder[_ <: ManagedChannelBuilder[_]]]) 38 | when(builder.build()).thenReturn(mockChannel) 39 | 40 | val map = new ConcurrentHashMap[String, ManagedChannel]() 41 | val pool = new MockCachedPool(map, builder) 42 | map should have size 0 43 | val channel = pool.get(MemberAddress("host1", 1), insecure = true) 44 | map should have size 1 45 | channel.shutdown() 46 | 47 | verify[ManagedChannelBuilder[_]](builder, times(1)).usePlaintext(true) 48 | verify[ManagedChannelBuilder[_]](builder, times(1)).build() 49 | verify(mockChannel, times(1)).shutdown() 50 | } 51 | 52 | it should "return the same channel if it's already cached" in { 53 | val mockChannel = mock(classOf[ManagedChannel]) 54 | val builder = mock(classOf[ManagedChannelBuilder[_ <: ManagedChannelBuilder[_]]]) 55 | when(builder.build()).thenReturn(mockChannel) 56 | 57 | val map = new ConcurrentHashMap[String, ManagedChannel]() 58 | val pool = new MockCachedPool(map, builder) 59 | map should have size 0 60 | val channel = pool.get(MemberAddress("host1", 1)) 61 | map should have size 1 62 | val anotherChannel = pool.get(MemberAddress("host1", 1)) 63 | map should have size 1 64 | 65 | verify[ManagedChannelBuilder[_]](builder, times(1)).build() 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /project/Dependencies.scala: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | 3 | object Dependencies { 4 | val scalaTest = "org.scalatest" %% "scalatest" % "3.0.0" % Test 5 | val mockito = "org.mockito" % "mockito-core" % "1.10.19" % Test 6 | 7 | val algebird = "com.twitter" %% "algebird-core" % "0.13.0" 8 | val bijection = "com.twitter" %% "bijection-core" % "0.9.5" 9 | 10 | val joda = "joda-time" % "joda-time" % "2.8.2" 11 | val hocon = "com.typesafe" % "config" % "1.2.1" 12 | val commonsIO = "commons-io" % "commons-io" % "2.5" 13 | 14 | val grpcVersion = "1.2.0" 15 | val grpcNetty = "io.grpc" % "grpc-netty" % grpcVersion 16 | val grpcStub = "io.grpc" % "grpc-stub" % grpcVersion 17 | val grpcCore = "io.grpc" % "grpc-core" % grpcVersion 18 | val grpcProtobuf = "io.grpc" % "grpc-protobuf" % grpcVersion 19 | val grpcServices = "io.grpc" % "grpc-services" % grpcVersion 20 | val grpcTesting = "io.grpc" % "grpc-testing" % grpcVersion % Test 21 | val grpc = Seq(grpcNetty, grpcStub, grpcCore, grpcProtobuf, grpcServices, grpcTesting) 22 | 23 | val nettyVersion = "4.1.8.Final" 24 | val nettyCodec = "io.netty" % "netty-codec" % nettyVersion 25 | val nettyCommon = "io.netty" % "netty-common" % nettyVersion 26 | val nettyTransport = "io.netty" % "netty-transport" % nettyVersion 27 | val nettyHandler = "io.netty" % "netty-handler" % nettyVersion 28 | val netty = Seq(nettyCodec, nettyCommon, nettyTransport, nettyHandler) 29 | 30 | val sbProtoRuntime = "com.trueaccord.scalapb" %% "scalapb-runtime" % com.trueaccord.scalapb.compiler.Version.scalapbVersion 31 | val sbGrpcRuntime = "com.trueaccord.scalapb" %% "scalapb-runtime-grpc" % com.trueaccord.scalapb.compiler.Version.scalapbVersion 32 | val scalaPB = Seq(sbProtoRuntime, sbGrpcRuntime) 33 | 34 | val slf4j = "org.slf4j" % "slf4j-api" % "1.7.12" 35 | 36 | val log4jVersion = "2.6.2" 37 | val log4jCore = "org.apache.logging.log4j" % "log4j-core" % log4jVersion 38 | val log4jApi = "org.apache.logging.log4j" % "log4j-api" % log4jVersion 39 | val log4jOverSlf4j = "org.apache.logging.log4j" % "log4j-slf4j-impl" % log4jVersion 40 | val log4j = Seq(log4jCore, log4jApi, log4jOverSlf4j) 41 | 42 | val rocksDBjni = "org.rocksdb" % "rocksdbjni" % "5.8.6" 43 | 44 | val atomixVersion = "1.0.6" 45 | val catalystVersion = "1.2.0" 46 | val atomixCore = "io.atomix" % "atomix" % atomixVersion 47 | val atomixResource = "io.atomix" % "atomix-resource" % atomixVersion 48 | val catalystTransport = "io.atomix.catalyst" % "catalyst-transport" % catalystVersion 49 | val catalystNetty = "io.atomix.catalyst" % "catalyst-netty" % catalystVersion 50 | val atomix = Seq(atomixCore, atomixResource, catalystTransport, catalystNetty) 51 | 52 | val scalecube = "io.scalecube" % "scalecube-cluster" % "0.9.0" 53 | val rxScala = "io.reactivex" %% "rxscala" % "0.26.2" 54 | 55 | val testDeps = Seq(scalaTest, mockito) 56 | 57 | lazy val coreDependencies = Seq(joda, slf4j, hocon, commonsIO, algebird) ++ scalaPB ++ grpc ++ netty ++ testDeps 58 | 59 | lazy val rocksDBDependencies = Seq(rocksDBjni) ++ testDeps 60 | 61 | lazy val examplesDependencies = Seq(slf4j) ++ log4j ++ testDeps 62 | 63 | lazy val atomixDependencies = atomix ++ testDeps 64 | 65 | lazy val scalecubeDependencies = Seq(scalecube, rxScala) ++ testDeps 66 | 67 | } 68 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/ShardedStore.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.util.concurrent.ConcurrentHashMap 4 | 5 | import in.ashwanthkumar.suuchi.partitioner.Hash 6 | import in.ashwanthkumar.suuchi.utils.Logging 7 | 8 | import scala.collection.JavaConversions._ 9 | 10 | /** 11 | * SharedStore shards the keys equally into [[partitionsPerNode]] stores and proxies store operations 12 | * against them for a given key. 13 | *

14 | * 15 | * DO NOT CHANGE THE [[hashFn]] and [[partitionsPerNode]] on an existing store, 16 | * we wouldn't be able to read previously hashed keys. 17 | * 18 | * TODO - May be build a tool that can migrate between the shards (hashes and number) 19 | * by doing a full scan of the underlying store and re-creating the data. 20 | * 21 | * @param partitionsPerNode Number of stores we need to shard the data into 22 | * @param hashFn HashFunction used to compute the shard 23 | * @param createStore Function to return a store instance given a partitionId. 24 | * This would be created in the form [1 -> partitionsPerNode]. 25 | * Take care to not throw exceptions in this method. If it does, 26 | * we propagate that error back to the service who invoked us. 27 | */ 28 | class ShardedStore(partitionsPerNode: Int, hashFn: Hash, createStore: (Int) => Store) 29 | extends Store 30 | with Logging { 31 | private val map = new ConcurrentHashMap[Integer, Store](partitionsPerNode) 32 | 33 | private val locks = Array.fill(partitionsPerNode)(new Object) 34 | 35 | override def get(key: Array[Byte]): Option[Array[Byte]] = 36 | logOnError(() => getStore(key).get(key)).getOrElse(None) 37 | override def put(key: Array[Byte], value: Array[Byte]): Boolean = 38 | logOnError(() => getStore(key).put(key, value)).isSuccess 39 | override def remove(key: Array[Byte]): Boolean = 40 | logOnError(() => getStore(key).remove(key)).isSuccess 41 | 42 | override def scanner(): Scanner[KV] = new Scanner[KV] { 43 | initializeStoresIfNot() 44 | private[this] lazy val scanners = map.values().map(_.scanner()) 45 | 46 | override def prepare(): Unit = { 47 | scanners.foreach(_.prepare()) 48 | } 49 | 50 | override def scan(prefix: Array[Byte]): Iterator[KV] = scanners.iterator.flatMap(_.scan(prefix)) 51 | 52 | override def scan(): Iterator[KV] = scanners.iterator.flatMap(_.scan()) 53 | 54 | override def close(): Unit = scanners.foreach(_.close()) 55 | } 56 | 57 | protected def getStore(key: Array[Byte]): Store = { 58 | openStore(getPartition(key)) 59 | } 60 | 61 | protected def getPartition(key: Array[Byte]): Int = { 62 | math.abs(hashFn.hash(key)) % partitionsPerNode 63 | } 64 | 65 | protected def openStore(partition: Int): Store = { 66 | if (map.containsKey(partition)) { 67 | map.get(partition) 68 | } else { 69 | locks(partition).synchronized { 70 | // To trade off between locks for most of the code path and consistency in not invoking `createStore` 71 | // unless absolutely we need it. 72 | if (!map.containsKey(partition)) { 73 | val store = logOnError(() => createStore(partition)).get 74 | map.put(partition, store) 75 | store 76 | } else { 77 | map.get(partition) 78 | } 79 | } 80 | } 81 | } 82 | 83 | protected def initializeStoresIfNot() = { 84 | if (map.size() != partitionsPerNode) (0 until partitionsPerNode).foreach(openStore) 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /suuchi-cluster-scalecube/src/main/scala/in/ashwanthkumar/suuchi/cluster/scalecube/ScaleCubeCluster.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.scalecube 2 | 3 | import com.typesafe.config.Config 4 | import in.ashwanthkumar.suuchi.cluster.{ 5 | MemberAddress, 6 | MemberListener, 7 | SeedProvider, 8 | Cluster => SuuchiCluster 9 | } 10 | import io.scalecube.cluster.gossip.GossipConfig 11 | import io.scalecube.cluster.membership.{MembershipConfig, MembershipEvent} 12 | import io.scalecube.cluster.{Cluster, ClusterConfig, ICluster} 13 | import io.scalecube.transport.{Address, TransportConfig} 14 | import org.slf4j.LoggerFactory 15 | import rx.lang.scala.ImplicitFunctionConversions._ 16 | 17 | import scala.collection.JavaConversions._ 18 | import scala.language.implicitConversions 19 | 20 | case class ScaleCubeConfig(port: Int, gossipConfig: Option[GossipConfig]) 21 | object ScaleCubeConfig { 22 | def apply(config: Config): ScaleCubeConfig = { 23 | val scalecube = config.getConfig("scalecube") 24 | ScaleCubeConfig( 25 | port = scalecube.getInt("port"), 26 | gossipConfig = toGossipConfig(scalecube) 27 | ) 28 | } 29 | 30 | private[this] def toGossipConfig(scalecube: Config): Option[GossipConfig] = { 31 | if (scalecube.hasPath("gossip")) { 32 | val gConfig = scalecube.getConfig("gossip") 33 | val gossipConfigBuilder = GossipConfig.builder() 34 | if (gConfig.hasPath("interval")) { 35 | gossipConfigBuilder.gossipInterval(gConfig.getInt("interval")) 36 | } 37 | if (gConfig.hasPath("fanout")) { 38 | gossipConfigBuilder.gossipFanout(gConfig.getInt("fanout")) 39 | } 40 | Some(gossipConfigBuilder.build()) 41 | } else None 42 | } 43 | 44 | } 45 | 46 | class ScaleCubeCluster(clusterConfig: Config, listeners: List[MemberListener]) 47 | extends SuuchiCluster(clusterConfig, listeners) { 48 | protected val log = LoggerFactory.getLogger(getClass) 49 | lazy val config = ScaleCubeConfig.apply(clusterConfig) 50 | 51 | var cluster: ICluster = _ 52 | 53 | override def start(seedProvider: SeedProvider): SuuchiCluster = { 54 | val clusterConfig = ClusterConfig 55 | .builder() 56 | .transportConfig( 57 | TransportConfig 58 | .builder() 59 | .port(config.port) 60 | .build() 61 | ) 62 | config.gossipConfig.foreach(clusterConfig.gossipConfig) 63 | if (seedProvider.nodes.isEmpty) { 64 | cluster = Cluster.joinAwait(clusterConfig.build()) 65 | } else { 66 | val seedNodes = seedProvider.nodes.map(m => Address.create(m.host, m.port)) 67 | cluster = Cluster.joinAwait( 68 | clusterConfig 69 | .membershipConfig(MembershipConfig.builder.seedMembers(seedNodes).build) 70 | .build() 71 | ) 72 | } 73 | cluster 74 | .listenMembership() 75 | .filter({ m: MembershipEvent => 76 | m.isAdded 77 | }) 78 | .map[MemberAddress]({ m: MembershipEvent => 79 | MemberAddress(m.member().address().toString) 80 | }) 81 | .subscribe(this.onJoin) 82 | 83 | cluster 84 | .listenMembership() 85 | .filter({ m: MembershipEvent => 86 | m.isRemoved 87 | }) 88 | .map[MemberAddress]({ m: MembershipEvent => 89 | MemberAddress(m.member().address().toString) 90 | }) 91 | .subscribe(this.onLeave) 92 | this 93 | } 94 | override def stop(): Unit = cluster.shutdown().get() 95 | override def nodes: Iterable[MemberAddress] = 96 | cluster.members().map(m => MemberAddress(m.address().toString)) 97 | override def whoami: MemberAddress = MemberAddress(cluster.address().toString) 98 | } 99 | -------------------------------------------------------------------------------- /suuchi-examples/_pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | suuchi-examples 4 | 0.3.6-SNAPSHOT 5 | Suuchi Examples 6 | Example modules using Suuchi 7 | jar 8 | 9 | 10 | in.ashwanthkumar 11 | suuchi 12 | 0.3.6-SNAPSHOT 13 | 14 | 15 | 16 | 2.6.2 17 | 18 | 19 | 20 | 21 | in.ashwanthkumar 22 | suuchi-core 23 | ${project.version} 24 | 25 | 26 | 27 | in.ashwanthkumar 28 | suuchi-rocksdb 29 | ${project.version} 30 | 31 | 32 | 33 | io.grpc 34 | grpc-netty 35 | ${grpc.version} 36 | 37 | 38 | org.apache.logging.log4j 39 | log4j-core 40 | ${log4j.version} 41 | 42 | 43 | org.apache.logging.log4j 44 | log4j-slf4j-impl 45 | ${log4j.version} 46 | 47 | 48 | 49 | org.mockito 50 | mockito-core 51 | 1.10.19 52 | test 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | kr.motd.maven 61 | os-maven-plugin 62 | 1.4.1.Final 63 | 64 | 65 | 66 | 67 | org.xolstice.maven.plugins 68 | protobuf-maven-plugin 69 | 0.5.0 70 | 71 | 76 | com.google.protobuf:protoc:3.0.2:exe:${os.detected.classifier} 77 | grpc-java 78 | io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} 79 | 80 | 81 | 82 | 83 | 84 | compile 85 | compile-custom 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/cluster/Cluster.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster 2 | 3 | import com.typesafe.config.Config 4 | 5 | /** 6 | * Listeners are notified whenever there are new nodes joining the cluster 7 | * or existing nodes removed from the cluster 8 | */ 9 | trait MemberListener { 10 | 11 | /** 12 | * Triggered when a node represented by [[MemberAddress]] is added to the cluster 13 | */ 14 | def onJoin: MemberAddress => Unit 15 | 16 | /** 17 | * Triggered when a node represented by [[MemberAddress]] is removed from the cluster 18 | */ 19 | def onLeave: MemberAddress => Unit 20 | } 21 | 22 | /** 23 | * [[Cluster]] implementation takes care of forming a inter node system (aka) distributed, 24 | * which helps in ops like 25 | * 26 | *

    27 | *
  1. Helps identify if a node is up or down
  2. 28 | *
  3. Dynamically scale up and down your system
  4. 29 | *
  5. Supports custom Listeners who'll be notified when a node joins / exits the cluster
  6. 30 | *
31 | * 32 | * @param listeners List[MemberListener] who would be notified about changes in cluster membership 33 | */ 34 | abstract class Cluster(config: Config, listeners: List[MemberListener]) { 35 | 36 | /** 37 | * Start / Join a given cluster instance, given a [[SeedProvider]] instance 38 | * to identify the initial list of nodes. If you're starting a single node 39 | * cluster, consider using [[InMemorySeedProvider.EMPTY]]. 40 | * 41 | * The List[MemberAddress] in SeedProvider.nodes represents the address of the cluster 42 | * communication and not that of the gRPC service. While you can technically still use the 43 | * existing gRPC as an underlying transport, but it would be a little hard for us to 44 | * find / build implementations that does it. Hence we assume the [[Cluster]] implementations 45 | * are free to choose any transport they deem fit for the type of cluster membership they provide. 46 | * 47 | * Another reason for making the assumption is systems like Apache Gossip (http://gossip.incubator.apache.org/) 48 | * uses UDP based transport while gRPC needs a reliable transport and hence uses TCP. 49 | * 50 | * @param seedProvider 51 | * @return 52 | */ 53 | def start(seedProvider: SeedProvider): Cluster 54 | 55 | /** 56 | * Stop and release this cluster related resources. This could mean sending 57 | * a LEAVE message and/or stop underlying transport for sending and receiving 58 | * messages. 59 | */ 60 | def stop(): Unit 61 | 62 | /** 63 | * Once this [[Cluster]] has been started using [[Cluster.start(SeedProvider)]] this method 64 | * should return list of all the nodes that are current part of this cluster. 65 | * 66 | * Depending on the type of [[Cluster]] implementation, it might take a while to converge 67 | * and report a correct number, but the application should not make any assumptions on that. 68 | * Any changes to the members in this [[Cluster]] will be notified to all [[MemberListener]] 69 | * objects. 70 | * 71 | * @return List[MemberAddress] of all the nodes in this cluster. 72 | */ 73 | def nodes: Iterable[MemberAddress] 74 | 75 | /** 76 | * Address of the current node when it is part of the cluster 77 | * @return 78 | */ 79 | def whoami: MemberAddress 80 | 81 | /** 82 | * Handler for Membership implementations to bind when new members join the cluster. 83 | */ 84 | final protected def onJoin: MemberAddress => Unit = m => listeners.foreach(_.onJoin(m)) 85 | 86 | /** 87 | * Handler for Membership implementations to bind when new members leave the cluster 88 | * or a FailureDetector implementation has detected the node as un-reachable. Either 89 | * ways the node can't participate in regular activity of the cluster and has to be 90 | * removed from it's duty. 91 | */ 92 | final protected def onLeave: MemberAddress => Unit = m => listeners.foreach(_.onLeave(m)) 93 | } 94 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/router/AggregationRouter.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import java.util 4 | import java.util.concurrent.TimeUnit 5 | 6 | import com.google.common.util.concurrent.Futures 7 | import com.twitter.algebird.Aggregator 8 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 9 | import in.ashwanthkumar.suuchi.rpc.CachedChannelPool 10 | import io.grpc._ 11 | import io.grpc.stub.{ClientCalls, MetadataUtils, StreamObserver, StreamObservers} 12 | import org.slf4j.LoggerFactory 13 | 14 | import scala.collection.JavaConverters._ 15 | 16 | trait Aggregation { 17 | def aggregator[ReqT, RespT]: PartialFunction[MethodDescriptor[ReqT, RespT], Aggregator[RespT, Any, RespT]] 18 | } 19 | 20 | class AggregationRouter(members: List[MemberAddress], agg: Aggregation) extends ServerInterceptor { 21 | val channelPool = CachedChannelPool() 22 | val log = LoggerFactory.getLogger(classOf[AggregationRouter]) 23 | 24 | override def interceptCall[ReqT, RespT](incomingRequest: ServerCall[ReqT, RespT], headers: Metadata, next: ServerCallHandler[ReqT, RespT]): ServerCall.Listener[ReqT] = { 25 | val isBroadcastRequest = headers.containsKey(Headers.BROADCAST_REQUEST_KEY) 26 | if (isBroadcastRequest || !agg.aggregator.isDefinedAt(incomingRequest.getMethodDescriptor)) { 27 | next.startCall(incomingRequest, headers) 28 | } else { 29 | // ServerCall.Listener for ServerStreaming methods 30 | headers.put(Headers.BROADCAST_REQUEST_KEY, true) 31 | incomingRequest.request(2) 32 | new ServerCall.Listener[ReqT] { 33 | val aggregator = agg.aggregator.apply(incomingRequest.getMethodDescriptor) 34 | var request: ReqT = _ 35 | 36 | override def onCancel() = { 37 | log.debug("AggregationRouter#onCancel") 38 | incomingRequest.close(Status.CANCELLED, headers) 39 | } 40 | override def onHalfClose() = { 41 | log.debug("AggregationRouter#onHalfClose") 42 | try { 43 | val gathered = scatter(members, channelPool, incomingRequest.getMethodDescriptor, headers, request) 44 | val reduced = aggregator.apply(gathered.asScala) 45 | incomingRequest.sendHeaders(headers) 46 | incomingRequest.sendMessage(reduced) 47 | incomingRequest.close(Status.OK, headers) 48 | } catch { 49 | case e: Throwable => 50 | log.error(e.getMessage, e) 51 | incomingRequest.close(Status.INTERNAL.withCause(e), headers) 52 | } 53 | } 54 | override def onReady() = { 55 | log.debug("AggregationRouter#onReady") 56 | } 57 | override def onMessage(message: ReqT) = { 58 | // We don't do the aggregation here but on onHalfClose() 59 | request = message 60 | } 61 | override def onComplete() = { 62 | log.debug("AggregationRouter#onComplete") 63 | } 64 | } 65 | } 66 | } 67 | 68 | protected def scatter[ReqT, RespT](nodes: List[MemberAddress], channelPool: CachedChannelPool, methodDescriptor: MethodDescriptor[ReqT, RespT], headers: Metadata, input: ReqT): util.List[RespT] = { 69 | AggregationRouter.scatter(nodes, channelPool, methodDescriptor, headers, input) 70 | } 71 | } 72 | 73 | object AggregationRouter { 74 | def scatter[ReqT, RespT](nodes: List[MemberAddress], channelPool: CachedChannelPool, methodDescriptor: MethodDescriptor[ReqT, RespT], headers:Metadata, input: ReqT): util.List[RespT] = { 75 | val scatterRequests = nodes.map(destination => { 76 | val channel = channelPool.get(destination, insecure = true) 77 | val clientCall = ClientInterceptors.interceptForward(channel, MetadataUtils.newAttachHeadersInterceptor(headers)) 78 | .newCall(methodDescriptor, CallOptions.DEFAULT.withDeadlineAfter(10, TimeUnit.MINUTES)) // TODO (ashwanthkumar): Make this deadline configurable 79 | ClientCalls.futureUnaryCall(clientCall, input) 80 | }) 81 | 82 | Futures.allAsList(scatterRequests: _*).get() 83 | } 84 | } 85 | 86 | -------------------------------------------------------------------------------- /suuchi-cluster-atomix/src/main/scala/in/ashwanthkumar/suuchi/cluster/atomix/AtomixCluster.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.cluster.atomix 2 | 3 | import java.io.File 4 | import java.time.Duration 5 | import java.util.function.Consumer 6 | 7 | import com.typesafe.config.Config 8 | import in.ashwanthkumar.suuchi.cluster.{Cluster, MemberAddress, MemberListener, SeedProvider} 9 | import io.atomix.AtomixReplica 10 | import io.atomix.catalyst.transport.Address 11 | import io.atomix.catalyst.transport.netty.NettyTransport 12 | import io.atomix.copycat.server.storage.{Storage, StorageLevel} 13 | import io.atomix.group.{GroupMember, LocalMember} 14 | import org.slf4j.LoggerFactory 15 | 16 | import scala.collection.JavaConversions._ 17 | 18 | /** 19 | * State object that's stored as metadata associated with the member. We store this node's [[MemberAddress]] information 20 | * so it's available for others to consume if they want use it for communication at a later point 21 | * 22 | * @param address MemberAddress of this node 23 | */ 24 | case class MemberState(address: MemberAddress) 25 | 26 | class AtomixCluster(host: String, 27 | atomixPort: Int, 28 | rpcPort: Int, 29 | workDir: String, 30 | clusterIdentifier: String, 31 | config: Config, 32 | listeners: List[MemberListener] = Nil) 33 | extends Cluster(config, listeners) { 34 | private val log = LoggerFactory.getLogger(classOf[AtomixCluster]) 35 | 36 | var atomix = AtomixReplica 37 | .builder(new Address(host, atomixPort)) 38 | .withTransport(NettyTransport.builder().build()) 39 | .withStorage( 40 | Storage 41 | .builder() 42 | .withDirectory(new File(workDir, host + "_" + atomixPort)) 43 | .withStorageLevel(StorageLevel.DISK) 44 | .withMinorCompactionInterval(Duration.ofSeconds(30)) 45 | .withMajorCompactionInterval(Duration.ofMinutes(10)) 46 | .withFlushOnCommit() 47 | .build() 48 | ) 49 | .build() 50 | 51 | var me: LocalMember = _ 52 | 53 | override def start(seedProvider: SeedProvider): AtomixCluster = { 54 | if (seedProvider.nodes.isEmpty) { 55 | atomix = atomix.bootstrap().join() 56 | } else { 57 | atomix = atomix.join(seedProvider.nodes.map(m => new Address(m.host, m.port))).join() 58 | } 59 | 60 | val group = atomix.getGroup(clusterIdentifier).join() 61 | me = group.join(MemberState(MemberAddress(host, atomixPort))).join() 62 | // register a shutdown hook to gracefully leave the cluster 63 | Runtime.getRuntime.addShutdownHook(new Thread() { 64 | override def run(): Unit = { 65 | me.leave() 66 | } 67 | }) 68 | 69 | group.onJoin(new Consumer[GroupMember] { 70 | override def accept(t: GroupMember): Unit = { 71 | val memberState = t.metadata[MemberState]() 72 | if (memberState.isPresent) { 73 | onJoin(memberState.get().address) 74 | } else { 75 | log.warn("No memberstate associated with the node. Listeners wouldn't be triggered.") 76 | } 77 | } 78 | }) 79 | 80 | group.onLeave(new Consumer[GroupMember] { 81 | override def accept(t: GroupMember): Unit = { 82 | val memberState = t.metadata[MemberState]() 83 | if (memberState.isPresent) { 84 | onLeave(memberState.get().address) 85 | } else { 86 | log.warn("No memberstate associated with the node. Listeners wouldn't be triggered.") 87 | } 88 | } 89 | }) 90 | 91 | this 92 | } 93 | 94 | override def nodes: Iterable[MemberAddress] = { 95 | atomix 96 | .getGroup(clusterIdentifier) 97 | .get() 98 | .members() 99 | .map(t => t.metadata[MemberState]()) 100 | .filter(_.isPresent) 101 | .map(_.get().address) 102 | } 103 | 104 | override def stop(): Unit = { 105 | me.leave().join() 106 | } 107 | 108 | override def whoami: MemberAddress = MemberAddress(host, atomixPort) 109 | } 110 | -------------------------------------------------------------------------------- /docs/index.md: -------------------------------------------------------------------------------- 1 | # Suuchi 2 | 3 | Having inspired from tools like [Uber's Ringpop](https://ringpop.readthedocs.io/) and a strong desire to understand how distributed systems work - Suuchi was born. 4 | 5 | Suuchi is toolkit to build distributed data systems, that uses [gRPC](http://www.grpc.io/) under the hood as the communication medium. The overall goal of this project is to build pluggable components that can be easily composed by the developer to build a data system of desired characteristics. 6 | 7 | > This project is in beta quality and it's currently running couple of systems in production setting [@indix](https://twitter.com/indix). We welcome all kinds of feedback to help improve the library. 8 | 9 | 10 | ### Latest versions 11 | 12 | [![Suuchi Core on Maven Central](https://img.shields.io/maven-central/v/in.ashwanthkumar/suuchi-core.svg?label=suuchi-core&style=plastic)](https://maven-badges.herokuapp.com/maven-central/in.ashwanthkumar/suuchi-core) 13 | 14 | [![Suuchi RocksDB on Maven Central](https://img.shields.io/maven-central/v/in.ashwanthkumar/suuchi-rocksdb.svg?label=suuchi-rocksdb&style=plastic)](https://maven-badges.herokuapp.com/maven-central/in.ashwanthkumar/suuchi-rocksdb) 15 | 16 | [![Suuchi Atomix Cluster on Maven Central](https://img.shields.io/maven-central/v/in.ashwanthkumar/suuchi-cluster-atomix.svg?label=suuchi-cluster-atomix&style=plastic)](https://maven-badges.herokuapp.com/maven-central/in.ashwanthkumar/suuchi-cluster-atomix) 17 | 18 | [![Suuchi Scalecube Cluster on Maven Central](https://img.shields.io/maven-central/v/in.ashwanthkumar/suuchi-cluster-scalecube.svg?label=suuchi-cluster-scalecube&style=plastic)](https://maven-badges.herokuapp.com/maven-central/in.ashwanthkumar/suuchi-cluster-scalecube) 19 | 20 | ### Dependencies 21 | #### Maven 22 | ```xml 23 | 24 | in.ashwanthkumar 25 | suuchi-core 26 | ${suuchi.version} 27 | 28 | ``` 29 | 30 | #### SBT 31 | ```sbt 32 | libraryDependencies += "in.ashwanthkumar" % "suuchi-core" % suuchiVersion 33 | ``` 34 | 35 | Releases are published to [Sonatype release repository](https://oss.sonatype.org/content/repositories/releases) that eventually gets mirrored to Maven Central. 36 | 37 | Development snapshots are available in [Sonatypes's snapshot repository](https://oss.sonatype.org/content/repositories/snapshots/). 38 | 39 | ## Features 40 | 41 | - Enable [partitioning](internals/partitioner.md) of data using [Consistent Hashing](https://en.wikipedia.org/wiki/Consistent_hashing) 42 | - Supports synchronous [replication](internals/replication.md) to desired number of nodes 43 | - Supports Reduce - Re-Reduce style [aggregation](internals/aggregation.md) for methods that can be expressed using a [SemiGroup](https://twitter.github.io/algebird/typeclasses/semigroup.html). 44 | - Enables above set of features for any gRPC based service definitions 45 | 46 | If you are a developer looking to use Suuchi, head over to [Quick Start](quick-start.md) guide to get started. 47 | 48 | ## Recipes 49 | - [Distributed InMemory Database](recipes/inmemorydb.md) 50 | - [Distributed RocksDB backed KV](recipes/rocksdb.md) 51 | - [Distributed KVClient](recipes/kvclient.md) 52 | 53 | ## Internals 54 | We try to document the internal workings of some core pieces of Suuchi for developers interested in contributing or understanding their systems better. 55 | 56 | - [Aggregation](internals/aggregation.md) 57 | - [Partitioner](internals/partitioner.md) 58 | - [Replication](internals/replication.md) 59 | - [Router](internals/router.md) 60 | 61 | ## Presentations 62 | Following presentations / videos explain motivation behind Suuchi 63 | 64 | - Video by [@brewkode](https://twitter.com/brewkode) on [Suuchi - Toolkit to build distributed systems](https://www.youtube.com/watch?v=GK0-ICFvIGw) at Fifth Elephant, 2017. 65 | - [Suuchi - Distributed Systems Primitives](https://speakerdeck.com/ashwanthkumar/suuchi-distributed-system-primitives) 66 | - [Suuchi - Application Layer Sharding](https://speakerdeck.com/ashwanthkumar/suuchi-application-layer-sharding) 67 | - [Suuchi - Distributed Data Systems Toolkit](https://speakerdeck.com/ashwanthkumar/suuchi-distributed-data-systems-toolkit/) 68 | 69 | ## License 70 | [https://www.apache.org/licenses/LICENSE-2.0](https://www.apache.org/licenses/LICENSE-2.0) 71 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/ParallelReplicatorSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import java.util.concurrent.Executor 4 | 5 | import com.google.common.util.concurrent.{Futures, ListenableFuture} 6 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 7 | import io.grpc.{Metadata, MethodDescriptor, ServerCall} 8 | import org.mockito.Matchers 9 | import org.mockito.Matchers._ 10 | import org.mockito.Mockito._ 11 | import org.scalatest.{BeforeAndAfter, FlatSpec} 12 | 13 | class MockParallelReplicator(nrReplicas: Int, self: MemberAddress, mock: ParallelReplicator)( 14 | implicit executor: Executor) 15 | extends ParallelReplicator(nrReplicas, self) { 16 | override def forwardAsync[RespT, ReqT](methodDescriptor: MethodDescriptor[ReqT, RespT], 17 | headers: Metadata, 18 | incomingRequest: ReqT, 19 | destination: MemberAddress): ListenableFuture[RespT] = { 20 | mock.forwardAsync(methodDescriptor, headers, incomingRequest, destination) 21 | } 22 | } 23 | 24 | class ParallelReplicatorSpec extends FlatSpec { 25 | "ParallelReplicator" should "forward requests to target nodes in parallel" in { 26 | implicit val mockExecutor = mock(classOf[Executor]) 27 | val mockReplicator = mock(classOf[ParallelReplicator]) 28 | val replicator = new MockParallelReplicator(2, MemberAddress("host1", 1), mockReplicator) 29 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 30 | val delegate = mock(classOf[ServerCall.Listener[Int]]) 31 | val headers = new Metadata() 32 | val destination1 = MemberAddress("host2", 2) 33 | val destination2 = MemberAddress("host3", 3) 34 | 35 | when( 36 | mockReplicator.forwardAsync(any(classOf[MethodDescriptor[Int, Int]]), 37 | any(classOf[Metadata]), 38 | anyInt(), 39 | Matchers.eq(destination1))) 40 | .thenReturn(Futures.immediateFuture(2)) 41 | when( 42 | mockReplicator.forwardAsync(any(classOf[MethodDescriptor[Int, Int]]), 43 | any(classOf[Metadata]), 44 | anyInt(), 45 | Matchers.eq(destination2))) 46 | .thenReturn(Futures.immediateFuture(3)) 47 | 48 | replicator 49 | .replicate[Int, Int](List(destination1, destination2), serverCall, headers, 1, delegate) 50 | 51 | verify(mockReplicator, times(1)).forwardAsync(any(classOf[MethodDescriptor[Int, Int]]), 52 | any(classOf[Metadata]), 53 | anyInt(), 54 | Matchers.eq(destination1)) 55 | verify(mockReplicator, times(1)).forwardAsync(any(classOf[MethodDescriptor[Int, Int]]), 56 | any(classOf[Metadata]), 57 | anyInt(), 58 | Matchers.eq(destination2)) 59 | 60 | } 61 | 62 | it should "forward requests to target nodes in parallel & once done, should delegate to local node if it's in the replica node list" in { 63 | implicit val mockExecutor = mock(classOf[Executor]) 64 | val mockReplicator = mock(classOf[ParallelReplicator]) 65 | val replicator = new MockParallelReplicator(2, MemberAddress("host1", 1), mockReplicator) 66 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 67 | val delegate = mock(classOf[ServerCall.Listener[Int]]) 68 | val headers = new Metadata() 69 | val destination1 = MemberAddress("host2", 2) 70 | val destination2 = MemberAddress("host1", 1) 71 | 72 | when( 73 | mockReplicator.forwardAsync(any(classOf[MethodDescriptor[Int, Int]]), 74 | any(classOf[Metadata]), 75 | anyInt(), 76 | Matchers.eq(destination1))) 77 | .thenReturn(Futures.immediateFuture(2)) 78 | 79 | replicator 80 | .replicate[Int, Int](List(destination1, destination2), serverCall, headers, 1, delegate) 81 | 82 | verify(mockReplicator, times(1)).forwardAsync(any(classOf[MethodDescriptor[Int, Int]]), 83 | any(classOf[Metadata]), 84 | anyInt(), 85 | Matchers.eq(destination1)) 86 | verify(delegate, times(1)).onMessage(Matchers.eq(1)) 87 | 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/router/HandleOrForwardRouter.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 6 | import in.ashwanthkumar.suuchi.rpc.CachedChannelPool 7 | import io.grpc.ServerCall.Listener 8 | import io.grpc._ 9 | import io.grpc.stub.{ClientCalls, MetadataUtils} 10 | import org.slf4j.LoggerFactory 11 | 12 | import scala.language.postfixOps 13 | import scala.util.Try 14 | 15 | /** 16 | * Router decides to route the incoming request to right node in the cluster as defined 17 | * by the [[RoutingStrategy]]. 18 | * 19 | * @param routingStrategy 20 | */ 21 | class HandleOrForwardRouter(routingStrategy: RoutingStrategy, self: MemberAddress) 22 | extends ServerInterceptor { 23 | private val log = LoggerFactory.getLogger(getClass) 24 | val channelPool = CachedChannelPool() 25 | 26 | override def interceptCall[ReqT, RespT](serverCall: ServerCall[ReqT, RespT], 27 | headers: Metadata, 28 | next: ServerCallHandler[ReqT, RespT]): Listener[ReqT] = { 29 | log.trace( 30 | "Intercepting " + serverCall.getMethodDescriptor.getFullMethodName + " method in " + self + ", headers= " + headers.toString) 31 | new Listener[ReqT] { 32 | val delegate = next.startCall(serverCall, headers) 33 | var forwarded = false 34 | 35 | override def onReady(): Unit = delegate.onReady() 36 | override def onMessage(incomingRequest: ReqT): Unit = { 37 | if (routingStrategy.route.isDefinedAt(incomingRequest)) { 38 | val eligibleNodes = routingStrategy route incomingRequest 39 | // Always set ELIGIBLE_NODES header to the list of nodes eligible in the current 40 | // operation - as defined by the RoutingStrategy 41 | headers.put(Headers.ELIGIBLE_NODES_KEY, eligibleNodes) 42 | // Always choose the first node as the PRIMARY_NODE 43 | eligibleNodes.headOption.foreach(primaryNode => 44 | headers.put(Headers.PRIMARY_NODE_KEY, primaryNode)) 45 | 46 | eligibleNodes match { 47 | case nodes if nodes.nonEmpty && !nodes.exists(_.equals(self)) => 48 | forwarded = nodes.exists(destination => 49 | Try { 50 | log.trace(s"Forwarding request to $destination") 51 | val clientResponse: RespT = 52 | forward(serverCall.getMethodDescriptor, headers, incomingRequest, destination) 53 | // sendHeaders is very important and should be called before sendMessage 54 | // else client wouldn't receive any data at all 55 | serverCall.sendHeaders(headers) 56 | serverCall.sendMessage(clientResponse) 57 | true 58 | } recover { 59 | case r: RuntimeException => 60 | log.error(r.getMessage, r) 61 | false 62 | } get) 63 | 64 | if (!forwarded) { 65 | serverCall.close( 66 | Status.FAILED_PRECONDITION.withDescription("No alive nodes to handle traffic."), 67 | headers) 68 | } 69 | case nodes if nodes.nonEmpty && nodes.exists(_.equals(self)) => 70 | log.trace("Calling delegate's onMessage") 71 | delegate.onMessage(incomingRequest) 72 | case Nil => 73 | log.trace( 74 | "Couldn't locate the right node for this request. Returning a NOT_FOUND response") 75 | serverCall.close(Status.NOT_FOUND, headers) 76 | } 77 | } else { 78 | log.trace("Calling delegate's onMessage since router can't understand this message") 79 | delegate.onMessage(incomingRequest) 80 | } 81 | } 82 | 83 | override def onHalfClose(): Unit = { 84 | // apparently default ServerCall listener seems to hold some state from OnMessage which fails 85 | // here and client fails with an exception message -- Half-closed without a request 86 | if (forwarded) serverCall.close(Status.OK, headers) else delegate.onHalfClose() 87 | } 88 | override def onCancel(): Unit = delegate.onCancel() 89 | override def onComplete(): Unit = delegate.onComplete() 90 | } 91 | } 92 | 93 | def forward[RespT, ReqT](method: MethodDescriptor[ReqT, RespT], 94 | headers: Metadata, 95 | incomingRequest: ReqT, 96 | destination: MemberAddress): RespT = { 97 | val channel = channelPool.get(destination, insecure = true) 98 | ClientCalls.blockingUnaryCall( 99 | ClientInterceptors.interceptForward(channel, 100 | MetadataUtils.newAttachHeadersInterceptor(headers)), 101 | method, 102 | CallOptions.DEFAULT 103 | .withDeadlineAfter(10, TimeUnit.MINUTES), // TODO (ashwanthkumar): Make this deadline configurable 104 | incomingRequest 105 | ) 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/SequentialReplicatorSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import io.grpc.{Metadata, MethodDescriptor, ServerCall, Status} 5 | import org.mockito.Matchers._ 6 | import org.mockito.Mockito._ 7 | import org.mockito.{ArgumentCaptor, Matchers} 8 | import org.scalatest.FlatSpec 9 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 10 | 11 | class TestSequentialReplicator(nrReplicas: Int, self: MemberAddress) 12 | extends SequentialReplicator(nrReplicas, self) { 13 | override def forward[RespT, ReqT](methodDescriptor: MethodDescriptor[ReqT, RespT], 14 | headers: Metadata, 15 | incomingRequest: ReqT, 16 | destination: MemberAddress): Any = {} 17 | } 18 | 19 | class MockSequentialReplicator(nrReplicas: Int, self: MemberAddress, mock: SequentialReplicator) 20 | extends SequentialReplicator(nrReplicas, self) { 21 | override def forward[RespT, ReqT](methodDescriptor: MethodDescriptor[ReqT, RespT], 22 | headers: Metadata, 23 | incomingRequest: ReqT, 24 | destination: MemberAddress): Any = { 25 | mock.forward(methodDescriptor, headers, incomingRequest, destination) 26 | } 27 | } 28 | 29 | class SequentialReplicatorSpec extends FlatSpec { 30 | "SequentialReplicator" should "fail if number of nodes is < expected replicas" in { 31 | val replicator = new TestSequentialReplicator(3, MemberAddress("host1", 1)) 32 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 33 | val delegate = mock(classOf[ServerCall.Listener[Int]]) 34 | val headers = new Metadata() 35 | replicator 36 | .replicate[Int, Int](List(MemberAddress("host1", 1)), serverCall, headers, 1, delegate) 37 | 38 | val statusCaptor = ArgumentCaptor.forClass(classOf[Status]) 39 | verify(serverCall, times(1)).close(statusCaptor.capture(), Matchers.eq(headers)) 40 | val actualStatus = statusCaptor.getValue 41 | actualStatus.getDescription should be( 42 | "We don't have enough nodes to satisfy the replication factor. Not processing this request") 43 | actualStatus.getCode should be(Status.FAILED_PRECONDITION.getCode) 44 | } 45 | 46 | it should "fail if no nodes were sent to replicate" in { 47 | val replicator = new TestSequentialReplicator(0, MemberAddress("host1", 1)) 48 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 49 | val delegate = mock(classOf[ServerCall.Listener[Int]]) 50 | val headers = new Metadata() 51 | replicator.replicate[Int, Int](Nil, serverCall, headers, 1, delegate) 52 | 53 | val statusCaptor = ArgumentCaptor.forClass(classOf[Status]) 54 | verify(serverCall, times(1)).close(statusCaptor.capture(), Matchers.eq(headers)) 55 | val actualStatus = statusCaptor.getValue 56 | actualStatus.getDescription should be( 57 | "This should never happen. No nodes found to place replica") 58 | actualStatus.getCode should be(Status.INTERNAL.getCode) 59 | } 60 | 61 | it should "sequentially send forwards to the replicas" in { 62 | val mockReplicator = mock(classOf[SequentialReplicator]) 63 | val replicator = new MockSequentialReplicator(2, MemberAddress("host1", 1), mockReplicator) 64 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 65 | val delegate = mock(classOf[ServerCall.Listener[Int]]) 66 | val headers = new Metadata() 67 | val destination1 = MemberAddress("host2", 2) 68 | val destination2 = MemberAddress("host3", 3) 69 | replicator 70 | .replicate[Int, Int](List(destination1, destination2), serverCall, headers, 1, delegate) 71 | 72 | verify(mockReplicator, times(1)).forward(any(classOf[MethodDescriptor[Int, Int]]), 73 | any(classOf[Metadata]), 74 | anyInt(), 75 | Matchers.eq(destination1)) 76 | verify(mockReplicator, times(1)).forward(any(classOf[MethodDescriptor[Int, Int]]), 77 | any(classOf[Metadata]), 78 | anyInt(), 79 | Matchers.eq(destination2)) 80 | } 81 | 82 | it should "call delegate.OnMesssage if one of the nodes to replica is self" in { 83 | val mockReplicator = mock(classOf[SequentialReplicator]) 84 | val replicator = new MockSequentialReplicator(2, MemberAddress("host1", 1), mockReplicator) 85 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 86 | val delegate = mock(classOf[ServerCall.Listener[Int]]) 87 | val headers = new Metadata() 88 | val destination1 = MemberAddress("host1", 1) 89 | val destination2 = MemberAddress("host2", 2) 90 | replicator 91 | .replicate[Int, Int](List(destination1, destination2), serverCall, headers, 1, delegate) 92 | 93 | verify(mockReplicator, times(0)).forward(any(classOf[MethodDescriptor[Int, Int]]), 94 | any(classOf[Metadata]), 95 | anyInt(), 96 | Matchers.eq(destination1)) 97 | verify(mockReplicator, times(1)).forward(any(classOf[MethodDescriptor[Int, Int]]), 98 | any(classOf[Metadata]), 99 | anyInt(), 100 | Matchers.eq(destination2)) 101 | verify(delegate, times(1)).onMessage(Matchers.eq(1)) 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /suuchi-core/_pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | suuchi-core 4 | 0.3.6-SNAPSHOT 5 | Suuchi Core 6 | Core modules of Suuchi 7 | jar 8 | 9 | 10 | in.ashwanthkumar 11 | suuchi 12 | 0.3.6-SNAPSHOT 13 | 14 | 15 | 16 | 1.7 17 | 2.11.8 18 | 4.1.3.Final 19 | 20 | 21 | 22 | 23 | joda-time 24 | joda-time 25 | 2.8.2 26 | 27 | 28 | 29 | com.twitter 30 | algebird-core_${scala.lib.version} 31 | 0.13.0 32 | 33 | 34 | 35 | com.twitter 36 | bijection-core_${scala.lib.version} 37 | 0.9.5 38 | 39 | 40 | 41 | com.typesafe 42 | config 43 | 1.2.1 44 | 45 | 46 | 47 | commons-io 48 | commons-io 49 | 2.4 50 | 51 | 52 | 53 | io.grpc 54 | grpc-protobuf 55 | ${grpc.version} 56 | 57 | 58 | io.grpc 59 | grpc-testing 60 | ${grpc.version} 61 | 62 | 63 | io.grpc 64 | grpc-stub 65 | ${grpc.version} 66 | 67 | 68 | io.grpc 69 | grpc-services 70 | ${grpc.version} 71 | 72 | 73 | 74 | io.netty 75 | netty-codec 76 | ${netty.version} 77 | 78 | 79 | io.netty 80 | netty-common 81 | ${netty.version} 82 | 83 | 84 | io.netty 85 | netty-transport 86 | ${netty.version} 87 | 88 | 89 | io.netty 90 | netty-handler 91 | ${netty.version} 92 | 93 | 94 | 95 | org.slf4j 96 | slf4j-api 97 | 1.7.12 98 | 99 | 100 | 101 | org.scala-lang 102 | scala-library 103 | ${scala.version} 104 | 105 | 106 | 107 | org.mockito 108 | mockito-core 109 | 1.10.19 110 | test 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | kr.motd.maven 119 | os-maven-plugin 120 | 1.4.1.Final 121 | 122 | 123 | 124 | 125 | org.xolstice.maven.plugins 126 | protobuf-maven-plugin 127 | 0.5.0 128 | 129 | 134 | com.google.protobuf:protoc:3.0.2:exe:${os.detected.classifier} 135 | grpc-java 136 | io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} 137 | 138 | 139 | 140 | 141 | 142 | test-compile 143 | test-compile-custom 144 | 145 | 146 | 147 | 148 | 149 | 150 | 151 | 152 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/HandleOrForwardRouterSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import io.grpc.ServerCall.Listener 5 | import io.grpc._ 6 | import io.grpc.testing.TestMethodDescriptors 7 | import org.mockito.Matchers._ 8 | import org.mockito.Mockito._ 9 | import org.scalatest.FlatSpec 10 | 11 | class NeverRoute extends RoutingStrategy { 12 | 13 | /** 14 | * @inheritdoc 15 | */ 16 | override def route[ReqT]: PartialFunction[ReqT, List[MemberAddress]] = PartialFunction.empty 17 | } 18 | 19 | class NoAliveNodes extends RoutingStrategy { 20 | 21 | /** 22 | * @inheritdoc 23 | */ 24 | override def route[ReqT]: PartialFunction[ReqT, List[MemberAddress]] = { 25 | case _ => Nil 26 | } 27 | } 28 | 29 | class HandleOrForwardRouterSpec extends FlatSpec { 30 | "Router" should "not forward messages if routing strategy doesn't say so" in { 31 | val router = new HandleOrForwardRouter(new NeverRoute(), MemberAddress("host2", 1)) 32 | verifyInteractions(router, isForwarded = false, isHandledLocally = true) 33 | } 34 | 35 | it should "not forward messages if no nodes are alive" in { 36 | val router = new HandleOrForwardRouter(new NoAliveNodes(), MemberAddress("host2", 1)) 37 | verifyInteractions(router, isForwarded = false, isHandledLocally = false) 38 | } 39 | 40 | it should "not forward message when router emits node to self" in { 41 | val router = new HandleOrForwardRouter(new AlwaysRouteTo(MemberAddress("host2", 1)), 42 | MemberAddress("host2", 1)) 43 | verifyInteractions(router, isForwarded = false, isHandledLocally = true) 44 | } 45 | 46 | it should "forward message when router says so" in { 47 | val router = new HandleOrForwardRouter(new AlwaysRouteTo(MemberAddress("host1", 1)), 48 | MemberAddress("host2", 1)) { 49 | // mocking the actual forward implementation 50 | override def forward[RespT, ReqT](method: MethodDescriptor[ReqT, RespT], 51 | headers: Metadata, 52 | incomingRequest: ReqT, 53 | destination: MemberAddress): RespT = 1.asInstanceOf[RespT] 54 | } 55 | verifyInteractions(router, isForwarded = true, isHandledLocally = false) 56 | } 57 | 58 | it should "forward message to next node in the ring if the first one fails" in { 59 | val router = new HandleOrForwardRouter( 60 | new AlwaysRouteTo(MemberAddress("host1", 1), MemberAddress("host2", 2)), 61 | MemberAddress("host2", 1)) { 62 | var shouldFail = true 63 | // mocking the actual forward implementation 64 | override def forward[RespT, ReqT](method: MethodDescriptor[ReqT, RespT], 65 | headers: Metadata, 66 | incomingRequest: ReqT, 67 | destination: MemberAddress): RespT = { 68 | if (shouldFail) { 69 | shouldFail = false 70 | throw new RuntimeException("An exception happened while trying to forward the request") 71 | } else { 72 | 1.asInstanceOf[RespT] 73 | } 74 | } 75 | } 76 | verifyInteractions(router, isForwarded = true, isHandledLocally = false) 77 | } 78 | 79 | it should "not forward message to any node in the ring if all node forwards fail" in { 80 | val router = new HandleOrForwardRouter( 81 | new AlwaysRouteTo(MemberAddress("host1", 1), MemberAddress("host2", 2)), 82 | MemberAddress("host2", 1)) { 83 | // mocking the actual forward implementation 84 | override def forward[RespT, ReqT](method: MethodDescriptor[ReqT, RespT], 85 | headers: Metadata, 86 | incomingRequest: ReqT, 87 | destination: MemberAddress): RespT = { 88 | throw new RuntimeException("An exception happened while trying to forward the request") 89 | } 90 | } 91 | verifyInteractions(router, isForwarded = false, isHandledLocally = false) 92 | } 93 | 94 | def verifyInteractions(router: HandleOrForwardRouter, 95 | isForwarded: Boolean, 96 | isHandledLocally: Boolean): Unit = { 97 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 98 | val serverMethodDesc = TestMethodDescriptors.noopMethod[Int, Int]() 99 | when(serverCall.getMethodDescriptor).thenReturn(serverMethodDesc) 100 | 101 | val delegate = mock(classOf[Listener[Int]]) 102 | val next = mock(classOf[ServerCallHandler[Int, Int]]) 103 | when(next.startCall(any(classOf[ServerCall[Int, Int]]), any(classOf[Metadata]))) 104 | .thenReturn(delegate) 105 | 106 | val listener = router.interceptCall(serverCall, new Metadata(), next) 107 | listener.onReady() 108 | listener.onMessage(1) 109 | listener.onHalfClose() 110 | listener.onComplete() 111 | listener.onCancel() 112 | 113 | verify(delegate, times(1)).onReady() 114 | if (isForwarded) { 115 | verify(delegate, times(0)).onMessage(1) 116 | verify(delegate, times(0)).onHalfClose() 117 | 118 | verify(serverCall, times(1)).sendHeaders(any(classOf[Metadata])) 119 | verify(serverCall, times(1)).sendMessage(1) 120 | } else if (isHandledLocally) { 121 | verify(delegate, times(1)).onMessage(1) 122 | verify(delegate, times(1)).onHalfClose() 123 | } else { 124 | verify(serverCall, times(0)).sendHeaders(any(classOf[Metadata])) 125 | verify(serverCall, times(0)).sendMessage(1) 126 | verify(serverCall, times(1)).close(any(classOf[Status]), any(classOf[Metadata])) 127 | } 128 | verify(delegate, times(1)).onComplete() 129 | verify(delegate, times(1)).onCancel() 130 | } 131 | } 132 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/ReplicationRouterSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import io.grpc.ServerCall.Listener 5 | import io.grpc.testing.TestMethodDescriptors 6 | import io.grpc.{Metadata, MethodDescriptor, ServerCall, ServerCallHandler} 7 | import org.mockito.Matchers._ 8 | import org.mockito.Mockito._ 9 | import org.scalatest.FlatSpec 10 | 11 | class NoReplicator(nrOfReplicas: Int, self: MemberAddress) 12 | extends ReplicationRouter(nrOfReplicas, self) { 13 | override def replicate[ReqT, RespT](eligibleNodes: scala.List[MemberAddress], 14 | serverCall: ServerCall[ReqT, RespT], 15 | headers: Metadata, 16 | incomingRequest: ReqT, 17 | delegate: ServerCall.Listener[ReqT]): Unit = {} 18 | override def doReplication[ReqT, RespT]( 19 | eligibleNodes: scala.List[_root_.in.ashwanthkumar.suuchi.cluster.MemberAddress], 20 | serverCall: _root_.io.grpc.ServerCall[ReqT, RespT], 21 | headers: _root_.io.grpc.Metadata, 22 | incomingRequest: ReqT, 23 | delegate: _root_.io.grpc.ServerCall.Listener[ReqT]): Unit = ??? 24 | } 25 | 26 | class MockReplicator(nrOfReplicas: Int, self: MemberAddress, mock: ReplicationRouter) 27 | extends ReplicationRouter(nrOfReplicas, self) { 28 | 29 | /** 30 | * @inheritdoc 31 | */ 32 | override def replicate[ReqT, RespT]( 33 | eligibleNodes: scala.List[_root_.in.ashwanthkumar.suuchi.cluster.MemberAddress], 34 | serverCall: _root_.io.grpc.ServerCall[ReqT, RespT], 35 | headers: _root_.io.grpc.Metadata, 36 | incomingRequest: ReqT, 37 | delegate: _root_.io.grpc.ServerCall.Listener[ReqT]): Unit = { 38 | mock.replicate(eligibleNodes, serverCall, headers, incomingRequest, delegate) 39 | } 40 | override def doReplication[ReqT, RespT]( 41 | eligibleNodes: scala.List[_root_.in.ashwanthkumar.suuchi.cluster.MemberAddress], 42 | serverCall: _root_.io.grpc.ServerCall[ReqT, RespT], 43 | headers: _root_.io.grpc.Metadata, 44 | incomingRequest: ReqT, 45 | delegate: _root_.io.grpc.ServerCall.Listener[ReqT]): Unit = ??? 46 | } 47 | 48 | class ReplicationRouterSpec extends FlatSpec { 49 | "ReplicationRouter" should "delegate the message to the local node if it's a REPLICATION_REQUEST" in { 50 | val whoami = MemberAddress("host1", 1) 51 | val replicator = new NoReplicator(1, whoami) 52 | 53 | setupAndVerify { 54 | (serverCall: ServerCall[Int, Int], 55 | delegate: ServerCall.Listener[Int], 56 | next: ServerCallHandler[Int, Int]) => 57 | when(next.startCall(any(classOf[ServerCall[Int, Int]]), any(classOf[Metadata]))) 58 | .thenReturn(delegate) 59 | val headers = new Metadata() 60 | headers.put(Headers.REPLICATION_REQUEST_KEY, whoami.toString) 61 | 62 | val listener = replicator.interceptCall(serverCall, headers, next) 63 | listener.onReady() 64 | listener.onMessage(1) 65 | listener.onHalfClose() 66 | listener.onComplete() 67 | listener.onCancel() 68 | 69 | verify(delegate, times(1)).onReady() 70 | verify(delegate, times(1)).onMessage(1) 71 | } 72 | } 73 | 74 | it should "not do anything if no required headers are present" in { 75 | val whoami = MemberAddress("host1", 1) 76 | val replicator = new NoReplicator(1, whoami) 77 | 78 | setupAndVerify { 79 | (serverCall: ServerCall[Int, Int], 80 | delegate: ServerCall.Listener[Int], 81 | next: ServerCallHandler[Int, Int]) => 82 | val headers = new Metadata() 83 | val listener = replicator.interceptCall(serverCall, headers, next) 84 | listener.onReady() 85 | listener.onMessage(1) 86 | listener.onHalfClose() 87 | listener.onComplete() 88 | listener.onCancel() 89 | 90 | verify(delegate, times(1)).onReady() 91 | verify(delegate, times(0)).onMessage(1) 92 | } 93 | } 94 | 95 | it should "replicate the request as per replication strategy" in { 96 | val whoami = MemberAddress("host1", 1) 97 | val mockReplicator = mock(classOf[ReplicationRouter]) 98 | val replicator = new MockReplicator(1, whoami, mockReplicator) 99 | 100 | setupAndVerify { 101 | (serverCall: ServerCall[Int, Int], 102 | delegate: ServerCall.Listener[Int], 103 | next: ServerCallHandler[Int, Int]) => 104 | val headers = new Metadata() 105 | headers.put(Headers.ELIGIBLE_NODES_KEY, List(whoami)) 106 | val listener = replicator.interceptCall(serverCall, headers, next) 107 | listener.onReady() 108 | listener.onMessage(1) 109 | listener.onHalfClose() 110 | listener.onComplete() 111 | listener.onCancel() 112 | 113 | verify(delegate, times(1)).onReady() 114 | verify(delegate, times(0)).onMessage(1) 115 | verify(mockReplicator, times(1)).replicate[Int, Int]( 116 | any(classOf[List[MemberAddress]]), 117 | any(classOf[ServerCall[Int, Int]]), 118 | any(classOf[Metadata]), 119 | anyInt(), 120 | any(classOf[ServerCall.Listener[Int]])) 121 | } 122 | } 123 | 124 | def setupAndVerify(verify: (ServerCall[Int, Int], 125 | ServerCall.Listener[Int], 126 | ServerCallHandler[Int, Int]) => Unit): Unit = { 127 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 128 | val serverMethodDesc = TestMethodDescriptors.noopMethod[Int, Int]() 129 | when(serverCall.getMethodDescriptor).thenReturn(serverMethodDesc) 130 | 131 | val delegate = mock(classOf[Listener[Int]]) 132 | val next = mock(classOf[ServerCallHandler[Int, Int]]) 133 | when(next.startCall(any(classOf[ServerCall[Int, Int]]), any(classOf[Metadata]))) 134 | .thenReturn(delegate) 135 | 136 | verify(serverCall, delegate, next) 137 | } 138 | } 139 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/partitioner/ConsistentHashRingSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.partitioner 2 | 3 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 4 | import org.scalatest.FlatSpec 5 | import org.scalatest.Matchers._ 6 | 7 | import scala.collection.JavaConversions._ 8 | 9 | object IdentityHash extends Hash { 10 | override def hash(bytes: Array[Byte]): Integer = bytes.toString.toInt 11 | } 12 | 13 | class ConsistentHashRingSpec extends FlatSpec { 14 | "ConsistentHashRing" should "pin nodes into the ring accounting for virtual nodes" in { 15 | val nodes = 16 | List(MemberAddress("host1", 1), MemberAddress("host2", 2), MemberAddress("host3", 3)) 17 | val ring = ConsistentHashRing(nodes, partitionsPerNode = 3) 18 | 19 | ring.nodes.size() should be(9) 20 | 21 | ring.add(MemberAddress("host100", 100)) 22 | ring.nodes.size() should be(12) 23 | } 24 | 25 | it should "remove nodes & its replica nodes on remove" in { 26 | val ring = new ConsistentHashRing(SuuchiHash, partitionsPerNode = 3) 27 | ring.init( 28 | List(MemberAddress("host1", 1), MemberAddress("host2", 2), MemberAddress("host3", 3))) 29 | 30 | ring.remove(MemberAddress("host1", 1)) 31 | ring.nodes.size should be(6) 32 | 33 | ring.remove(MemberAddress("host2", 2)) 34 | ring.nodes.size should be(3) 35 | 36 | ring.remove(MemberAddress("host3", 3)) 37 | ring.nodes.size should be(0) 38 | 39 | ring.remove(MemberAddress("host4", 4)) 40 | ring.nodes.size should be(0) 41 | } 42 | 43 | it should "return None when no nodes are present in the ring" in { 44 | val ring = ConsistentHashRing(Nil, partitionsPerNode = 3) 45 | ring.find("1".getBytes) should be(None) 46 | } 47 | 48 | it should "return the only node on find when only 1 node is present in the ring" in { 49 | val ring = ConsistentHashRing(List(MemberAddress("host1", 1)), partitionsPerNode = 1) 50 | ring.find("1".getBytes) should be(Some(MemberAddress("host1", 1))) 51 | } 52 | 53 | it should "return the same node multiple times when the number of unique nodes is less but requested bins are more" in { 54 | val ring = ConsistentHashRing(List(MemberAddress("host1", 1)), partitionsPerNode = 3) 55 | val list = ring.find("1".getBytes, 3) 56 | list should have size 3 57 | list.head should be(MemberAddress("host1", 1)) 58 | list(1) should be(MemberAddress("host1", 1)) 59 | list(2) should be(MemberAddress("host1", 1)) 60 | } 61 | 62 | it should "might return same node multiple times even when we have enough number of nodes" in { 63 | val members = (1 to 5).map { index => 64 | MemberAddress(s"host$index", index) 65 | }.toList 66 | val ring = ConsistentHashRing(members, partitionsPerNode = 3) 67 | val list = ring.find("1".getBytes, 3) 68 | list should have size 3 69 | list.head should be(MemberAddress("host3", 3)) 70 | list(1) should be(MemberAddress("host3", 3)) 71 | list(2) should be(MemberAddress("host4", 4)) 72 | } 73 | 74 | it should "not return the same node multiple times" in { 75 | val ring = ConsistentHashRing(List(MemberAddress("host1", 1)), partitionsPerNode = 3) 76 | val list = ring.findUnique("1".getBytes, 3) 77 | list should have size 1 78 | list should contain(MemberAddress("host1", 1)) 79 | } 80 | 81 | it should "return unique set of nodes when we've more then replica count nodes in the ring" in { 82 | val members = (1 to 5).map { index => 83 | MemberAddress(s"host$index", index) 84 | }.toList 85 | val ring = ConsistentHashRing(members, partitionsPerNode = 3) 86 | val list = ring.findUnique("1".getBytes, 3) 87 | list should have size 3 88 | list should contain(MemberAddress("host3", 3)) 89 | list should contain(MemberAddress("host4", 4)) 90 | list should contain(MemberAddress("host1", 1)) 91 | } 92 | 93 | it should "return the right ringState that wraps around the HashRing" in { 94 | val ring = ConsistentHashRing(Nil, partitionsPerNode = 2) 95 | val host1 = MemberAddress("host1", 1) 96 | val host2 = MemberAddress("host2", 2) 97 | val host3 = MemberAddress("host3", 3) 98 | 99 | ring.sortedMap.put(10, VNode(host1, 1)) 100 | ring.sortedMap.put(20, VNode(host2, 1)) 101 | ring.sortedMap.put(30, VNode(host3, 1)) 102 | ring.sortedMap.put(40, VNode(host3, 2)) 103 | ring.sortedMap.put(50, VNode(host2, 2)) 104 | ring.sortedMap.put(60, VNode(host1, 2)) 105 | 106 | val ringState = ring.ringState 107 | ringState.ranges should have size 6 108 | ringState.byNodes(host1) should contain(TokenRange(10, 19, VNode(host1, 1))) 109 | ringState.byNodes(host1) should contain(TokenRange(60, 9, VNode(host1, 2))) 110 | 111 | ringState.byNodes(host2) should contain(TokenRange(20, 29, VNode(host2, 1))) 112 | ringState.byNodes(host2) should contain(TokenRange(50, 59, VNode(host2, 2))) 113 | 114 | ringState.byNodes(host3) should contain(TokenRange(30, 39, VNode(host3, 1))) 115 | ringState.byNodes(host3) should contain(TokenRange(40, 49, VNode(host3, 2))) 116 | } 117 | 118 | it should "return list of token ranges for each VNode in a ring" in { 119 | val ring = ConsistentHashRing(Nil, partitionsPerNode = 2) 120 | val host1 = MemberAddress("host1", 1) 121 | val host2 = MemberAddress("host2", 2) 122 | val host3 = MemberAddress("host3", 3) 123 | 124 | ring.sortedMap.put(10, VNode(host1, 1)) 125 | ring.sortedMap.put(20, VNode(host2, 1)) 126 | ring.sortedMap.put(30, VNode(host3, 1)) 127 | ring.sortedMap.put(40, VNode(host3, 2)) 128 | ring.sortedMap.put(50, VNode(host2, 2)) 129 | ring.sortedMap.put(60, VNode(host1, 2)) 130 | 131 | val ringState = ring.ringState 132 | val totalShards = ringState.withReplication(2) 133 | totalShards should have size 6 134 | 135 | val range1 = TokenRange(10, 19, VNode(host1, 1)) 136 | totalShards(range1) should have size 2 137 | totalShards(range1) should be(List(host1, host2)) 138 | 139 | totalShards(TokenRange(20, 29, VNode(host2, 1))) should be(List(host2, host3)) 140 | totalShards(TokenRange(30, 39, VNode(host3, 1))) should be(List(host3, host2)) 141 | totalShards(TokenRange(40, 49, VNode(host3, 2))) should be(List(host3, host2)) 142 | totalShards(TokenRange(50, 59, VNode(host2, 2))) should be(List(host2, host1)) 143 | totalShards(TokenRange(60, 9, VNode(host1, 2))) should be(List(host1, host2)) 144 | } 145 | } 146 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/store/ShardedStoreSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.nio.ByteBuffer 4 | 5 | import in.ashwanthkumar.suuchi.partitioner.Hash 6 | import org.mockito.Mockito.{mock, times, verify, when} 7 | import org.mockito.Matchers.{anyInt} 8 | import org.scalatest.FlatSpec 9 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 10 | 11 | class ShardedStoreSpec extends FlatSpec { 12 | "ShardedStore" should "create 2 stores for 2 different Partitions" in { 13 | val hash = mock(classOf[Hash]) 14 | when(hash.hash("1".getBytes)).thenReturn(1) 15 | when(hash.hash("2".getBytes)).thenReturn(2) 16 | 17 | val store1 = mock(classOf[Store]) 18 | when(store1.get("1".getBytes)).thenReturn(None) 19 | val store2 = mock(classOf[Store]) 20 | when(store2.get("2".getBytes)).thenReturn(Some(Array(Byte.MaxValue))) 21 | 22 | val createStore = mock(classOf[(Int) => Store]) 23 | when(createStore.apply(1)).thenReturn(store1) 24 | when(createStore.apply(2)).thenReturn(store2) 25 | 26 | val shardedStore = new ShardedStore(3, hash, createStore) 27 | val response = shardedStore.get("1".getBytes) 28 | verify(hash, times(1)).hash("1".getBytes) 29 | verify(createStore, times(1)).apply(1) 30 | verify(store1, times(1)).get("1".getBytes) 31 | response should be(None) 32 | 33 | val response2 = shardedStore.get("2".getBytes) 34 | verify(hash, times(1)).hash("2".getBytes) 35 | verify(createStore, times(1)).apply(2) 36 | verify(store2, times(1)).get("2".getBytes) 37 | response2.map(ByteBuffer.wrap) should be(Some(Array(Byte.MaxValue)).map(ByteBuffer.wrap)) 38 | } 39 | 40 | it should "return the same store instance for the same partition but for different keys too" in { 41 | val hash = mock(classOf[Hash]) 42 | when(hash.hash("1".getBytes)).thenReturn(1) 43 | when(hash.hash("2".getBytes)).thenReturn(1) 44 | 45 | val store = mock(classOf[Store]) 46 | when(store.get("1".getBytes)).thenReturn(None) 47 | 48 | val createStore = mock(classOf[(Int) => Store]) 49 | when(createStore.apply(1)).thenReturn(store) 50 | 51 | val shardedStore = new ShardedStore(3, hash, createStore) 52 | shardedStore.put("1".getBytes, "1".getBytes) should be(true) 53 | shardedStore.put("2".getBytes, "3".getBytes) should be(true) 54 | verify(store, times(1)).put("1".getBytes, "1".getBytes) 55 | verify(store, times(1)).put("2".getBytes, "3".getBytes) 56 | } 57 | 58 | it should "call get / put / remove / scan / scan with prefix on the delegate store when corresponding methods are called" in { 59 | val hash = mock(classOf[Hash]) 60 | when(hash.hash("1".getBytes)).thenReturn(1) 61 | 62 | val store = mock(classOf[Store]) 63 | when(store.get("1".getBytes)).thenReturn(None) 64 | when(store.put("1".getBytes, "2".getBytes)).thenReturn(true) 65 | when(store.remove("1".getBytes)).thenReturn(true) 66 | val scanner = mock(classOf[Scanner[KV]]) 67 | when(store.scanner()).thenReturn(scanner) 68 | when(scanner.scan()).thenReturn(Iterator.empty) 69 | when(scanner.scan("1".getBytes)).thenReturn(Iterator.empty) 70 | 71 | val createStore = mock(classOf[(Int) => Store]) 72 | when(createStore.apply(0)).thenReturn(store) 73 | 74 | val shardedStore = new ShardedStore(1, hash, createStore) 75 | shardedStore.get("1".getBytes) should be(None) 76 | shardedStore.put("1".getBytes, "2".getBytes) should be(true) 77 | shardedStore.remove("1".getBytes) should be(true) 78 | StoreUtils.scan(shardedStore.scanner()).toList should be(List.empty) 79 | StoreUtils.scan("1".getBytes, shardedStore.scanner()).toList should be(List.empty) 80 | 81 | verify(store, times(1)).get("1".getBytes) 82 | verify(store, times(1)).put("1".getBytes, "2".getBytes) 83 | verify(store, times(1)).remove("1".getBytes) 84 | verify(scanner, times(1)).scan() 85 | verify(scanner, times(1)).scan("1".getBytes) 86 | } 87 | 88 | it should "return None for store.get when underlying store throws an Exception" in { 89 | val hash = mock(classOf[Hash]) 90 | when(hash.hash("1".getBytes)).thenReturn(1) 91 | 92 | val store = mock(classOf[Store]) 93 | when(store.get("1".getBytes)).thenThrow(classOf[RuntimeException]) 94 | val createStore = mock(classOf[(Int) => Store]) 95 | when(createStore.apply(1)).thenReturn(store) 96 | 97 | val shardedStore = new ShardedStore(3, hash, createStore) 98 | shardedStore.get("1".getBytes) should be(None) 99 | } 100 | 101 | it should "return false for store.put when underlying store throws an Exception" in { 102 | val hash = mock(classOf[Hash]) 103 | when(hash.hash("1".getBytes)).thenReturn(1) 104 | 105 | val store = mock(classOf[Store]) 106 | when(store.put("1".getBytes, "2".getBytes)).thenThrow(classOf[RuntimeException]) 107 | val createStore = mock(classOf[(Int) => Store]) 108 | when(createStore.apply(1)).thenReturn(store) 109 | 110 | val shardedStore = new ShardedStore(3, hash, createStore) 111 | shardedStore.put("1".getBytes, "2".getBytes) should be(false) 112 | } 113 | 114 | it should "return false for store.remove when underlying store throws an Exception" in { 115 | val hash = mock(classOf[Hash]) 116 | when(hash.hash("1".getBytes)).thenReturn(1) 117 | 118 | val store = mock(classOf[Store]) 119 | when(store.remove("1".getBytes)).thenThrow(classOf[RuntimeException]) 120 | val createStore = mock(classOf[(Int) => Store]) 121 | when(createStore.apply(1)).thenReturn(store) 122 | 123 | val shardedStore = new ShardedStore(3, hash, createStore) 124 | shardedStore.remove("1".getBytes) should be(false) 125 | } 126 | 127 | it should "initialize all store references during scan() if we already don't have them open" in { 128 | val hash = mock(classOf[Hash]) 129 | val createStore = mock(classOf[(Int) => Store]) 130 | val store = mock(classOf[Store]) 131 | when(createStore.apply(anyInt())).thenReturn(store) 132 | 133 | val mockScanner = mock(classOf[Scanner[KV]]) 134 | when(mockScanner.scan()).thenReturn(Iterator.empty) 135 | when(mockScanner.scan("prefix".getBytes)).thenReturn(Iterator.empty) 136 | when(store.scanner()).thenReturn(mockScanner) 137 | 138 | val shardedStore = new ShardedStore(3, hash, createStore) 139 | 140 | StoreUtils.scan(shardedStore.scanner()).toList 141 | verify(createStore, times(3)).apply(anyInt()) 142 | verify(mockScanner, times(3)).prepare() 143 | verify(mockScanner, times(3)).close() 144 | 145 | StoreUtils.scan("prefix".getBytes, shardedStore.scanner()).toList 146 | verify(createStore, times(3)).apply(anyInt()) 147 | } 148 | 149 | // TODO - Write tests for the synchronized {} in getStore. 150 | } 151 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/store/VersionedStore.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.util 4 | 5 | import com.google.common.io.ByteStreams 6 | import in.ashwanthkumar.suuchi.utils.DateUtils 7 | 8 | import scala.util.hashing.MurmurHash3 9 | import scala.language.postfixOps 10 | 11 | object Versions { 12 | /* 13 | * 14 | * Serialization util for managing list of Versions of type Long 15 | * We follow the below protocol to serialize List[Long] to Array[Byte] 16 | * 17 | * First byte - # of elements in the list - N 18 | * We will have N "longs represented as bytes - 8 bytes each" following it 19 | * Since, we know each version is a long, we just read 8 bytes to construct a long and move forward. 20 | * */ 21 | 22 | def fromBytes(bytes: Array[Byte]): List[Version] = { 23 | val reader = ByteStreams.newDataInput(bytes) 24 | val numVersions = reader.readInt() 25 | (1 to numVersions).map {_ => 26 | val versionTs = reader.readLong() 27 | val writtenTs = reader.readLong() 28 | Version(versionTs, writtenTs) 29 | }.toList 30 | } 31 | 32 | def toBytes(versions: List[Version]): Array[Byte] = { 33 | val writer = ByteStreams.newDataOutput() 34 | writer.writeInt(versions.size) 35 | versions.foreach {version => 36 | writer.writeLong(version.versionTs) 37 | writer.writeLong(version.writtenTs) 38 | } 39 | writer.toByteArray 40 | } 41 | } 42 | 43 | object VersionedStore { 44 | val VERSION_PREFIX = "V_".getBytes 45 | val DATA_PREFIX = "D_".getBytes 46 | 47 | def isVkeyKey(key: Array[Byte]) = 48 | util.Arrays.equals(key.take(VERSION_PREFIX.length), VERSION_PREFIX) 49 | def isDataKey(key: Array[Byte]) = util.Arrays.equals(key.take(DATA_PREFIX.length), DATA_PREFIX) 50 | def vkey(key: Array[Byte]) = VERSION_PREFIX ++ key 51 | def dkey(key: Array[Byte]): Array[Byte] = DATA_PREFIX ++ key 52 | def dkey(key: Array[Byte], version: Array[Byte]): Array[Byte] = DATA_PREFIX ++ key ++ version 53 | def dkey(key: Array[Byte], version: Long): Array[Byte] = 54 | DATA_PREFIX ++ key ++ PrimitivesSerDeUtils.longToBytes(version) 55 | } 56 | 57 | case class Version(versionTs: Long, writtenTs: Long) 58 | case class VRecord(key: Array[Byte], versions: List[Version]) { 59 | override def equals(obj: scala.Any): Boolean = obj match { 60 | case v: VRecord => 61 | util.Arrays.equals(v.key, key) && versions.equals(v.versions) 62 | case _ => false 63 | } 64 | override def toString = s"VRecord(${util.Arrays.toString(key)},${versions.toString})" 65 | } 66 | 67 | class VersionedStore(store: Store, 68 | versionedBy: VersionedBy, 69 | numVersions: Int, 70 | concurrencyFactor: Int = 8192) 71 | extends Store 72 | with DateUtils { 73 | 74 | import VersionedStore._ 75 | 76 | val SYNC_SLOTS = Array.fill(concurrencyFactor)(new Object) 77 | 78 | override def get(key: Array[Byte]): Option[Array[Byte]] = { 79 | // fetch version record 80 | val vRecord = store.get(vkey(key)) 81 | if (vRecord.isEmpty) None 82 | else { 83 | val versions = Versions.fromBytes(vRecord.get) 84 | get(key, versions.map(versionedBy.sortOn).max(versionedBy.versionOrdering)) 85 | } 86 | } 87 | 88 | def get(key: Array[Byte], version: Long): Option[Array[Byte]] = { 89 | store.get(dkey(key, version)) 90 | } 91 | 92 | override def put(key: Array[Byte], value: Array[Byte]): Boolean = { 93 | val currentVersion: Version = putAndPurgeVersions(key, value) 94 | putData(key, value, currentVersion) 95 | } 96 | 97 | override def remove(key: Array[Byte]) = { 98 | val versions = getVersions(key) 99 | removeVersion(key) 100 | versions.forall(v => removeData(key, v)) 101 | } 102 | 103 | def getVersions(key: Array[Byte]): List[Version] = { 104 | val vRecord = store.get(vkey(key)) 105 | vRecord 106 | .map(vr => Versions.fromBytes(vr)) 107 | .getOrElse(List.empty[Version]) 108 | } 109 | 110 | def putAndPurgeVersions(key: Array[Byte], value: Array[Byte], writtenTs: Long = now): Version = { 111 | val currentVersionTs = versionedBy.version(key, value) 112 | val currentVersion = Version(currentVersionTs, writtenTs) 113 | // atomically update version metadata 114 | val versions = atomicUpdate(key, currentVersion) 115 | // remove oldest version, if we've exceeded max # of versions per record 116 | if (versions.size > numVersions) removeData(key, versions.minBy(versionedBy.sortOn)) 117 | 118 | currentVersion 119 | } 120 | 121 | def putData(key: Array[Byte], value: Array[Byte], currentVersion: Version): Boolean = { 122 | // Write out the actual data record 123 | store.put(dkey(key, currentVersion.versionTs), value) 124 | } 125 | 126 | private def atomicUpdate(key: Array[Byte], currentVersion: Version) = { 127 | val versionKey = vkey(key) 128 | val absHash = math.abs(MurmurHash3.arrayHash(versionKey)) 129 | // Synchronizing the version metadata update part alone 130 | val monitor = SYNC_SLOTS(absHash % SYNC_SLOTS.length) 131 | val versions = monitor.synchronized { 132 | val vRecord = store.get(versionKey) 133 | val updatedVersions = currentVersion :: vRecord 134 | .map(bytes => Versions.fromBytes(bytes)) 135 | .getOrElse(List.empty[Version]) 136 | store.put( 137 | versionKey, 138 | Versions.toBytes(updatedVersions.sortBy(versionedBy.sortOn)(versionedBy.versionOrdering).take(numVersions))) 139 | updatedVersions 140 | } 141 | versions 142 | } 143 | 144 | def toVRecord(kv: KV) = VRecord(kv.key.drop(VERSION_PREFIX.length), Versions.fromBytes(kv.value)) 145 | 146 | override def scanner(): Scanner[KV] = new Scanner[KV] { 147 | 148 | private val delegate = store.scanner() 149 | 150 | override def prepare(): Unit = delegate.prepare() 151 | override def scan(prefix: Array[Byte]): Iterator[KV] = delegate.scan(dkey(prefix)) 152 | override def scan(): Iterator[KV] = 153 | delegate.scan(DATA_PREFIX) 154 | override def close(): Unit = delegate.close() 155 | } 156 | 157 | def versionsScanner(): Scanner[VRecord] = new Scanner[VRecord] { 158 | private val delegate = store.scanner() 159 | 160 | override def prepare(): Unit = delegate.prepare() 161 | override def scan(prefix: Array[Byte]): Iterator[VRecord] = 162 | delegate.scan(vkey(prefix)).map(toVRecord) 163 | override def scan(): Iterator[VRecord] = 164 | delegate.scan(VERSION_PREFIX).map(toVRecord) 165 | override def close(): Unit = delegate.close() 166 | } 167 | 168 | private def removeData(key: Array[Byte], version: Version) = store.remove(dkey(key, version.versionTs)) 169 | private def removeVersion(key: Array[Byte]) = store.remove(vkey(key)) 170 | } 171 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/router/AggregationRouterSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.router 2 | 3 | import java.util.{List => JList} 4 | 5 | import com.twitter.algebird.Aggregator 6 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 7 | import in.ashwanthkumar.suuchi.core.tests.{FooRequest, FooResponse, RandomGrpc} 8 | import in.ashwanthkumar.suuchi.rpc.CachedChannelPool 9 | import io.grpc.ServerCall.Listener 10 | import io.grpc._ 11 | import io.grpc.testing.TestMethodDescriptors 12 | import org.mockito.ArgumentCaptor 13 | import org.mockito.Matchers.{any, eq => mockEq} 14 | import org.mockito.Mockito.{mock, times, verify, when} 15 | import org.scalatest.FlatSpec 16 | import org.scalatest.Matchers.{be, convertToAnyShouldWrapper} 17 | 18 | import scala.collection.JavaConverters._ 19 | 20 | class AggregationRouterSpec extends FlatSpec { 21 | class RandomAggregation extends Aggregation { 22 | override def aggregator[A, B] 23 | : PartialFunction[MethodDescriptor[A, B], Aggregator[B, Any, B]] = { 24 | case RandomGrpc.METHOD_FOO => 25 | Aggregator.const(FooResponse.defaultInstance).asInstanceOf[Aggregator[B, Any, B]] 26 | } 27 | } 28 | 29 | "AggregationRouter" should "not do if the aggregation is not defined for the method" in { 30 | val router = 31 | new AggregationRouter(List(MemberAddress("host:1")), new RandomAggregation) 32 | val serverCall = mock(classOf[ServerCall[Int, Int]]) 33 | val serverMethodDesc = TestMethodDescriptors.noopMethod[Int, Int]() 34 | when(serverCall.getMethodDescriptor).thenReturn(serverMethodDesc) 35 | 36 | val headers = new Metadata 37 | val delegate = mock(classOf[Listener[Int]]) 38 | val next = mock(classOf[ServerCallHandler[Int, Int]]) 39 | when(next.startCall(any(classOf[ServerCall[Int, Int]]), mockEq(headers))) 40 | .thenReturn(delegate) 41 | 42 | val listener = router.interceptCall(serverCall, headers, next) 43 | listener.onReady() 44 | listener.onMessage(1) 45 | listener.onHalfClose() 46 | listener.onComplete() 47 | listener.onCancel() 48 | 49 | verify(next, times(1)).startCall(mockEq(serverCall), mockEq(headers)) 50 | } 51 | 52 | it should "not scatter requests if it already has BROADCAST header" in { 53 | val router = 54 | new AggregationRouter(List(MemberAddress("host:1")), new RandomAggregation) 55 | 56 | val serverCall = mock(classOf[ServerCall[FooRequest, FooResponse]]) 57 | val serverMethodDesc = TestMethodDescriptors.noopMethod[FooRequest, FooResponse]() 58 | when(serverCall.getMethodDescriptor).thenReturn(serverMethodDesc) 59 | 60 | val headers = new Metadata() 61 | headers.put(Headers.BROADCAST_REQUEST_KEY, true) 62 | 63 | val delegate = mock(classOf[Listener[FooRequest]]) 64 | val next = mock(classOf[ServerCallHandler[FooRequest, FooResponse]]) 65 | when(next.startCall(any(classOf[ServerCall[FooRequest, FooResponse]]), mockEq(headers))) 66 | .thenReturn(delegate) 67 | 68 | val listener = router.interceptCall(serverCall, headers, next) 69 | listener.onReady() 70 | listener.onMessage(FooRequest()) 71 | listener.onHalfClose() 72 | listener.onComplete() 73 | listener.onCancel() 74 | 75 | verify(next, times(1)).startCall(mockEq(serverCall), mockEq(headers)) 76 | } 77 | 78 | it should "scatter requests to all the nodes when aggregation is defined on the method" in { 79 | val router = 80 | new AggregationRouter(List(MemberAddress("host:1")), 81 | new RandomAggregation) { 82 | override protected def scatter[ReqT, RespT](nodes: List[MemberAddress], channelPool: CachedChannelPool, methodDescriptor: MethodDescriptor[ReqT, RespT], headers: Metadata, input: ReqT) = { 83 | List(FooResponse.defaultInstance.asInstanceOf[RespT]).asJava 84 | } 85 | } 86 | 87 | val serverCall = mock(classOf[ServerCall[FooRequest, FooResponse]]) 88 | when(serverCall.getMethodDescriptor).thenReturn(RandomGrpc.METHOD_FOO) 89 | 90 | val headers = new Metadata() 91 | 92 | val delegate = mock(classOf[Listener[FooRequest]]) 93 | val next = mock(classOf[ServerCallHandler[FooRequest, FooResponse]]) 94 | when(next.startCall(any(classOf[ServerCall[FooRequest, FooResponse]]), mockEq(headers))) 95 | .thenReturn(delegate) 96 | 97 | val listener = router.interceptCall(serverCall, headers, next) 98 | listener.onReady() 99 | listener.onMessage(FooRequest()) 100 | listener.onHalfClose() 101 | // during onHalfClose 102 | verify(serverCall, times(1)).sendHeaders(mockEq(headers)) 103 | verify(serverCall, times(1)).sendMessage(mockEq(FooResponse.defaultInstance)) 104 | verify(serverCall, times(1)).close(mockEq(Status.OK), mockEq(headers)) 105 | 106 | listener.onComplete() 107 | 108 | listener.onCancel() 109 | // during onCancel 110 | verify(serverCall, times(1)).close(mockEq(Status.CANCELLED), mockEq(headers)) 111 | 112 | // general interactions 113 | verify(next, times(0)).startCall(mockEq(serverCall), mockEq(headers)) 114 | verify(serverCall, times(1)).request(2) 115 | headers.containsKey(Headers.BROADCAST_REQUEST_KEY) should be(true) 116 | } 117 | 118 | it should "fail with INTERNAL when scatter request fail" in { 119 | val router = 120 | new AggregationRouter(List(MemberAddress("host:1")), new RandomAggregation) { 121 | override protected def scatter[ReqT, RespT]( 122 | nodes: List[MemberAddress], 123 | channelPool: CachedChannelPool, 124 | methodDescriptor: MethodDescriptor[ReqT, RespT], 125 | headers: Metadata, 126 | input: ReqT) = { 127 | throw new RuntimeException("scatter failed") 128 | } 129 | } 130 | 131 | val serverCall = mock(classOf[ServerCall[FooRequest, FooResponse]]) 132 | when(serverCall.getMethodDescriptor).thenReturn(RandomGrpc.METHOD_FOO) 133 | 134 | val headers = new Metadata() 135 | 136 | val delegate = mock(classOf[Listener[FooRequest]]) 137 | val next = mock(classOf[ServerCallHandler[FooRequest, FooResponse]]) 138 | when(next.startCall(any(classOf[ServerCall[FooRequest, FooResponse]]), mockEq(headers))) 139 | .thenReturn(delegate) 140 | 141 | val listener = router.interceptCall(serverCall, headers, next) 142 | listener.onReady() 143 | listener.onMessage(FooRequest()) 144 | listener.onHalfClose() 145 | // during onHalfClose 146 | val statusCaptor = ArgumentCaptor.forClass(classOf[Status]) 147 | verify(serverCall, times(1)).close(statusCaptor.capture(), mockEq(headers)) 148 | val status = statusCaptor.getValue 149 | status.getCode should be(Status.INTERNAL.getCode) 150 | status.getCause.getMessage should be("scatter failed") 151 | 152 | listener.onComplete() 153 | 154 | // general interactions 155 | verify(next, times(0)).startCall(mockEq(serverCall), mockEq(headers)) 156 | verify(serverCall, times(1)).request(2) 157 | headers.containsKey(Headers.BROADCAST_REQUEST_KEY) should be(true) 158 | } 159 | 160 | } 161 | -------------------------------------------------------------------------------- /suuchi-core/src/test/scala/in/ashwanthkumar/suuchi/store/VersionedStoreSpec.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.store 2 | 3 | import java.nio.ByteBuffer 4 | 5 | import in.ashwanthkumar.suuchi.store.PrimitivesSerDeUtils.{bytesToLong, longToBytes} 6 | import in.ashwanthkumar.suuchi.utils.{ByteArrayUtils, DateUtils} 7 | import org.scalatest.FlatSpec 8 | import org.scalatest.Matchers._ 9 | 10 | trait MockDateUtils extends DateUtils { 11 | var cnt = 0 12 | override def now: Long = { 13 | cnt += 1 14 | cnt 15 | } 16 | } 17 | class ByWriteTimestampMocked extends ByWriteTimestamp with MockDateUtils 18 | class KeyAsVersion extends VersionedBy { 19 | override val versionOrdering: Ordering[Long] = Ordering.Long.reverse 20 | override def version(key: Array[Byte], value: Array[Byte]): Long = bytesToLong(key) 21 | } 22 | 23 | class VersionedStoreSpec extends FlatSpec { 24 | "VersionedStore" should "return no version info for a key for the first time" in { 25 | val store = new VersionedStore(new InMemoryStore, new ByWriteTimestampMocked, 3) with MockDateUtils 26 | store.getVersions(Array(1.toByte)).size should be(0) 27 | } 28 | 29 | it should "return version info appropriately after every insert" in { 30 | val store = new VersionedStore(new InMemoryStore, new ByWriteTimestampMocked, 3) with MockDateUtils 31 | store.getVersions(Array(1.toByte)).size should be(0) 32 | 33 | store.put(Array(1.toByte), Array(100.toByte)) 34 | store.getVersions(Array(1.toByte)) should be(List(Version(1, 1))) 35 | 36 | store.put(Array(1.toByte), Array(101.toByte)) 37 | store.getVersions(Array(1.toByte)) should be(List(Version(2, 2), Version(1, 1))) 38 | 39 | store.put(Array(1.toByte), Array(102.toByte)) 40 | store.getVersions(Array(1.toByte)) should be(List(Version(3, 3), Version(2, 2), Version(1, 1))) 41 | 42 | store.put(Array(1.toByte), Array(103.toByte)) 43 | store.getVersions(Array(1.toByte)) should be(List(Version(4, 4), Version(3, 3), Version(2, 2))) 44 | } 45 | 46 | it should "write data for value with an earlier version" in { 47 | val store = new VersionedStore(new InMemoryStore, new KeyAsVersion, 3) with MockDateUtils 48 | store.put(longToBytes(456), longToBytes(456)) 49 | store.getVersions(longToBytes(456)) should be(List(Version(456, 1))) 50 | store.get(longToBytes(456)).map(ByteBuffer.wrap) should be( 51 | Some(ByteBuffer.wrap(longToBytes(456)))) 52 | 53 | store.put(longToBytes(123), longToBytes(123)) 54 | store.getVersions(longToBytes(123)) should be(List(Version(123, 2))) 55 | store.get(longToBytes(123)).map(ByteBuffer.wrap) should be( 56 | Some(ByteBuffer.wrap(longToBytes(123)))) 57 | } 58 | 59 | it should "delete old versions of data for a key when we exceed numVersions" in { 60 | val inMemoryStore = new InMemoryStore 61 | val store = new VersionedStore(inMemoryStore, new ByWriteTimestampMocked, 3) with MockDateUtils 62 | store.getVersions(Array(1.toByte)).size should be(0) 63 | 64 | store.put(Array(1.toByte), Array(100.toByte)) 65 | store.getVersions(Array(1.toByte)) should be(List(Version(1, 1))) 66 | 67 | store.put(Array(1.toByte), Array(101.toByte)) 68 | store.getVersions(Array(1.toByte)) should be(List(Version(2, 2), Version(1, 1))) 69 | 70 | store.put(Array(1.toByte), Array(102.toByte)) 71 | store.getVersions(Array(1.toByte)) should be(List(Version(3, 3), Version(2, 2), Version(1, 1))) 72 | 73 | store.put(Array(1.toByte), Array(103.toByte)) 74 | inMemoryStore.get(VersionedStore.dkey(Array(1.toByte), 1)) should be(None) 75 | } 76 | 77 | it should "support full store scan" in { 78 | val store = new VersionedStore(new InMemoryStore, new ByWriteTimestampMocked, 3) 79 | val inputs = List( 80 | ("one".getBytes, "1".getBytes), 81 | ("two".getBytes, "2".getBytes), 82 | ("three".getBytes, "3".getBytes), 83 | ("four".getBytes, "4".getBytes), 84 | ("five".getBytes, "5".getBytes) 85 | ) 86 | val fn = store.put _ 87 | val put = fn.tupled 88 | inputs.foreach(put) 89 | 90 | val scannedResult = StoreUtils.scan(store.scanner()).toList 91 | 92 | scannedResult should have size 5 93 | scannedResult.sortBy(kv => new String(kv.value)).zip(inputs).foreach { 94 | case (kv, (inputKey, inputValue)) => 95 | ByteArrayUtils.hasPrefix(kv.key, VersionedStore.dkey(inputKey)) should be(true) 96 | kv.value should be(inputValue) 97 | } 98 | } 99 | 100 | it should "support prefix scan" in { 101 | val store = new VersionedStore(new InMemoryStore, new ByWriteTimestampMocked, 3) 102 | val inputs = List( 103 | ("prefix1/one".getBytes, "1".getBytes), 104 | ("prefix1/two".getBytes, "2".getBytes), 105 | ("prefix1/three".getBytes, "3".getBytes), 106 | ("prefix2/one".getBytes, "1".getBytes), 107 | ("prefix2/two".getBytes, "2".getBytes), 108 | ("prefix2/three".getBytes, "3".getBytes) 109 | ) 110 | val fn = store.put _ 111 | val put = fn.tupled 112 | inputs.foreach(put) 113 | val prefix = "prefix1".getBytes 114 | 115 | val scannedResult = StoreUtils.scan(prefix, store.scanner()).toList 116 | 117 | scannedResult should have size 3 118 | scannedResult.foreach { kv => 119 | new String(kv.key) should startWith(prefixWithDkey(prefix)) 120 | } 121 | } 122 | 123 | it should "support version scan" in { 124 | val store = new VersionedStore(new InMemoryStore, new ByWriteTimestampMocked, 3) 125 | store.put("prefix1/one".getBytes, "1".getBytes) 126 | store.put("prefix2/two".getBytes, "2".getBytes) 127 | store.put("prefix3/three".getBytes, "3".getBytes) 128 | store.put("prefix1/one".getBytes, "11".getBytes) 129 | store.put("prefix2/two".getBytes, "22".getBytes) 130 | store.put("prefix1/one".getBytes, "111".getBytes) 131 | 132 | StoreUtils.scan(store.versionsScanner()).flatMap(_.versions) should have size 6 133 | } 134 | 135 | it should "support version scan based on prefix" in { 136 | val store = new VersionedStore(new InMemoryStore, new ByWriteTimestampMocked, 3) with MockDateUtils 137 | store.put("prefix1/one".getBytes, "1".getBytes) 138 | store.put("prefix2/two".getBytes, "2".getBytes) 139 | store.put("prefix3/three".getBytes, "3".getBytes) 140 | store.put("prefix1/one".getBytes, "11".getBytes) 141 | store.put("prefix2/two".getBytes, "22".getBytes) 142 | store.put("prefix1/one".getBytes, "111".getBytes) 143 | 144 | StoreUtils.scan("prefix1".getBytes, store.versionsScanner()).flatMap(_.versions) should have size 3 145 | StoreUtils.scan("prefix2".getBytes, store.versionsScanner()).flatMap(_.versions) should have size 2 146 | StoreUtils.scan("prefix3".getBytes, store.versionsScanner()).flatMap(_.versions) should have size 1 147 | StoreUtils.scan("prefix4".getBytes, store.versionsScanner()).flatMap(_.versions) should have size 0 148 | StoreUtils.scan("prefix3".getBytes, store.versionsScanner()).next() should be(VRecord("prefix3/three".getBytes, List(Version(3l, 3l)))) 149 | } 150 | 151 | private def prefixWithDkey(prefix: Array[Byte]) = new String(VersionedStore.dkey(prefix)) 152 | } 153 | -------------------------------------------------------------------------------- /suuchi-core/src/main/scala/in/ashwanthkumar/suuchi/partitioner/ConsistentHashRing.scala: -------------------------------------------------------------------------------- 1 | package in.ashwanthkumar.suuchi.partitioner 2 | 3 | import java.util 4 | 5 | import in.ashwanthkumar.suuchi.cluster.MemberAddress 6 | import in.ashwanthkumar.suuchi.utils.ByteArrayUtils 7 | 8 | import scala.annotation.tailrec 9 | import scala.collection.mutable 10 | 11 | case class VNode(node: MemberAddress, nodeReplicaId: Int) { 12 | def key = node.host + "_" + node.port + "_" + nodeReplicaId 13 | } 14 | 15 | case class TokenRange(start: Int, end: Int, node: VNode) { 16 | def member = node.node 17 | } 18 | 19 | object RingState { 20 | 21 | /** 22 | * Check if `key` falls within the given range using the `hashFn` 23 | * 24 | * @param key Key to check for 25 | * @param tokenRange TokenRange to check against 26 | * @param hashFn HashFunction used in CHRing 27 | * @return true if he key falls within the range 28 | * false otherwise 29 | */ 30 | def contains(key: Array[Byte], tokenRange: TokenRange, hashFn: Hash): Boolean = 31 | contains(key, tokenRange.start, tokenRange.end, hashFn) 32 | 33 | /** 34 | * Check if `key` falls within the given range using the `hashFn` 35 | * 36 | * @param key Key to check for 37 | * @param start Start range of the Token 38 | * @param end Last end of the Token 39 | * @param hashFn HashFunction used in CHRing 40 | * @return true if he key falls within the range 41 | * false otherwise 42 | */ 43 | def contains(key: Array[Byte], start: Int, end: Int, hashFn: Hash): Boolean = { 44 | ByteArrayUtils.isHashKeyWithinRange(start, end, key, hashFn) 45 | } 46 | 47 | def find(key: Array[Byte], ringState: RingState, hashFn: Hash): Option[TokenRange] = { 48 | ringState.ranges.find(r => contains(key, r, hashFn)) 49 | } 50 | } 51 | 52 | case class RingState(private[partitioner] val lastKnown: Int, ranges: List[TokenRange]) { 53 | def byNodes = ranges.groupBy(_.node.node) 54 | 55 | def withReplication(replicationFactor: Int) = 56 | pick(ranges.length, replicationFactor, ranges ::: ranges, Map()) 57 | 58 | @tailrec 59 | private final def pick( 60 | remaining: Int, 61 | replicationFactor: Int, 62 | ranges: List[TokenRange], 63 | result: Map[TokenRange, List[MemberAddress]]): Map[TokenRange, List[MemberAddress]] = { 64 | if (remaining == 0) result 65 | else { 66 | val replicas = ranges.map(_.member).distinct.take(replicationFactor) 67 | val tokens = Map(ranges.head -> replicas) 68 | pick(remaining - 1, replicationFactor, ranges.tail, result ++ tokens) 69 | } 70 | } 71 | } 72 | 73 | // Ref - https://git.io/vPOP5 74 | class ConsistentHashRing(hashFn: Hash, partitionsPerNode: Int, replicationFactor: Int = 2) { 75 | val sortedMap = new util.TreeMap[Integer, VNode]() 76 | 77 | // when looking for n unique nodes, give up after a streak of MAX_DUPES 78 | // duplicates 79 | val MAX_DUPES = 10 80 | 81 | def init(nodes: List[MemberAddress]) = { 82 | nodes.foreach(add) 83 | this 84 | } 85 | 86 | private def hash(vnode: VNode): Int = hashFn.hash(vnode.key.getBytes) 87 | 88 | def add(node: MemberAddress) = { 89 | (1 to partitionsPerNode).map(i => VNode(node, i)).foreach { vnode => 90 | sortedMap.put(hash(vnode), vnode) 91 | } 92 | this 93 | } 94 | 95 | def remove(node: MemberAddress) = { 96 | (1 to partitionsPerNode).map(i => VNode(node, i)).foreach { vnode => 97 | sortedMap.remove(hash(vnode)) 98 | } 99 | this 100 | } 101 | 102 | def find(key: Array[Byte]): Option[MemberAddress] = { 103 | find(key, 1).headOption 104 | } 105 | 106 | /** 107 | * This returns the closest n nodes in order for the object. There may be 108 | * duplicates. 109 | */ 110 | def find(key: Array[Byte], n: Int) = { 111 | if (sortedMap.isEmpty) Nil 112 | else { 113 | val (_, nodes) = (0 until n).foldLeft((hashFn.hash(key), List.empty[MemberAddress])) { 114 | case ((hash, members), idx) => 115 | val (newHash, candidate) = findCandidate(hash) 116 | (newHash + 1, candidate :: members) 117 | } 118 | nodes.reverse 119 | } 120 | } 121 | 122 | /** 123 | * This returns the closest n nodes in order for the object. There is extra 124 | * code that forces the node values to be unique. 125 | * 126 | * This will return a list that has all the nodes (and is smaller than n) if n 127 | * > number of nodes. 128 | */ 129 | def findUnique(key: Array[Byte], n: Int) = { 130 | if (sortedMap.isEmpty) Nil 131 | else { 132 | var duped = 0 133 | var hashIdx = hashFn.hash(key) 134 | val uniqueNodes = mutable.MutableList[MemberAddress]() 135 | var index = 0 136 | while (index < n) { 137 | val (newHash, candidate) = findCandidate(hashIdx) 138 | hashIdx = newHash 139 | if (!uniqueNodes.contains(candidate)) { 140 | duped = 0 141 | uniqueNodes += candidate 142 | } else { 143 | duped += 1 144 | index -= 1 // try again 145 | if (duped > MAX_DUPES) { 146 | index += 1; // we've been duped too many times, just skip to next, returning 147 | // fewer than n 148 | } 149 | } 150 | 151 | // was a hit so we increment and loop to find the next node in the circle 152 | hashIdx += 1 153 | index += 1 154 | } 155 | uniqueNodes.toList 156 | } 157 | } 158 | 159 | /** 160 | * Represent the ConsistentHashRing as [[RingState]] which is more easier to work with in terms of Ranges that each node manages. 161 | * 162 | * @return RingState 163 | */ 164 | def ringState = { 165 | import scala.collection.JavaConversions._ 166 | 167 | val firstToken = sortedMap.firstKey() 168 | val tokenRings = sortedMap.keysIterator.drop(1).foldLeft(RingState(firstToken, Nil)) { 169 | (state, token) => 170 | RingState( 171 | token, 172 | ranges = TokenRange(state.lastKnown, token - 1, sortedMap.get(state.lastKnown)) :: state.ranges) 173 | } 174 | RingState(Int.MaxValue, 175 | ranges = 176 | (TokenRange(tokenRings.lastKnown, 177 | firstToken - 1, 178 | sortedMap.get(tokenRings.lastKnown)) :: tokenRings.ranges).reverse) 179 | } 180 | 181 | private[partitioner] def findCandidate(hash: Integer) = { 182 | if (sortedMap.containsKey(hash)) { 183 | hash -> sortedMap.get(hash).node 184 | } else { 185 | val tailMap = sortedMap.tailMap(hash) 186 | val newHash = if (tailMap.isEmpty) sortedMap.firstKey() else tailMap.firstKey() 187 | newHash -> sortedMap.get(newHash).node 188 | } 189 | } 190 | 191 | // USED ONLY FOR TESTS 192 | private[partitioner] def nodes = sortedMap.values() 193 | } 194 | 195 | object ConsistentHashRing { 196 | def apply(hashFn: Hash, nodes: List[MemberAddress], partitionsPerNode: Int): ConsistentHashRing = 197 | new ConsistentHashRing(SuuchiHash, partitionsPerNode).init(nodes) 198 | 199 | def apply(nodes: List[MemberAddress], partitionsPerNode: Int): ConsistentHashRing = 200 | apply(SuuchiHash, nodes, partitionsPerNode) 201 | } 202 | --------------------------------------------------------------------------------