├── .env.sample ├── .gitignore ├── Procfile ├── bin └── get-dynamodb-local ├── build.sbt ├── perf-test.md ├── project └── plugins.sbt ├── readme.md └── src ├── it ├── resources │ └── application.conf └── scala │ └── akka │ └── persistence │ └── journal │ └── dynamodb │ └── DynamoDBLoadIntegrationSpec.scala ├── main ├── resources │ └── reference.conf └── scala │ └── akka │ └── persistence │ └── journal │ └── dynamodb │ ├── DynamoDBJournal.scala │ ├── DynamoDBRecovery.scala │ └── DynamoDBRequests.scala └── test ├── resources └── application.conf └── scala └── akka └── persistence └── journal └── dynamodb ├── DynamoDBJournalSpec.scala └── DynamoDBLoadSpec.scala /.env.sample: -------------------------------------------------------------------------------- 1 | DYNAMODB_RELEASE=2014-01-08 2 | LOG_LEVEL=INFO 3 | #INTEGRATION TEST ONLY 4 | JOURNAL_TABLE= 5 | AWS_ACCESS_KEY_ID= 6 | AWS_SECRET_ACCESS_KEY= 7 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | .idea_modules 3 | dynamodb-local 4 | .env 5 | target 6 | project/target 7 | project/project/target 8 | -------------------------------------------------------------------------------- /Procfile: -------------------------------------------------------------------------------- 1 | dynamo: java -Djava.library.path=dynamodb-local/dynamodb_local_$DYNAMODB_RELEASE/DynamoDBLocal_lib -jar dynamodb-local/dynamodb_local_$DYNAMODB_RELEASE/DynamoDBLocal.jar -inMemory -------------------------------------------------------------------------------- /bin/get-dynamodb-local: -------------------------------------------------------------------------------- 1 | #! /usr/bin/env bash 2 | 3 | ROOT=$(cd $(dirname $0)/..; pwd) 4 | 5 | mkdir -p $ROOT/dynamodb-local 6 | 7 | cd $ROOT/dynamodb-local 8 | 9 | curl -L http://dynamodb-local.s3-website-us-west-2.amazonaws.com/dynamodb_local_latest | tar xvz -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | organization := "com.sclasen" 2 | 3 | name := "akka-persistence-dynamodb" 4 | 5 | version := "0.3.4-SNAPSHOT" 6 | 7 | scalaVersion := "2.11.1" 8 | 9 | crossScalaVersions := Seq("2.11.1", "2.10.4") 10 | 11 | parallelExecution in Test := false 12 | 13 | //resolvers += "Sonatype OSS Snapshots" at "https://oss.sonatype.org/content/repositories/snapshots" 14 | 15 | resolvers += "spray repo" at "http://repo.spray.io" 16 | 17 | libraryDependencies += "com.sclasen" %% "spray-dynamodb" % "0.3.2" % "compile" 18 | 19 | libraryDependencies += "com.typesafe.akka" %% "akka-persistence-experimental" % "2.3.4" % "compile" 20 | 21 | libraryDependencies += "com.typesafe.akka" %% "akka-testkit" % "2.3.4" % "test,it" 22 | 23 | libraryDependencies += "org.scalatest" %% "scalatest" % "2.1.7" % "test,it" 24 | 25 | libraryDependencies += "commons-io" % "commons-io" % "2.4" % "test,it" 26 | 27 | resolvers += "krasserm at bintray" at "http://dl.bintray.com/krasserm/maven" 28 | 29 | libraryDependencies += "com.github.krasserm" %% "akka-persistence-testkit" % "0.3.2" % "test" 30 | 31 | parallelExecution in Test := false 32 | 33 | pomExtra := ( 34 | http://github.com/sclasen/akka-persistence-dynamodb 35 | 36 | 37 | The Apache Software License, Version 2.0 38 | http://www.apache.org/licenses/LICENSE-2.0.html 39 | repo 40 | 41 | 42 | 43 | git@github.com:sclasen/akka-persistence-dynamodb.git 44 | scm:git:git@github.com:sclasen/akka-persistence-dynamodb.git 45 | 46 | 47 | 48 | sclasen 49 | Scott Clasen 50 | http://github.com/sclasen 51 | 52 | ) 53 | 54 | 55 | publishTo <<= version { 56 | (v: String) => 57 | val nexus = "https://oss.sonatype.org/" 58 | if (v.trim.endsWith("SNAPSHOT")) Some("snapshots" at nexus + "content/repositories/snapshots") 59 | else Some("releases" at nexus + "service/local/staging/deploy/maven2") 60 | } 61 | 62 | 63 | val root = Project("akka-persistence-dynamodb", file(".")).configs(IntegrationTest).settings(Defaults.itSettings:_*) 64 | -------------------------------------------------------------------------------- /perf-test.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sclasen/akka-persistence-dynamodb/d68f53a2378775fde29a02f7992c36dc7dfa0499/perf-test.md -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers += "Typesafe repository" at "http://repo.typesafe.com/typesafe/releases/" 2 | 3 | addSbtPlugin("com.typesafe.sbt" % "sbt-scalariform" % "1.2.1") 4 | 5 | addSbtPlugin("com.typesafe.sbt" % "sbt-pgp" % "0.8.1") 6 | 7 | resolvers += "sbt-idea-repo" at "http://mpeltonen.github.com/maven/" 8 | 9 | addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.5.2") 10 | 11 | 12 | -------------------------------------------------------------------------------- /readme.md: -------------------------------------------------------------------------------- 1 | DynamoDBJournal for Akka Persistence 2 | ==================================== 3 | 4 | A replicated [Akka Persistence](http://doc.akka.io/docs/akka/2.3.0-RC3/scala/persistence.html) journal backed by 5 | [Amazon DynamoDB](http://aws.amazon.com/dynamodb/). 6 | 7 | Scala: `2.10.4` and `2.11.1` Akka: `2.3.3` 8 | 9 | Important Notice 10 | ---------------- 11 | 12 | ***Further development of this plugin has moved to the Akka organization, please file issues and submit pull requests against [akka/akka-persistence-dynamodb](https://github.com/akka/akka-persistence-dynamodb).*** In particular you will find versions built for current versions of Scala and Akka at the new location, published under the `com.typesafe.akka` groupId. 13 | 14 | Installation 15 | ------------ 16 | 17 | ```scala 18 | 19 | // build.sbt 20 | 21 | resolvers += "Sonatype OSS Snapshots" at "https://oss.sonatype.org/content/repositories/snapshots" 22 | 23 | libraryDependencies += "com.sclasen" %% "akka-persistence-dynamodb" % "0.3.4" % "compile" 24 | 25 | ``` 26 | 27 | Configuration 28 | ------------- 29 | 30 | ``` 31 | // application.conf - all config except endpoint, journal-name, sequence-shards is required 32 | 33 | akka.persistence.journal.plugin = "dynamodb-journal" 34 | 35 | dynamodb-journal { 36 | journal-table = "the name of a dynamodb table you create with hash key: `key`" 37 | # journal-name = "prefixes all keys, allows multiple journals per table, default: `journal`" 38 | aws-access-key-id = "yourKey" 39 | aws-secret-access-key = "yourSecret" 40 | operation-timeout = 10 seconds 41 | # endpoint = "defaults to https://dynamodb.us-east-1.amazonaws.com" 42 | sequence-shards = 1000 43 | } 44 | 45 | Note on `sequence-shards`: the high and low sequence numbers are stored across `sequence-shards` number of keys. 46 | The more shards used, the less likely that throttling will occur when writing at a high rate. Sequence shards should 47 | be set to as least as high as the write throughput of your table. 48 | 49 | The trade off with a higher number of shards is the number of requests needed to find the high or low sequence number for a processor. 50 | We can read the value of 100 shards per request to dynamodb, so reading 1000 shards takes 10 (parallel) requests, 10000 takes 100, etc. 51 | 52 | The reference.conf for this journal also contains the following settings for spray and akka. 53 | 54 | ``` 55 | spray.can.host-connector.max-connections = 600 56 | akka.persistence.journal.max-message-batch-size = 4000 57 | ``` 58 | 59 | ``` 60 | 61 | Development 62 | ----------- 63 | 64 | ### dev setup 65 | 66 | * install [forego](https://github.com/ddollar/forego) if you dont have it. 67 | 68 | * run `bin/get-dynamodb-local` 69 | 70 | this downloads and unpacks the dynamodb local to a subdir of ./dynamodb-local 71 | 72 | * `cp .env.sample .env` 73 | 74 | * make sure the DYNAMODB_RELEASE var in .env matches the date of the distro that was placed in ./dynamodb-local 75 | 76 | * `forego start` 77 | 78 | This starts the local dynamodb instance 79 | 80 | In another shell 81 | 82 | * forego run sbt test 83 | 84 | ### dynamodb table structure discussion 85 | 86 | the structure for journal storage in dynamodb has evolved over iterations of performance tuning. Most of these lessons were learned 87 | in creating the eventsourced dynamodb journal, but apply here as well. 88 | 89 | ##### naiive structure 90 | 91 | When initially modelling journal storage in dynamo, it seems natural to use a simple structure similar to this 92 | 93 | ``` 94 | processorId : S : HashKey 95 | sequenceNr : N : RangeKey 96 | deleted : S 97 | confirmations: SS 98 | payload : B 99 | ``` 100 | 101 | This maps very well to the operations a journal needs to solve. 102 | 103 | ``` 104 | writeMessage -> PutItem 105 | writeConfirmation -> UpdateItem with set add 106 | deleteMessage -> UpdateItem (mark deleted) or DeleteItem (permanent delete) 107 | replayMessages -> Query by processorId, conditions and ordered by sequenceNr, ascending 108 | highCounter -> Query by processorId, conditions and ordered by sequenceNr, descending limit 1 109 | ``` 110 | 111 | However this layout suffers from scalability problems. Since the hash key is used to locate the data storage node, all writes for a 112 | single processor will go to the same dynamodb node, which limits throughput and invites throttling, no matter the level of throughput provisioned 113 | for a table. The hash key just gets too hot. Also this limits replay throughput since you have to step through a sequence of queries, where 114 | you use the last processed item in query N for query N+1. 115 | 116 | ##### higher throughput structure. 117 | 118 | ``` 119 | P -> PersistentRepr 120 | SH -> SequenceHigh 121 | SL -> SequenceLow 122 | 123 | Persistent Data 124 | 125 | journalName"-P"-processorId-sequenceNr : S : HashKey 126 | deleted : S 127 | confirmations : SS 128 | payload : B 129 | 130 | High and Low Sequence Numbers 131 | 132 | journalName"-SH"-processorId-(sequenceNr % sequenceShards): S : HashKey 133 | sequenceNr : N 134 | 135 | journalName"-SL"-processorId-(sequenceNr % sequenceShards): S : HashKey 136 | sequenceNr : N 137 | ``` 138 | 139 | This is somewhat more difficult to code, but offers higher throughput possibilities. Notice that the items that hold the high and low sequence are sharded, 140 | rather than using a single item to store the counter. If we only used a single item, we would suffer from the same hot key problems as our 141 | first structure. 142 | 143 | ``` 144 | writeMessage -> BatchPutItem (P and SH) 145 | writeConfirmation -> UpdateItem with set add 146 | deleteMessage -> BatchPutItem (mark deleted, set SL) or BatchPutItem (permanent delete, set SL) 147 | replayMessages -> Parallel BatchGet all P items for the processor 148 | highCounter -> Parallel BatchGet all SH shards for the processor, find max. 149 | lowCounter -> Parallel BatchGet all SL shards for the processor, find min. 150 | ``` 151 | 152 | 153 | -------------------------------------------------------------------------------- /src/it/resources/application.conf: -------------------------------------------------------------------------------- 1 | dynamodb-journal { 2 | journal-table = ${JOURNAL_TABLE} 3 | aws-access-key-id = ${AWS_ACCESS_KEY_ID} 4 | aws-secret-access-key = ${AWS_SECRET_ACCESS_KEY} 5 | operation-timeout = 30 seconds 6 | sequence-shards = 10000 7 | } 8 | 9 | akka.persistence.journal.plugin = "dynamodb-journal" 10 | akka.persistence.publish-confirmations = on 11 | akka.persistence.publish-plugin-commands = on 12 | akka.loglevel = ${LOG_LEVEL} 13 | akka.event-handlers = ["akka.event.Logging$DefaultLogger"] 14 | -------------------------------------------------------------------------------- /src/it/scala/akka/persistence/journal/dynamodb/DynamoDBLoadIntegrationSpec.scala: -------------------------------------------------------------------------------- 1 | package akka.persistence.journal.dynamodb 2 | 3 | import scala.concurrent.duration._ 4 | import scala.util.control.NoStackTrace 5 | 6 | import akka.actor._ 7 | import akka.persistence._ 8 | import akka.testkit._ 9 | 10 | import org.scalatest._ 11 | import com.typesafe.config.{ConfigValueFactory, ConfigValue, ConfigFactory, Config} 12 | 13 | object DynamoDBIntegrationLoadSpec { 14 | 15 | trait Measure extends { this: Actor ⇒ 16 | val NanoToSecond = 1000.0 * 1000 * 1000 17 | 18 | var startTime: Long = 0L 19 | var stopTime: Long = 0L 20 | 21 | var startSequenceNr = 0L; 22 | var stopSequenceNr = 0L; 23 | 24 | def startMeasure(): Unit = { 25 | startSequenceNr = lastSequenceNr 26 | startTime = System.nanoTime 27 | } 28 | 29 | def stopMeasure(): Unit = { 30 | stopSequenceNr = lastSequenceNr 31 | stopTime = System.nanoTime 32 | sender ! (NanoToSecond * (stopSequenceNr - startSequenceNr) / (stopTime - startTime)) 33 | } 34 | 35 | def lastSequenceNr: Long 36 | } 37 | 38 | class ProcessorA(override val processorId: String) extends Processor with Measure { 39 | def receive = { 40 | case "start" => startMeasure() 41 | case "stop" => stopMeasure() 42 | case Persistent(payload: String, sequenceNr) => 43 | } 44 | } 45 | 46 | class ProcessorB(override val processorId: String, failAt: Option[Long]) extends Processor { 47 | def receive = { 48 | case Persistent(payload: String, sequenceNr) => 49 | failAt.foreach(snr => if (snr == sequenceNr) throw new Exception("boom") with NoStackTrace) 50 | sender ! s"${payload}-${sequenceNr}" 51 | } 52 | 53 | override def preRestart(reason: Throwable, message: Option[Any]): Unit = { 54 | message match { 55 | case Some(p: Persistent) => deleteMessage(p.sequenceNr) 56 | case _ => 57 | } 58 | super.preRestart(reason, message) 59 | } 60 | } 61 | 62 | def config:Config = { 63 | ConfigFactory.load(ActorSystem.findClassLoader()).withValue("dynamodb-journal.journal-name", ConfigValueFactory.fromAnyRef(System.currentTimeMillis().toString)) 64 | } 65 | } 66 | 67 | import DynamoDBIntegrationLoadSpec._ 68 | 69 | class Listener extends Actor { 70 | def receive = { 71 | case d: DeadLetter => println(d) 72 | } 73 | } 74 | 75 | class DynamoDBIntegrationLoadSpec extends TestKit(ActorSystem("test", config)) with ImplicitSender with WordSpecLike with Matchers { 76 | "A DynamoDB journal" should { 77 | "have some reasonable write throughput" in { 78 | val warmCycles = 10000L 79 | val loadCycles = 1000000L 80 | 81 | val listener = system.actorOf(Props(classOf[Listener])) 82 | //println dead letters to see what timesout/fails 83 | system.eventStream.subscribe(listener, classOf[DeadLetter]) 84 | 85 | val processor1 = system.actorOf(Props(classOf[ProcessorA], "p1a")) 86 | 1L to warmCycles foreach { i => processor1 ! Persistent("a") } 87 | processor1 ! "start" 88 | 1L to loadCycles foreach { i => 89 | processor1 ! Persistent("a") 90 | if(i % 1000 == 0 ) { 91 | Thread.sleep(10) 92 | println(i) 93 | } 94 | } 95 | processor1 ! "stop" 96 | expectMsgPF(1000 seconds) { case throughput: Double ⇒ println(f"\nthroughput = $throughput%.2f persistent commands per second") } 97 | } 98 | "work properly under load" in { 99 | val cycles = 1000L 100 | 101 | val processor1 = system.actorOf(Props(classOf[ProcessorB], "p1b", None)) 102 | 1L to cycles foreach { i => processor1 ! Persistent("a") } 103 | 1L to cycles foreach { i => expectMsg(s"a-${i}") } 104 | 105 | val processor2 = system.actorOf(Props(classOf[ProcessorB], "p1b", None)) 106 | 1L to cycles foreach { i => expectMsg(s"a-${i}") } 107 | 108 | processor2 ! Persistent("b") 109 | expectMsg(s"b-${cycles + 1L}") 110 | } 111 | "work properly under load and failure conditions" in { 112 | val cycles = 1000L 113 | val failAt = 217L 114 | 115 | val processor1 = system.actorOf(Props(classOf[ProcessorB], "p1c", Some(failAt))) 116 | 1L to cycles foreach { i => processor1 ! Persistent("a") } 117 | 1L until (failAt) foreach { i => expectMsg(s"a-${i}") } 118 | 1L to cycles foreach { i => if (i != failAt) expectMsg(s"a-${i}") } 119 | 120 | val processor2 = system.actorOf(Props(classOf[ProcessorB], "p1c", None)) 121 | 1L to cycles foreach { i => if (i != failAt) expectMsg(s"a-${i}") } 122 | 123 | processor2 ! Persistent("b") 124 | expectMsg(s"b-${cycles + 1L}") 125 | } 126 | } 127 | } -------------------------------------------------------------------------------- /src/main/resources/reference.conf: -------------------------------------------------------------------------------- 1 | dynamodb-journal { 2 | endpoint = "https://dynamodb.us-east-1.amazonaws.com" 3 | class = "akka.persistence.journal.dynamodb.DynamoDBJournal" 4 | replay-dispatcher = "akka.persistence.dispatchers.default-replay-dispatcher" 5 | plugin-dispatcher = "akka.actor.default-dispatcher" 6 | journal-name = "journal" 7 | sequence-shards = 1000 8 | } 9 | 10 | spray.can.host-connector.max-connections = 600 11 | akka.persistence.journal.max-message-batch-size = 4000 -------------------------------------------------------------------------------- /src/main/scala/akka/persistence/journal/dynamodb/DynamoDBJournal.scala: -------------------------------------------------------------------------------- 1 | package akka.persistence.journal.dynamodb 2 | 3 | import DynamoDBJournal._ 4 | import akka.actor.{ActorLogging, ActorRefFactory, ActorSystem} 5 | import akka.persistence._ 6 | import akka.persistence.journal.AsyncWriteJournal 7 | import akka.serialization.SerializationExtension 8 | import akka.util.ByteString 9 | import com.amazonaws.AmazonServiceException 10 | import com.amazonaws.services.dynamodbv2.model._ 11 | import com.sclasen.spray.aws.dynamodb.DynamoDBClient 12 | import com.sclasen.spray.aws.dynamodb.DynamoDBClientProps 13 | import com.typesafe.config.Config 14 | import java.nio.ByteBuffer 15 | import java.util.concurrent.TimeUnit 16 | import java.util.{HashMap => JHMap, Map => JMap} 17 | import scala.collection.immutable 18 | import scala.concurrent.Future 19 | import scala.concurrent.duration._ 20 | 21 | class DynamoDBJournal extends AsyncWriteJournal with DynamoDBRecovery with DynamoDBRequests with ActorLogging { 22 | 23 | val config = context.system.settings.config.getConfig(Conf) 24 | val extension = Persistence(context.system) 25 | val serialization = SerializationExtension(context.system) 26 | val dynamo = dynamoClient(context.system, context, config) 27 | val journalTable = config.getString(JournalTable) 28 | val journalName = config.getString(JournalName) 29 | val sequenceShards = config.getInt(SequenceShards) 30 | val maxDynamoBatchGet = 100 31 | val replayParallelism = 10 32 | 33 | type Item = JMap[String, AttributeValue] 34 | type ItemUpdates = JMap[String, AttributeValueUpdate] 35 | 36 | def asyncWriteMessages(messages: immutable.Seq[PersistentRepr]): Future[Unit] = writeMessages(messages) 37 | 38 | //do we need to store the confirmations in a separate key to avoid hot keys? 39 | def asyncWriteConfirmations(confirmations: immutable.Seq[PersistentConfirmation]): Future[Unit] = writeConfirmations(confirmations) 40 | 41 | def asyncDeleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Future[Unit] = deleteMessages(messageIds, permanent) 42 | 43 | def asyncDeleteMessagesTo(processorId: String, toSequenceNr: Long, permanent: Boolean): Future[Unit] = { 44 | log.debug("at=delete-messages-to processorId={} to={} perm={}", processorId, toSequenceNr, permanent) 45 | readLowestSequenceNr(processorId).flatMap { 46 | fromSequenceNr => 47 | val asyncDeletions = (fromSequenceNr to toSequenceNr).grouped(extension.settings.journal.maxDeletionBatchSize).map { 48 | group => 49 | asyncDeleteMessages(group.map(sequenceNr => PersistentIdImpl(processorId, sequenceNr)), permanent) 50 | } 51 | Future.sequence(asyncDeletions).map(_ => log.debug("finished asyncDeleteMessagesTo {} {} {}", processorId, toSequenceNr, permanent)) 52 | } 53 | } 54 | 55 | def fields[T](fs: (String, T)*): JMap[String, T] = { 56 | val map = new JHMap[String, T]() 57 | fs.foreach { 58 | case (k, v) => map.put(k, v) 59 | } 60 | map 61 | } 62 | 63 | def withBackoff[I, O](i: I, retriesRemaining: Int = 10)(op: I => Future[Either[AmazonServiceException, O]]): Future[O] = { 64 | op(i).flatMap { 65 | case Left(t: ProvisionedThroughputExceededException) => 66 | backoff(10 - retriesRemaining, i.getClass.getSimpleName) 67 | withBackoff(i, retriesRemaining - 1)(op) 68 | case Left(e) => 69 | log.error(e, "exception in withBackoff") 70 | throw e 71 | case Right(resp) => 72 | Future.successful(resp) 73 | } 74 | } 75 | 76 | def backoff(retries: Int, what: String) { 77 | if(retries == 0) Thread.`yield`() 78 | else { 79 | val sleep = math.pow(2, retries).toLong 80 | log.warning("at=backoff request={} sleep={}", what, sleep) 81 | Thread.sleep(sleep) 82 | } 83 | } 84 | 85 | def S(value: String): AttributeValue = new AttributeValue().withS(value) 86 | 87 | def S(value: Boolean): AttributeValue = new AttributeValue().withS(value.toString) 88 | 89 | def N(value: Long): AttributeValue = new AttributeValue().withN(value.toString) 90 | 91 | def SS(value: String): AttributeValue = new AttributeValue().withSS(value) 92 | 93 | def SS(values: Seq[String]): AttributeValue = new AttributeValue().withSS(values: _*) 94 | 95 | def B(value: Array[Byte]): AttributeValue = new AttributeValue().withB(ByteBuffer.wrap(value)) 96 | 97 | def US(value: String): AttributeValueUpdate = new AttributeValueUpdate().withAction(AttributeAction.ADD).withValue(SS(value)) 98 | 99 | def messageKey(procesorId: String, sequenceNr: Long) = S(str(journalName, "-P-", procesorId, "-", sequenceNr)) 100 | 101 | def highSeqKey(procesorId: String, sequenceNr: Long) = S(str(journalName, "-SH-", procesorId, "-", sequenceNr)) 102 | 103 | def lowSeqKey(procesorId: String, sequenceNr: Long) = S(str(journalName, "-SL-", procesorId, "-", sequenceNr)) 104 | 105 | def str(ss: Any*): String = ss.foldLeft(new StringBuilder)(_.append(_)).toString() 106 | 107 | def persistentToByteBuffer(p: PersistentRepr): ByteBuffer = 108 | ByteBuffer.wrap(serialization.serialize(p).get) 109 | 110 | def persistentFromByteBuffer(b: ByteBuffer): PersistentRepr = { 111 | serialization.deserialize(ByteString(b).toArray, classOf[PersistentRepr]).get 112 | } 113 | 114 | def logging[T](f: Future[T]): Future[T] = { 115 | f.onFailure { 116 | case e: Exception => 117 | log.error(e, "error in async op") 118 | e.printStackTrace 119 | } 120 | f 121 | } 122 | 123 | } 124 | 125 | class InstrumentedDynamoDBClient(props: DynamoDBClientProps) extends DynamoDBClient(props) { 126 | def logging[T](op: String)(f: Future[Either[AmazonServiceException, T]]): Future[Either[AmazonServiceException, T]] = { 127 | f.onFailure { 128 | case e: Exception => props.system.log.error(e, "error in async op {}", op) 129 | } 130 | f 131 | } 132 | 133 | override def batchWriteItem(awsWrite: BatchWriteItemRequest): Future[Either[AmazonServiceException, BatchWriteItemResult]] = 134 | logging("batchWriteItem")(super.batchWriteItem(awsWrite)) 135 | 136 | override def batchGetItem(awsGet: BatchGetItemRequest): Future[Either[AmazonServiceException, BatchGetItemResult]] = 137 | logging("batchGetItem")(super.batchGetItem(awsGet)) 138 | 139 | override def updateItem(aws: UpdateItemRequest): Future[Either[AmazonServiceException, UpdateItemResult]] = 140 | logging("updateItem")(super.updateItem(aws)) 141 | } 142 | 143 | object DynamoDBJournal { 144 | // field names 145 | val Key = "key" 146 | val ProcessorId = "processorId" 147 | val SequenceNr = "sequenceNr" 148 | val Confirmations = "confirmations" 149 | val Deleted = "deleted" 150 | val Payload = "payload" 151 | // config names 152 | val Conf = "dynamodb-journal" 153 | val JournalTable = "journal-table" 154 | val JournalName = "journal-name" 155 | val AwsKey = "aws-access-key-id" 156 | val AwsSecret = "aws-secret-access-key" 157 | val OpTimeout = "operation-timeout" 158 | val Endpoint = "endpoint" 159 | val ReplayDispatcher = "replay-dispatcher" 160 | val SequenceShards = "sequence-shards" 161 | 162 | import collection.JavaConverters._ 163 | 164 | val schema = Seq(new KeySchemaElement().withKeyType(KeyType.HASH).withAttributeName(Key)).asJava 165 | val schemaAttributes = Seq(new AttributeDefinition().withAttributeName(Key).withAttributeType("S")).asJava 166 | 167 | def dynamoClient(system: ActorSystem, context: ActorRefFactory, config: Config): DynamoDBClient = { 168 | val props = DynamoDBClientProps( 169 | config.getString(AwsKey), 170 | config.getString(AwsSecret), 171 | config.getDuration(OpTimeout, TimeUnit.MILLISECONDS) milliseconds, 172 | system, 173 | context, 174 | config.getString(Endpoint) 175 | ) 176 | new InstrumentedDynamoDBClient(props) 177 | } 178 | 179 | 180 | } -------------------------------------------------------------------------------- /src/main/scala/akka/persistence/journal/dynamodb/DynamoDBRecovery.scala: -------------------------------------------------------------------------------- 1 | package akka.persistence.journal.dynamodb 2 | 3 | import DynamoDBJournal._ 4 | import akka.persistence.PersistentRepr 5 | import akka.persistence.journal.AsyncRecovery 6 | import collection.JavaConverters._ 7 | import collection.immutable 8 | import com.amazonaws.services.dynamodbv2.model._ 9 | import java.util.Collections 10 | import java.util.{HashMap => JHMap, Map => JMap, List => JList} 11 | import scala.concurrent.Future 12 | 13 | 14 | trait DynamoDBRecovery extends AsyncRecovery { 15 | this: DynamoDBJournal => 16 | 17 | implicit lazy val replayDispatcher = context.system.dispatchers.lookup(config.getString(ReplayDispatcher)) 18 | 19 | def asyncReplayMessages(processorId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(replayCallback: (PersistentRepr) => Unit): Future[Unit] = logging { 20 | if (fromSequenceNr > toSequenceNr) return Future.successful(()) 21 | var delivered = 0L 22 | var maxDeliveredSeq = 0L 23 | getReplayBatch(processorId, fromSequenceNr).map { 24 | replayBatch => 25 | replayBatch.keys.foreach { 26 | case (sequenceNr, key) => 27 | val k = key.get(Key) 28 | Option(replayBatch.batch.get(k)).map { 29 | item => 30 | val repr = readPersistentRepr(item) 31 | repr.foreach { 32 | r => 33 | if (delivered < max && maxDeliveredSeq < toSequenceNr) { 34 | replayCallback(r) 35 | delivered += 1 36 | maxDeliveredSeq = r.sequenceNr 37 | log.debug("in=replay at=deliver {} {}", processorId, sequenceNr) 38 | } 39 | } 40 | } 41 | } 42 | replayBatch.batch.size() 43 | }.flatMap { 44 | last => 45 | if (last < maxDynamoBatchGet * replayParallelism || delivered >= max || maxDeliveredSeq >= toSequenceNr) { 46 | Future.successful(()) 47 | } else { 48 | val from = fromSequenceNr + maxDynamoBatchGet * replayParallelism 49 | asyncReplayMessages(processorId, from, toSequenceNr, max - delivered)(replayCallback) 50 | } 51 | } 52 | } 53 | 54 | case class ReplayBatch(keys: Stream[(Long, Item)], batch: JMap[AttributeValue, Item]) 55 | 56 | def getReplayBatch(processorId: String, fromSequenceNr: Long): Future[ReplayBatch] = { 57 | val batchKeys = Stream.iterate(fromSequenceNr, maxDynamoBatchGet * replayParallelism)(_ + 1).map(s => s -> fields(Key -> messageKey(processorId, s))) 58 | //there will be replayParallelism number of gets 59 | val gets = batchKeys.grouped(maxDynamoBatchGet).map { 60 | keys => 61 | val ka = new KeysAndAttributes().withKeys(keys.map(_._2).asJava).withConsistentRead(true).withAttributesToGet(Key, Payload, Deleted, Confirmations) 62 | val get = batchGetReq(Collections.singletonMap(journalTable, ka)) 63 | batchGet(get).flatMap(r => getUnprocessedItems(r)).map { 64 | result => mapBatch(result.getResponses.get(journalTable)) 65 | } 66 | } 67 | 68 | Future.sequence(gets).map { 69 | responses => 70 | val batch = responses.foldLeft(mapBatch(Collections.emptyList())) { 71 | case (all, one) => 72 | all.putAll(one) 73 | all 74 | } 75 | ReplayBatch(batchKeys, batch) 76 | } 77 | } 78 | 79 | def readPersistentRepr(item: JMap[String, AttributeValue]): Option[PersistentRepr] = { 80 | Option(item.get(Payload)).map { 81 | payload => 82 | val repr = persistentFromByteBuffer(payload.getB) 83 | val isDeleted = item.get(Deleted).getS == "true" 84 | val confirmations = item.asScala.get(Confirmations).map { 85 | ca => ca.getSS.asScala.to[immutable.Seq] 86 | }.getOrElse(immutable.Seq[String]()) 87 | repr.update(deleted = isDeleted, confirms = confirmations) 88 | } 89 | } 90 | 91 | def asyncReadHighestSequenceNr(processorId: String, fromSequenceNr: Long): Future[Long] = { 92 | log.debug("in=read-highest processorId={} from={}", processorId, fromSequenceNr) 93 | Future.sequence { 94 | Stream.iterate(0L, sequenceShards)(_ + 1).map(l => highSeqKey(processorId, l)).grouped(100).map { 95 | keys => 96 | val keyColl = keys.map(k => fields(Key -> k)).toSeq.asJava 97 | val ka = new KeysAndAttributes().withKeys(keyColl).withConsistentRead(true) 98 | val get = batchGetReq(Collections.singletonMap(journalTable, ka)) 99 | log.debug("in=read-highest at=batch-request") 100 | batchGet(get).flatMap(r => getUnprocessedItems(r)).map { 101 | resp => 102 | log.debug("in=read-highest at=batch-response") 103 | val batchMap = mapBatch(resp.getResponses.get(journalTable)) 104 | keys.map { 105 | key => 106 | Option(batchMap.get(key)).map(item => item.get(SequenceNr).getN.toLong) 107 | }.flatten.append(Stream(0L)).max 108 | } 109 | } 110 | }.map(_.max).map { 111 | max => 112 | log.debug("at=finish-read-high-sequence high={}", max) 113 | max 114 | } 115 | } 116 | 117 | def readLowestSequenceNr(processorId: String): Future[Long] = { 118 | log.debug("at=read-lowest-sequence processorId={}", processorId) 119 | Future.sequence { 120 | Stream.iterate(0L, sequenceShards)(_ + 1).map(l => lowSeqKey(processorId, l)).grouped(100).map { 121 | keys => 122 | val keyColl = keys.map(k => fields(Key -> k)).toSeq.asJava 123 | val ka = new KeysAndAttributes().withKeys(keyColl).withConsistentRead(true) 124 | val get = batchGetReq(Collections.singletonMap(journalTable, ka)) 125 | batchGet(get).flatMap(r => getUnprocessedItems(r)).map { 126 | resp => 127 | log.debug("at=read-lowest-sequence-batch-response processorId={}", processorId) 128 | val batchMap = mapBatch(resp.getResponses.get(journalTable)) 129 | val min: Long = keys.map { 130 | key => 131 | Option(batchMap.get(key)).map(item => item.get(SequenceNr).getN.toLong) 132 | }.flatten.append(Stream(Long.MaxValue)).min 133 | min 134 | } 135 | } 136 | }.map(_.min).map { 137 | min => 138 | log.debug("at=finish-read-lowest lowest={}", min) 139 | if (min == Long.MaxValue) 0 140 | else min 141 | } 142 | } 143 | 144 | def getUnprocessedItems(result: BatchGetItemResult, retriesRemaining: Int = 10): Future[BatchGetItemResult] = { 145 | val unprocessed = Option(result.getUnprocessedKeys.get(journalTable)).map(_.getKeys.size()).getOrElse(0) 146 | if (unprocessed == 0) Future.successful(result) 147 | else if (retriesRemaining == 0) { 148 | throw new RuntimeException(s"unable to batch get ${result} after 10 tries") 149 | } else { 150 | log.warning("at=unprocessed-reads, unprocessed={}", unprocessed) 151 | backoff(10 - retriesRemaining, classOf[BatchGetItemRequest].getSimpleName) 152 | val rest = batchGetReq(result.getUnprocessedKeys) 153 | batchGet(rest, retriesRemaining - 1).map { 154 | rr => 155 | val items = rr.getResponses.get(journalTable) 156 | val responses = result.getResponses.get(journalTable) 157 | items.asScala.foreach { 158 | i => responses.add(i) 159 | } 160 | result 161 | } 162 | } 163 | } 164 | 165 | def batchGet(r: BatchGetItemRequest, retriesRemaining: Int = 10): Future[BatchGetItemResult] = withBackoff(r, retriesRemaining)(dynamo.batchGetItem) 166 | 167 | def batchGetReq(items: JMap[String, KeysAndAttributes]) = new BatchGetItemRequest() 168 | .withRequestItems(items) 169 | .withReturnConsumedCapacity(ReturnConsumedCapacity.TOTAL) 170 | 171 | def mapBatch(b: JList[Item]): JMap[AttributeValue, Item] = { 172 | val map = new JHMap[AttributeValue, JMap[String, AttributeValue]] 173 | b.asScala.foreach { 174 | item => map.put(item.get(Key), item) 175 | } 176 | map 177 | } 178 | 179 | } 180 | 181 | 182 | -------------------------------------------------------------------------------- /src/main/scala/akka/persistence/journal/dynamodb/DynamoDBRequests.scala: -------------------------------------------------------------------------------- 1 | package akka.persistence.journal.dynamodb 2 | 3 | import DynamoDBJournal._ 4 | import akka.persistence.{PersistentConfirmation, PersistentId, PersistentRepr} 5 | import collection.JavaConverters._ 6 | import com.amazonaws.services.dynamodbv2.model._ 7 | import java.util.{HashMap => JHMap, Map => JMap, List => JList} 8 | import scala.collection.{mutable, immutable} 9 | import scala.concurrent.Future 10 | 11 | 12 | trait DynamoDBRequests { 13 | this: DynamoDBJournal => 14 | 15 | def writeMessages(messages: immutable.Seq[PersistentRepr]): Future[Unit] = unitSequence { 16 | // 25 is max items so group by 12 since 2 items per message 17 | // todo size calculation < 1M 18 | val writes = messages.grouped(12).map { 19 | msgs => 20 | val writes = msgs.foldLeft(new mutable.ArrayBuffer[WriteRequest](messages.length)) { 21 | case (ws, repr) => 22 | ws += putReq(toMsgItem(repr)) 23 | ws += putReq(toHSItem(repr)) 24 | ws 25 | } 26 | val reqItems = fields(journalTable -> writes.asJava) 27 | batchWriteReq(reqItems) 28 | } 29 | 30 | writes.map { 31 | write => 32 | batchWrite(write).flatMap(r => sendUnprocessedItems(r)).map { 33 | _ => if (log.isDebugEnabled) { 34 | log.debug("at=batch-write-finish writes={}", write.getRequestItems.get(journalTable).size()) 35 | } else () 36 | } 37 | } 38 | 39 | } 40 | 41 | private[dynamodb] def sendUnprocessedItems(result: BatchWriteItemResult, retriesRemaining: Int = 10): Future[BatchWriteItemResult] = { 42 | val unprocessed: Int = Option(result.getUnprocessedItems.get(JournalTable)).map(_.size()).getOrElse(0) 43 | if (unprocessed == 0) Future.successful(result) 44 | else if (retriesRemaining == 0) { 45 | throw new RuntimeException(s"unable to batch write ${result} after 10 tries") 46 | } else { 47 | log.warning("at=unprocessed-writes unprocessed={}", unprocessed) 48 | backoff(10 - retriesRemaining, classOf[BatchWriteItemRequest].getSimpleName) 49 | val rest = batchWriteReq(result.getUnprocessedItems) 50 | batchWrite(rest, retriesRemaining - 1).flatMap(r => sendUnprocessedItems(r, retriesRemaining - 1)) 51 | } 52 | } 53 | 54 | def putItem(r: PutItemRequest): Future[PutItemResult] = withBackoff(r)(dynamo.putItem) 55 | 56 | def deleteItem(r: DeleteItemRequest): Future[DeleteItemResult] = withBackoff(r)(dynamo.deleteItem) 57 | 58 | def updateItem(r: UpdateItemRequest): Future[UpdateItemResult] = withBackoff(r)(dynamo.updateItem) 59 | 60 | def batchWrite(r: BatchWriteItemRequest, retriesRemaining: Int = 10): Future[BatchWriteItemResult] = withBackoff(r, retriesRemaining)(dynamo.batchWriteItem) 61 | 62 | def writeConfirmations(confirmations: immutable.Seq[PersistentConfirmation]): Future[Unit] = unitSequence { 63 | confirmations.groupBy(c => (c.processorId, c.sequenceNr)).map { 64 | case ((processorId, sequenceNr), confirms) => 65 | val key = fields(Key -> messageKey(processorId, sequenceNr)) 66 | val update = fields(Confirmations -> setAdd(SS(confirmations.map(_.channelId)))) 67 | updateItem(updateReq(key, update)).map { 68 | result => log.debug("at=confirmed key={} update={}", key, update) 69 | } 70 | } 71 | } 72 | 73 | def deleteMessages(messageIds: immutable.Seq[PersistentId], permanent: Boolean): Future[Unit] = unitSequence { 74 | messageIds.map { 75 | msg => 76 | if (permanent) { 77 | deleteItem(permanentDeleteToDelete(msg)).map { 78 | _ => log.debug("at=permanent-delete-item processorId={} sequenceId={}", msg.processorId, msg.sequenceNr) 79 | } 80 | } else { 81 | updateItem(impermanentDeleteToUpdate(msg)).map { 82 | _ => log.debug("at=mark-delete-item processorId={} sequenceId={}", msg.processorId, msg.sequenceNr) 83 | } 84 | }.flatMap { 85 | _ => 86 | val item = toLSItem(msg) 87 | val put = new PutItemRequest().withTableName(journalTable).withItem(item) 88 | putItem(put).map(_ => log.debug("at=update-sequence-low-shard processorId={} sequenceId={}", msg.processorId, msg.sequenceNr)) 89 | } 90 | } 91 | } 92 | 93 | 94 | def toMsgItem(repr: PersistentRepr): Item = fields( 95 | Key -> messageKey(repr.processorId, repr.sequenceNr), 96 | Payload -> B(serialization.serialize(repr).get), 97 | Deleted -> S(false) 98 | ) 99 | 100 | def toHSItem(repr: PersistentRepr): Item = fields( 101 | Key -> highSeqKey(repr.processorId, repr.sequenceNr % sequenceShards), 102 | SequenceNr -> N(repr.sequenceNr) 103 | ) 104 | 105 | def toLSItem(id: PersistentId): Item = fields( 106 | Key -> lowSeqKey(id.processorId, id.sequenceNr % sequenceShards), 107 | SequenceNr -> N(id.sequenceNr) 108 | ) 109 | 110 | def putReq(item: Item): WriteRequest = new WriteRequest().withPutRequest(new PutRequest().withItem(item)) 111 | 112 | def deleteReq(item: Item): WriteRequest = new WriteRequest().withDeleteRequest(new DeleteRequest().withKey(item)) 113 | 114 | def updateReq(key: Item, updates: ItemUpdates): UpdateItemRequest = new UpdateItemRequest() 115 | .withTableName(journalTable) 116 | .withKey(key) 117 | .withAttributeUpdates(updates) 118 | .withReturnConsumedCapacity(ReturnConsumedCapacity.TOTAL) 119 | 120 | def setAdd(value: AttributeValue): AttributeValueUpdate = new AttributeValueUpdate().withAction(AttributeAction.ADD).withValue(value) 121 | 122 | def batchWriteReq(items: JMap[String, JList[WriteRequest]]) = new BatchWriteItemRequest() 123 | .withRequestItems(items) 124 | .withReturnConsumedCapacity(ReturnConsumedCapacity.TOTAL) 125 | 126 | def permanentDeleteToDelete(id: PersistentId): DeleteItemRequest = { 127 | log.debug("delete permanent {}", id) 128 | val key = fields(Key -> messageKey(id.processorId, id.sequenceNr)) 129 | new DeleteItemRequest().withTableName(journalTable).withKey(key) 130 | } 131 | 132 | def impermanentDeleteToUpdate(id: PersistentId): UpdateItemRequest = { 133 | log.debug("delete {}", id) 134 | val key = fields(Key -> messageKey(id.processorId, id.sequenceNr)) 135 | val updates = fields(Deleted -> new AttributeValueUpdate().withAction(AttributeAction.PUT).withValue(S(true))) 136 | new UpdateItemRequest().withTableName(journalTable).withKey(key).withAttributeUpdates(updates) 137 | } 138 | 139 | def unitSequence(seq: TraversableOnce[Future[Unit]]): Future[Unit] = Future.sequence(seq).map(_ => ()) 140 | 141 | } 142 | -------------------------------------------------------------------------------- /src/test/resources/application.conf: -------------------------------------------------------------------------------- 1 | dynamodb-journal { 2 | journal-table = "test-journal" 3 | journal-name = "journal" 4 | aws-access-key-id = "test123" 5 | aws-secret-access-key = "test123" 6 | operation-timeout = 10 seconds 7 | endpoint = "http://localhost:8000" 8 | } 9 | 10 | akka.persistence.journal.plugin = "dynamodb-journal" 11 | akka.persistence.publish-confirmations = on 12 | akka.persistence.publish-plugin-commands = on 13 | akka.loglevel = ${LOG_LEVEL} 14 | akka.event-handlers = ["akka.event.Logging$DefaultLogger"] 15 | -------------------------------------------------------------------------------- /src/test/scala/akka/persistence/journal/dynamodb/DynamoDBJournalSpec.scala: -------------------------------------------------------------------------------- 1 | package akka.persistence.journal.dynamodb 2 | 3 | import DynamoDBJournal._ 4 | import akka.actor.ActorSystem 5 | import akka.testkit.TestKit 6 | import com.amazonaws.services.dynamodbv2.model._ 7 | import scala.concurrent.Await 8 | import scala.concurrent.duration._ 9 | import org.scalatest.{Suite, BeforeAndAfterEach} 10 | import com.typesafe.config.{ConfigFactory, Config} 11 | import akka.persistence.journal.JournalSpec 12 | 13 | trait DynamoDBSpec extends BeforeAndAfterEach { 14 | this: Suite => 15 | 16 | val system:ActorSystem 17 | 18 | override def beforeEach(): Unit = { 19 | val config = system.settings.config.getConfig(Conf) 20 | val table = config.getString(JournalTable) 21 | val client = dynamoClient(system, system, config) 22 | val create = new CreateTableRequest() 23 | .withTableName(table) 24 | .withKeySchema(DynamoDBJournal.schema) 25 | .withAttributeDefinitions(DynamoDBJournal.schemaAttributes) 26 | .withProvisionedThroughput(new ProvisionedThroughput(10, 10)) 27 | import system.dispatcher 28 | 29 | val setup = client.sendListTables(new ListTablesRequest()).flatMap { 30 | list => 31 | if (list.getTableNames.size() > 0) { 32 | client.sendDeleteTable(new DeleteTableRequest(table)).flatMap { 33 | res => 34 | client.sendCreateTable(create).map(_ => ()) 35 | } 36 | } else { 37 | client.sendCreateTable(create).map(_ => ()) 38 | } 39 | } 40 | Await.result(setup, 5 seconds) 41 | super.beforeEach() 42 | } 43 | 44 | override protected def afterEach(): Unit = { 45 | super.afterEach() 46 | } 47 | } 48 | 49 | class DynamoDBJournalSpec extends JournalSpec with DynamoDBSpec{ 50 | override lazy val config: Config = ConfigFactory.load() 51 | } 52 | -------------------------------------------------------------------------------- /src/test/scala/akka/persistence/journal/dynamodb/DynamoDBLoadSpec.scala: -------------------------------------------------------------------------------- 1 | package akka.persistence.journal.dynamodb 2 | 3 | import scala.concurrent.duration._ 4 | import scala.util.control.NoStackTrace 5 | 6 | import akka.actor._ 7 | import akka.persistence._ 8 | import akka.testkit._ 9 | 10 | import org.scalatest._ 11 | 12 | object DynamoDBLoadSpec { 13 | 14 | trait Measure extends { this: Actor ⇒ 15 | val NanoToSecond = 1000.0 * 1000 * 1000 16 | 17 | var startTime: Long = 0L 18 | var stopTime: Long = 0L 19 | 20 | var startSequenceNr = 0L; 21 | var stopSequenceNr = 0L; 22 | 23 | def startMeasure(): Unit = { 24 | startSequenceNr = lastSequenceNr 25 | startTime = System.nanoTime 26 | } 27 | 28 | def stopMeasure(): Unit = { 29 | stopSequenceNr = lastSequenceNr 30 | stopTime = System.nanoTime 31 | sender ! (NanoToSecond * (stopSequenceNr - startSequenceNr) / (stopTime - startTime)) 32 | } 33 | 34 | def lastSequenceNr: Long 35 | } 36 | 37 | class ProcessorA(override val processorId: String) extends Processor with Measure { 38 | def receive = { 39 | case "start" => startMeasure() 40 | case "stop" => stopMeasure() 41 | case Persistent(payload: String, sequenceNr) => 42 | } 43 | } 44 | 45 | class ProcessorB(override val processorId: String, failAt: Option[Long]) extends Processor { 46 | def receive = { 47 | case Persistent(payload: String, sequenceNr) => 48 | failAt.foreach(snr => if (snr == sequenceNr) throw new Exception("boom") with NoStackTrace) 49 | sender ! s"${payload}-${sequenceNr}" 50 | } 51 | 52 | override def preRestart(reason: Throwable, message: Option[Any]): Unit = { 53 | message match { 54 | case Some(p: Persistent) => deleteMessage(p.sequenceNr) 55 | case _ => 56 | } 57 | super.preRestart(reason, message) 58 | } 59 | } 60 | } 61 | 62 | import DynamoDBLoadSpec._ 63 | 64 | class DynamoDBLoadSpec extends TestKit(ActorSystem("test")) with ImplicitSender with WordSpecLike with Matchers with DynamoDBSpec { 65 | "A DynamoDB journal" should { 66 | "have some reasonable write throughput" in { 67 | val warmCycles = 100L 68 | val loadCycles = 1000L 69 | 70 | val processor1 = system.actorOf(Props(classOf[ProcessorA], "p1a")) 71 | 1L to warmCycles foreach { i => processor1 ! Persistent("a") } 72 | processor1 ! "start" 73 | 1L to loadCycles foreach { i => processor1 ! Persistent("a") } 74 | processor1 ! "stop" 75 | expectMsgPF(100 seconds) { case throughput: Double ⇒ println(f"\nthroughput = $throughput%.2f persistent commands per second") } 76 | } 77 | "work properly under load" in { 78 | val cycles = 1000L 79 | 80 | val processor1 = system.actorOf(Props(classOf[ProcessorB], "p1b", None)) 81 | 1L to cycles foreach { i => processor1 ! Persistent("a") } 82 | 1L to cycles foreach { i => expectMsg(s"a-${i}") } 83 | 84 | val processor2 = system.actorOf(Props(classOf[ProcessorB], "p1b", None)) 85 | 1L to cycles foreach { i => expectMsg(s"a-${i}") } 86 | 87 | processor2 ! Persistent("b") 88 | expectMsg(s"b-${cycles + 1L}") 89 | } 90 | "work properly under load and failure conditions" in { 91 | val cycles = 1000L 92 | val failAt = 217L 93 | 94 | val processor1 = system.actorOf(Props(classOf[ProcessorB], "p1c", Some(failAt))) 95 | 1L to cycles foreach { i => processor1 ! Persistent("a") } 96 | 1L until (failAt) foreach { i => expectMsg(s"a-${i}") } 97 | 1L to cycles foreach { i => if (i != failAt) expectMsg(s"a-${i}") } 98 | 99 | val processor2 = system.actorOf(Props(classOf[ProcessorB], "p1c", None)) 100 | 1L to cycles foreach { i => if (i != failAt) expectMsg(s"a-${i}") } 101 | 102 | processor2 ! Persistent("b") 103 | expectMsg(s"b-${cycles + 1L}") 104 | } 105 | } 106 | } --------------------------------------------------------------------------------