Merge pull request #29495 from akka/feature-active-active-event-sourcing
Replicated event sourcing
This commit is contained in:
commit
d5bb125ae0
137 changed files with 22637 additions and 295 deletions
47
akka-actor/src/main/scala/akka/util/WallClock.scala
Normal file
47
akka-actor/src/main/scala/akka/util/WallClock.scala
Normal file
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util
|
||||
|
||||
import akka.annotation.ApiMayChange
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import java.util.function.LongUnaryOperator
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
/**
|
||||
* A time source.
|
||||
*/
|
||||
@ApiMayChange
|
||||
trait WallClock {
|
||||
def currentTimeMillis(): Long
|
||||
}
|
||||
|
||||
object WallClock {
|
||||
|
||||
/**
|
||||
* Always increasing time source. Based on `System.currentTimeMillis()` but
|
||||
* guaranteed to always increase for each invocation.
|
||||
*/
|
||||
val AlwaysIncreasingClock: WallClock = new AlwaysIncreasingClock()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API: Always increasing wall clock time.
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] final class AlwaysIncreasingClock() extends AtomicLong with WallClock {
|
||||
|
||||
override def currentTimeMillis(): Long = {
|
||||
val currentSystemTime = System.currentTimeMillis()
|
||||
updateAndGet {
|
||||
new LongUnaryOperator {
|
||||
override def applyAsLong(time: Long): Long = {
|
||||
if (currentSystemTime <= time) time + 1
|
||||
else currentSystemTime
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,2 @@
|
|||
# new private method on type that is do not inherit
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.sharding.typed.scaladsl.EntityRef.asJava")
|
||||
|
|
@ -12,7 +12,7 @@ abstract class Murmur2NoEnvelopeMessageExtractor[M](val numberOfShards: Int) ext
|
|||
}
|
||||
|
||||
/**
|
||||
* The murmur2 message extractor uses the same algorithm as the default kafka partitoiner
|
||||
* The murmur2 message extractor uses the same algorithm as the default kafka partitioner
|
||||
* allowing kafka partitions to be mapped to shards.
|
||||
* This can be used with the [[akka.cluster.sharding.external.ExternalShardAllocationStrategy]] to have messages
|
||||
* processed locally.
|
||||
|
|
|
|||
|
|
@ -0,0 +1,185 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import akka.cluster.sharding.typed.scaladsl.Entity
|
||||
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
|
||||
import akka.cluster.sharding.typed.javadsl.{ Entity => JEntity, EntityTypeKey => JEntityTypeKey }
|
||||
import akka.persistence.typed.ReplicaId
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.reflect.ClassTag
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
import java.util.{ Set => JSet }
|
||||
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.annotation.ApiMayChange
|
||||
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.ReplicationId.Separator
|
||||
|
||||
@ApiMayChange
|
||||
object ReplicatedEntityProvider {
|
||||
|
||||
/**
|
||||
* Java API:
|
||||
*
|
||||
* Provides full control over the [[ReplicatedEntity]] and the [[Entity]]
|
||||
* Most use cases can use the [[createPerDataCenter]] and [[createPerRole]]
|
||||
*
|
||||
* @tparam M The type of messages the replicated entity accepts
|
||||
*/
|
||||
def create[M](
|
||||
messageClass: Class[M],
|
||||
typeName: String,
|
||||
allReplicaIds: JSet[ReplicaId],
|
||||
settingsPerReplicaFactory: akka.japi.function.Function2[JEntityTypeKey[M], ReplicaId, ReplicatedEntity[M]])
|
||||
: ReplicatedEntityProvider[M] = {
|
||||
implicit val classTag: ClassTag[M] = ClassTag(messageClass)
|
||||
apply[M](typeName, allReplicaIds.asScala.toSet)((key, replica) =>
|
||||
settingsPerReplicaFactory(key.asInstanceOf[EntityTypeKeyImpl[M]], replica))
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API:
|
||||
*
|
||||
* Provides full control over the [[ReplicatedEntity]] and the [[Entity]]
|
||||
* Most use cases can use the [[perDataCenter]] and [[perRole]]
|
||||
*
|
||||
* @param typeName The type name used in the [[EntityTypeKey]]
|
||||
* @tparam M The type of messages the replicated entity accepts
|
||||
*/
|
||||
def apply[M: ClassTag](typeName: String, allReplicaIds: Set[ReplicaId])(
|
||||
settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId) => ReplicatedEntity[M]): ReplicatedEntityProvider[M] = {
|
||||
new ReplicatedEntityProvider(allReplicaIds.map { replicaId =>
|
||||
if (typeName.contains(Separator))
|
||||
throw new IllegalArgumentException(s"typeName [$typeName] contains [$Separator] which is a reserved character")
|
||||
|
||||
val typeKey = EntityTypeKey[M](s"$typeName${Separator}${replicaId.id}")
|
||||
(settingsPerReplicaFactory(typeKey, replicaId), typeName)
|
||||
}.toVector, directReplication = true)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API
|
||||
*
|
||||
* Create a [[ReplicatedEntityProvider]] that uses the defaults for [[Entity]] when running in
|
||||
* ClusterSharding. A replica will be run per data center.
|
||||
*/
|
||||
def perDataCenter[M: ClassTag, E](typeName: String, allReplicaIds: Set[ReplicaId])(
|
||||
create: ReplicationId => Behavior[M]): ReplicatedEntityProvider[M] = {
|
||||
apply(typeName, allReplicaIds) { (typeKey, replicaId) =>
|
||||
ReplicatedEntity(replicaId, Entity(typeKey) { entityContext =>
|
||||
create(ReplicationId.fromString(entityContext.entityId))
|
||||
}.withDataCenter(replicaId.id))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API
|
||||
*
|
||||
* Create a [[ReplicatedEntityProvider]] that uses the defaults for [[Entity]] when running in
|
||||
* ClusterSharding. The replicas in allReplicaIds should be roles used by nodes. A replica for each
|
||||
* entity will run on each role.
|
||||
*/
|
||||
def perRole[M: ClassTag, E](typeName: String, allReplicaIds: Set[ReplicaId])(
|
||||
create: ReplicationId => Behavior[M]): ReplicatedEntityProvider[M] = {
|
||||
apply(typeName, allReplicaIds) { (typeKey, replicaId) =>
|
||||
ReplicatedEntity(replicaId, Entity(typeKey) { entityContext =>
|
||||
create(ReplicationId.fromString(entityContext.entityId))
|
||||
}.withRole(replicaId.id))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*
|
||||
* Create a [[ReplicatedEntityProvider]] that uses the defaults for [[Entity]] when running in
|
||||
* ClusterSharding. A replica will be run per data center.
|
||||
*/
|
||||
def createPerDataCenter[M](
|
||||
messageClass: Class[M],
|
||||
typeName: String,
|
||||
allReplicaIds: JSet[ReplicaId],
|
||||
createBehavior: java.util.function.Function[ReplicationId, Behavior[M]]): ReplicatedEntityProvider[M] = {
|
||||
implicit val classTag: ClassTag[M] = ClassTag(messageClass)
|
||||
apply(typeName, allReplicaIds.asScala.toSet) { (typeKey, replicaId) =>
|
||||
ReplicatedEntity(replicaId, Entity(typeKey) { entityContext =>
|
||||
createBehavior(ReplicationId.fromString(entityContext.entityId))
|
||||
}.withDataCenter(replicaId.id))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*
|
||||
* Create a [[ReplicatedEntityProvider]] that uses the defaults for [[Entity]] when running in
|
||||
* ClusterSharding.
|
||||
*
|
||||
* Map replicas to roles and then there will be a replica per role e.g. to match to availability zones/racks
|
||||
*/
|
||||
def createPerRole[M](
|
||||
messageClass: Class[M],
|
||||
typeName: String,
|
||||
allReplicaIds: JSet[ReplicaId],
|
||||
createBehavior: akka.japi.function.Function[ReplicationId, Behavior[M]]): ReplicatedEntityProvider[M] = {
|
||||
implicit val classTag: ClassTag[M] = ClassTag(messageClass)
|
||||
apply(typeName, allReplicaIds.asScala.toSet) { (typeKey, replicaId) =>
|
||||
ReplicatedEntity(replicaId, Entity(typeKey) { entityContext =>
|
||||
createBehavior(ReplicationId.fromString(entityContext.entityId))
|
||||
}.withRole(replicaId.id))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @tparam M The type of messages the replicated entity accepts
|
||||
*/
|
||||
@ApiMayChange
|
||||
final class ReplicatedEntityProvider[M] private (
|
||||
val replicas: immutable.Seq[(ReplicatedEntity[M], String)],
|
||||
val directReplication: Boolean) {
|
||||
|
||||
/**
|
||||
* Start direct replication over sharding when replicated sharding starts up, requires the entities
|
||||
* to also have it enabled through [[akka.persistence.typed.scaladsl.EventSourcedBehavior.withEventPublishing]]
|
||||
* or [[akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior.withEventPublishing]]
|
||||
* to work.
|
||||
*
|
||||
*/
|
||||
def withDirectReplication(enabled: Boolean): ReplicatedEntityProvider[M] =
|
||||
new ReplicatedEntityProvider(replicas, directReplication = enabled)
|
||||
|
||||
}
|
||||
|
||||
@ApiMayChange
|
||||
object ReplicatedEntity {
|
||||
|
||||
/**
|
||||
* Java API: Defines the [[akka.cluster.sharding.typed.javadsl.Entity]] to use for a given replica, note that the behavior
|
||||
* can be a [[akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior]] or an arbitrary non persistent
|
||||
* [[akka.actor.typed.Behavior]] but must never be a regular [[akka.persistence.typed.javadsl.EventSourcedBehavior]]
|
||||
* as that requires a single writer and that would cause it to have multiple writers.
|
||||
*/
|
||||
def create[M](replicaId: ReplicaId, entity: JEntity[M, ShardingEnvelope[M]]): ReplicatedEntity[M] =
|
||||
apply(replicaId, entity.toScala)
|
||||
|
||||
/**
|
||||
* Scala API: Defines the [[akka.cluster.sharding.typed.scaladsl.Entity]] to use for a given replica, note that the behavior
|
||||
* can be a behavior created with [[akka.persistence.typed.scaladsl.ReplicatedEventSourcing]] or an arbitrary non persistent
|
||||
* [[akka.actor.typed.Behavior]] but must never be a regular [[akka.persistence.typed.scaladsl.EventSourcedBehavior]]
|
||||
* as that requires a single writer and that would cause it to have multiple writers.
|
||||
*/
|
||||
def apply[M](replicaId: ReplicaId, entity: Entity[M, ShardingEnvelope[M]]): ReplicatedEntity[M] =
|
||||
new ReplicatedEntity(replicaId, entity)
|
||||
}
|
||||
|
||||
/**
|
||||
* Settings for a specific replica id in replicated sharding
|
||||
* Currently only Entity's with ShardingEnvelope are supported but this may change in the future
|
||||
*/
|
||||
@ApiMayChange
|
||||
final class ReplicatedEntity[M] private (val replicaId: ReplicaId, val entity: Entity[M, ShardingEnvelope[M]])
|
||||
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.actor.typed.Extension
|
||||
import akka.actor.typed.ExtensionId
|
||||
import akka.annotation.ApiMayChange
|
||||
import akka.annotation.DoNotInherit
|
||||
import akka.cluster.sharding.typed.internal.ReplicatedShardingExtensionImpl
|
||||
import akka.cluster.sharding.typed.scaladsl.EntityRef
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import java.util.{ Map => JMap }
|
||||
|
||||
/**
|
||||
* Extension for running Replicated Event Sourcing in sharding by starting one separate instance of sharding per replica.
|
||||
* The sharding instances can be confined to datacenters or cluster roles or run on the same set of cluster nodes.
|
||||
*/
|
||||
@ApiMayChange
|
||||
object ReplicatedShardingExtension extends ExtensionId[ReplicatedShardingExtension] {
|
||||
|
||||
override def createExtension(system: ActorSystem[_]): ReplicatedShardingExtension =
|
||||
new ReplicatedShardingExtensionImpl(system)
|
||||
|
||||
def get(system: ActorSystem[_]): ReplicatedShardingExtension = apply(system)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Not for user extension.
|
||||
*/
|
||||
@DoNotInherit
|
||||
@ApiMayChange
|
||||
trait ReplicatedShardingExtension extends Extension {
|
||||
|
||||
/**
|
||||
* Init one instance sharding per replica in the given settings and return a [[ReplicatedSharding]] representing those.
|
||||
*
|
||||
* @tparam M The type of messages the replicated event sourced actor accepts
|
||||
*
|
||||
* Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]]
|
||||
*/
|
||||
def init[M](settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M]
|
||||
|
||||
/**
|
||||
* Init one instance sharding per replica in the given settings and return a [[ReplicatedSharding]] representing those.
|
||||
*
|
||||
* @param thisReplica If provided saves messages being forwarded to sharding for this replica
|
||||
* @tparam M The type of messages the replicated event sourced actor accepts
|
||||
*
|
||||
* Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]]
|
||||
*/
|
||||
def init[M](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M]
|
||||
}
|
||||
|
||||
/**
|
||||
* Represents the sharding instances for the replicas of one replicated event sourcing entity type
|
||||
*
|
||||
* Not for user extension.
|
||||
*/
|
||||
@DoNotInherit
|
||||
@ApiMayChange
|
||||
trait ReplicatedSharding[M] {
|
||||
|
||||
/**
|
||||
* Scala API: Returns the entity ref for each replica for user defined routing/replica selection
|
||||
*/
|
||||
def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]]
|
||||
|
||||
/**
|
||||
* Java API: Returns the entity ref for each replica for user defined routing/replica selection
|
||||
*/
|
||||
def getEntityRefsFor(entityId: String): JMap[ReplicaId, javadsl.EntityRef[M]]
|
||||
}
|
||||
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.eventstream.EventStream
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.annotation.DoNotInherit
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.typed.PublishedEvent
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Used when sharding Replicated Event Sourced entities in multiple instances of sharding, for example one per DC in a Multi DC
|
||||
* Akka Cluster.
|
||||
*
|
||||
* This actor should be started once on each node where Replicated Event Sourced entities will run (the same nodes that you start
|
||||
* sharding on). The entities should be set up with [[akka.persistence.typed.scaladsl.EventSourcedBehavior.withEventPublishing]]
|
||||
* or [[akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior#withEventPublishing()]]
|
||||
* If using [[ReplicatedSharding]] the replication can be enabled through [[ReplicatedEntityProvider.withDirectReplication]]
|
||||
* instead of starting this actor manually.
|
||||
*
|
||||
* Subscribes to locally written events through the event stream and sends the seen events to all the sharded replicas
|
||||
* which can then fast forward their cross-replica event streams to improve latency while allowing less frequent poll
|
||||
* for the cross replica queries. Note that since message delivery is at-most-once this can not be the only
|
||||
* channel for replica events - the entities must still tail events from the journals of other replicas.
|
||||
*
|
||||
* The events are forwarded as [[akka.cluster.sharding.typed.ShardingEnvelope]] this will work out of the box both
|
||||
* by default and with a custom extractor since the envelopes are handled internally.
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] object ShardingDirectReplication {
|
||||
|
||||
/**
|
||||
* Not for user extension
|
||||
*/
|
||||
@DoNotInherit
|
||||
sealed trait Command
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] case class VerifyStarted(replyTo: ActorRef[Done]) extends Command
|
||||
|
||||
private final case class WrappedPublishedEvent(publishedEvent: PublishedEvent) extends Command
|
||||
|
||||
def apply[T](selfReplica: Option[ReplicaId], replicaShardingProxies: Map[ReplicaId, ActorRef[T]]): Behavior[Command] =
|
||||
Behaviors.setup[Command] { context =>
|
||||
context.log.debug(
|
||||
"Subscribing to event stream to forward events to [{}] sharded replicas",
|
||||
replicaShardingProxies.size - 1)
|
||||
val publishedEventAdapter = context.messageAdapter[PublishedEvent](WrappedPublishedEvent.apply)
|
||||
context.system.eventStream ! EventStream.Subscribe[PublishedEvent](publishedEventAdapter)
|
||||
|
||||
Behaviors.receiveMessage {
|
||||
case WrappedPublishedEvent(event) =>
|
||||
context.log.trace(
|
||||
"Forwarding event for persistence id [{}] sequence nr [{}] to replicas",
|
||||
event.persistenceId,
|
||||
event.sequenceNumber)
|
||||
replicaShardingProxies.foreach {
|
||||
case (replica, proxy) =>
|
||||
val newId = ReplicationId.fromString(event.persistenceId.id).withReplica(replica)
|
||||
val envelopedEvent = ShardingEnvelope(newId.persistenceId.id, event)
|
||||
if (!selfReplica.contains(replica)) {
|
||||
proxy.asInstanceOf[ActorRef[ShardingEnvelope[PublishedEvent]]] ! envelopedEvent
|
||||
}
|
||||
}
|
||||
Behaviors.same
|
||||
case VerifyStarted(replyTo) =>
|
||||
replyTo ! Done
|
||||
Behaviors.same
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -385,6 +385,11 @@ import akka.util.JavaDurationConverters._
|
|||
}
|
||||
|
||||
override def toString: String = s"EntityRef($typeKey, $entityId)"
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def asJava: javadsl.EntityRef[M] = this
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed.internal
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import java.util.{ Map => JMap }
|
||||
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.annotation.InternalApi
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingExtension
|
||||
import akka.cluster.sharding.typed.ReplicatedSharding
|
||||
import akka.cluster.sharding.typed.ReplicatedEntityProvider
|
||||
import akka.cluster.sharding.typed.scaladsl.ClusterSharding
|
||||
import akka.cluster.sharding.typed.scaladsl.EntityRef
|
||||
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import org.slf4j.LoggerFactory
|
||||
import akka.actor.typed.scaladsl.LoggerOps
|
||||
import akka.cluster.ClusterSettings.DataCenter
|
||||
import akka.cluster.sharding.typed.ShardingDirectReplication
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_]) extends ReplicatedShardingExtension {
|
||||
|
||||
private val counter = new AtomicLong(0)
|
||||
|
||||
private val logger = LoggerFactory.getLogger(getClass)
|
||||
|
||||
override def init[M](settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] =
|
||||
initInternal(None, settings)
|
||||
|
||||
override def init[M](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] =
|
||||
initInternal(Some(thisReplica), settings)
|
||||
|
||||
private def initInternal[M](
|
||||
thisReplica: Option[ReplicaId],
|
||||
settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] = {
|
||||
val sharding = ClusterSharding(system)
|
||||
val initializedReplicas = settings.replicas.map {
|
||||
case (replicaSettings, typeName) =>
|
||||
// start up a sharding instance per replica id
|
||||
logger.infoN(
|
||||
"Starting Replicated Event Sourcing sharding for replica [{}] (ShardType: [{}])",
|
||||
replicaSettings.replicaId.id,
|
||||
replicaSettings.entity.typeKey.name)
|
||||
val regionOrProxy = sharding.init(replicaSettings.entity)
|
||||
(
|
||||
typeName,
|
||||
replicaSettings.replicaId,
|
||||
replicaSettings.entity.typeKey,
|
||||
regionOrProxy,
|
||||
replicaSettings.entity.dataCenter)
|
||||
}
|
||||
val replicaToRegionOrProxy = initializedReplicas.map {
|
||||
case (_, replicaId, _, regionOrProxy, _) => replicaId -> regionOrProxy
|
||||
}.toMap
|
||||
if (settings.directReplication) {
|
||||
logger.infoN("Starting Replicated Event Sourcing Direct Replication")
|
||||
system.systemActorOf(
|
||||
ShardingDirectReplication(thisReplica, replicaToRegionOrProxy),
|
||||
s"directReplication-${counter.incrementAndGet()}")
|
||||
}
|
||||
|
||||
val replicaToTypeKey = initializedReplicas.map {
|
||||
case (typeName, id, typeKey, _, dc) => id -> ((typeKey, dc, typeName))
|
||||
}.toMap
|
||||
new ReplicatedShardingImpl(sharding, replicaToTypeKey)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] final class ReplicatedShardingImpl[M](
|
||||
sharding: ClusterSharding,
|
||||
replicaTypeKeys: Map[ReplicaId, (EntityTypeKey[M], Option[DataCenter], String)])
|
||||
extends ReplicatedSharding[M] {
|
||||
|
||||
override def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] =
|
||||
replicaTypeKeys.map {
|
||||
case (replicaId, (typeKey, dc, typeName)) =>
|
||||
replicaId -> (dc match {
|
||||
case None => sharding.entityRefFor(typeKey, ReplicationId(typeName, entityId, replicaId).persistenceId.id)
|
||||
case Some(dc) =>
|
||||
sharding.entityRefFor(typeKey, ReplicationId(typeName, entityId, replicaId).persistenceId.id, dc)
|
||||
})
|
||||
}
|
||||
|
||||
override def getEntityRefsFor(entityId: String): JMap[ReplicaId, akka.cluster.sharding.typed.javadsl.EntityRef[M]] =
|
||||
entityRefsFor(entityId).transform((_, v) => v.asJava).asJava
|
||||
|
||||
}
|
||||
|
|
@ -15,6 +15,7 @@ import akka.actor.typed.Scheduler
|
|||
import akka.actor.typed.internal.InternalRecipientRef
|
||||
import akka.annotation.InternalApi
|
||||
import akka.cluster.sharding.typed.javadsl
|
||||
import akka.cluster.sharding.typed.javadsl.EntityRef
|
||||
import akka.cluster.sharding.typed.scaladsl
|
||||
import akka.japi.function.{ Function => JFunction }
|
||||
import akka.pattern.StatusReply
|
||||
|
|
@ -59,4 +60,6 @@ import akka.util.Timeout
|
|||
}
|
||||
|
||||
override def toString: String = s"TestEntityRef($entityId)"
|
||||
|
||||
override private[akka] def asJava: EntityRef[M] = this
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
|
|||
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
|
||||
import akka.japi.function.{ Function => JFunction }
|
||||
import akka.pattern.StatusReply
|
||||
|
||||
import scala.compat.java8.OptionConverters._
|
||||
@FunctionalInterface
|
||||
trait EntityFactory[M] {
|
||||
def apply(shardRegion: ActorRef[ClusterSharding.ShardCommand], entityId: String): Behavior[M]
|
||||
|
|
@ -185,6 +185,9 @@ abstract class ClusterSharding {
|
|||
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||
* here provided `entityId`.
|
||||
*
|
||||
* This can only be used if the default [[ShardingEnvelope]] is used, when using custom envelopes or in message
|
||||
* entity ids you will need to use the `ActorRef<E>` returned by sharding init for messaging with the sharded actors.
|
||||
*
|
||||
* For in-depth documentation of its semantics, see [[EntityRef]].
|
||||
*/
|
||||
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String): EntityRef[M]
|
||||
|
|
@ -197,6 +200,9 @@ abstract class ClusterSharding {
|
|||
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||
* provided `entityId`.
|
||||
*
|
||||
* This can only be used if the default [[ShardingEnvelope]] is used, when using custom envelopes or in message
|
||||
* entity ids you will need to use the `ActorRef[E]` returned by sharding init for messaging with the sharded actors.
|
||||
*
|
||||
* For in-depth documentation of its semantics, see [[EntityRef]].
|
||||
*/
|
||||
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String, dataCenter: String): EntityRef[M]
|
||||
|
|
@ -337,6 +343,22 @@ final class Entity[M, E] private (
|
|||
dataCenter)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] def toScala: akka.cluster.sharding.typed.scaladsl.Entity[M, E] =
|
||||
new akka.cluster.sharding.typed.scaladsl.Entity(
|
||||
eCtx => createBehavior(eCtx.toJava),
|
||||
typeKey.asScala,
|
||||
stopMessage.asScala,
|
||||
entityProps,
|
||||
settings.asScala,
|
||||
messageExtractor.asScala,
|
||||
allocationStrategy.asScala,
|
||||
role.asScala,
|
||||
dataCenter.asScala)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -419,7 +441,7 @@ object EntityTypeKey {
|
|||
*
|
||||
* Not for user extension.
|
||||
*/
|
||||
@DoNotInherit abstract class EntityRef[M] extends RecipientRef[M] {
|
||||
@DoNotInherit abstract class EntityRef[-M] extends RecipientRef[M] {
|
||||
scaladslSelf: scaladsl.EntityRef[M] with InternalRecipientRef[M] =>
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -185,6 +185,9 @@ trait ClusterSharding extends Extension { javadslSelf: javadsl.ClusterSharding =
|
|||
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||
* here provided `entityId`.
|
||||
*
|
||||
* This can only be used if the default [[ShardingEnvelope]] is used, when using custom envelopes or in message
|
||||
* entity ids you will need to use the `ActorRef[E]` returned by sharding init for messaging with the sharded actors.
|
||||
*
|
||||
* For in-depth documentation of its semantics, see [[EntityRef]].
|
||||
*/
|
||||
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String): EntityRef[M]
|
||||
|
|
@ -197,6 +200,9 @@ trait ClusterSharding extends Extension { javadslSelf: javadsl.ClusterSharding =
|
|||
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||
* here provided `entityId`.
|
||||
*
|
||||
* This can only be used if the default [[ShardingEnvelope]] is used, when using custom envelopes or in message
|
||||
* entity ids you will need to use the `ActorRef[E]` returned by sharding init for messaging with the sharded actors.
|
||||
*
|
||||
* For in-depth documentation of its semantics, see [[EntityRef]].
|
||||
*/
|
||||
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String, dataCenter: DataCenter): EntityRef[M]
|
||||
|
|
@ -348,7 +354,18 @@ final class Entity[M, E] private[akka] (
|
|||
final class EntityContext[M](
|
||||
val entityTypeKey: EntityTypeKey[M],
|
||||
val entityId: String,
|
||||
val shard: ActorRef[ClusterSharding.ShardCommand])
|
||||
val shard: ActorRef[ClusterSharding.ShardCommand]) {
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] def toJava: akka.cluster.sharding.typed.javadsl.EntityContext[M] =
|
||||
new akka.cluster.sharding.typed.javadsl.EntityContext[M](
|
||||
entityTypeKey.asInstanceOf[EntityTypeKeyImpl[M]],
|
||||
entityId,
|
||||
shard)
|
||||
}
|
||||
|
||||
/** Allows starting a specific Sharded Entity by its entity identifier */
|
||||
object StartEntity {
|
||||
|
|
@ -483,6 +500,11 @@ object EntityTypeKey {
|
|||
def ?[Res](message: ActorRef[Res] => M)(implicit timeout: Timeout): Future[Res] =
|
||||
this.ask(message)(timeout)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] def asJava: javadsl.EntityRef[M]
|
||||
|
||||
}
|
||||
|
||||
object ClusterShardingSetup {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,189 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import akka.actor.typed.scaladsl.LoggerOps
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.scaladsl.ActorContext
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.TestRES.GetState
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.TestRES.State
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.TestRES.StoreMe
|
||||
import akka.cluster.sharding.typed.scaladsl.Entity
|
||||
import akka.cluster.typed.MultiNodeTypedClusterSpec
|
||||
import akka.persistence.journal.PersistencePluginProxy
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
import org.scalatest.time.Span
|
||||
|
||||
object ReplicatedShardingSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.loglevel = DEBUG
|
||||
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
|
||||
// for the proxy plugin
|
||||
akka.actor.allow-java-serialization = on
|
||||
akka.actor.warn-about-java-serializer-usage = off
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
nodeConfig(first)(ConfigFactory.parseString("""
|
||||
akka.persistence.journal.plugin = "akka.persistence.journal.proxy"
|
||||
akka.persistence.journal.proxy {
|
||||
start-target-journal = on
|
||||
target-journal-plugin = "akka.persistence.journal.inmem"
|
||||
}
|
||||
"""))
|
||||
|
||||
nodeConfig(second)(ConfigFactory.parseString("""
|
||||
akka.persistence.journal.plugin = "akka.persistence.journal.proxy"
|
||||
akka.persistence.journal.proxy {
|
||||
start-target-journal = off
|
||||
target-journal-plugin = "akka.persistence.journal.inmem"
|
||||
}
|
||||
"""))
|
||||
|
||||
val AllReplicas = Set(ReplicaId("R1"), ReplicaId("R2"))
|
||||
|
||||
object TestRES {
|
||||
sealed trait Command extends CborSerializable
|
||||
case class GetState(replyTo: ActorRef[State]) extends Command
|
||||
case class StoreMe(description: String, replyTo: ActorRef[Done]) extends Command
|
||||
|
||||
case class State(all: List[String]) extends CborSerializable
|
||||
|
||||
def apply(id: ReplicationId, ctx: ActorContext[Command]): EventSourcedBehavior[Command, String, State] = {
|
||||
// Relies on direct replication as there is no proxy query journal
|
||||
ReplicatedEventSourcing.commonJournalConfig(id, AllReplicas, PersistenceTestKitReadJournal.Identifier) {
|
||||
replicationContext =>
|
||||
ctx.log.info("Creating replica {}", replicationContext.replicationId)
|
||||
EventSourcedBehavior[Command, String, State](
|
||||
replicationContext.persistenceId,
|
||||
State(Nil),
|
||||
(state, command) =>
|
||||
command match {
|
||||
case GetState(replyTo) =>
|
||||
replyTo ! state
|
||||
Effect.none
|
||||
case StoreMe(evt, ack) =>
|
||||
ctx.log.infoN("StoreMe {} {}", evt, replicationContext.replicationId)
|
||||
Effect.persist(evt).thenRun(_ => ack ! Done)
|
||||
},
|
||||
(state, event) => {
|
||||
ctx.log.infoN(
|
||||
"EventHandler [{}] origin [{}] at [{}]",
|
||||
event,
|
||||
replicationContext.origin,
|
||||
replicationContext.replicationId)
|
||||
state.copy(all = event :: state.all)
|
||||
}).withEventPublishing(true)
|
||||
}
|
||||
}
|
||||
|
||||
def provider(): ReplicatedEntityProvider[Command] = {
|
||||
ReplicatedEntityProvider[Command]("TestRES", AllReplicas) { (entityTypeKey, replicaId) =>
|
||||
ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext =>
|
||||
Behaviors.setup { ctx =>
|
||||
TestRES(ReplicationId.fromString(entityContext.entityId), ctx)
|
||||
}
|
||||
})
|
||||
}.withDirectReplication(true) // this is required as we don't have a shared read journal
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicatedShardingSpecMultiJvmNode1 extends ReplicatedShardingSpec
|
||||
class ReplicatedShardingSpecMultiJvmNode2 extends ReplicatedShardingSpec
|
||||
|
||||
abstract class ReplicatedShardingSpec
|
||||
extends MultiNodeSpec(ReplicatedShardingSpec)
|
||||
with MultiNodeTypedClusterSpec
|
||||
with ScalaFutures
|
||||
with Eventually {
|
||||
import ReplicatedShardingSpec._
|
||||
|
||||
implicit val patience: PatienceConfig =
|
||||
PatienceConfig(testKitSettings.DefaultTimeout.duration * 2, Span(500, org.scalatest.time.Millis))
|
||||
|
||||
"Replicated sharding" should {
|
||||
"form cluster" in {
|
||||
formCluster(first, second)
|
||||
enterBarrier("cluster-fored")
|
||||
}
|
||||
|
||||
"setup proxy plugin" in {
|
||||
PersistencePluginProxy.setTargetLocation(system, address(first))
|
||||
enterBarrier("proxy-setup")
|
||||
}
|
||||
|
||||
"start replicated entities" in {
|
||||
val replicatedSharding: ReplicatedSharding[TestRES.Command] =
|
||||
ReplicatedShardingExtension(typedSystem).init(TestRES.provider())
|
||||
|
||||
runOn(first) {
|
||||
val entityRefs = replicatedSharding.entityRefsFor("id1")
|
||||
val probe = TestProbe[Done]()
|
||||
entityRefs.size shouldEqual 2
|
||||
entityRefs.foreach {
|
||||
case (replica, ref) => ref ! StoreMe(s"from first to ${replica.id}", probe.ref)
|
||||
}
|
||||
probe.expectMessage(Done)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
eventually {
|
||||
entityRefs.foreach {
|
||||
case (_, ref) =>
|
||||
val probe = TestProbe[State]()
|
||||
ref ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set(s"from first to R1", s"from first to R2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
eventually {
|
||||
val probe = TestProbe[State]()
|
||||
replicatedSharding.entityRefsFor("id1").head._2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from first to R1", "from first to R2")
|
||||
}
|
||||
}
|
||||
enterBarrier("done")
|
||||
|
||||
runOn(second) {
|
||||
val entityRefs = replicatedSharding.entityRefsFor("id2")
|
||||
val probe = TestProbe[Done]()
|
||||
entityRefs.foreach {
|
||||
case (replica, ref) => ref ! StoreMe(s"from first to ${replica.id}", probe.ref)
|
||||
}
|
||||
probe.expectMessage(Done)
|
||||
probe.expectMessage(Done)
|
||||
}
|
||||
|
||||
runOn(first) {
|
||||
eventually {
|
||||
val probe = TestProbe[State]()
|
||||
replicatedSharding.entityRefsFor("id2").head._2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from first to R1", "from first to R2")
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrier("done-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,256 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed;
|
||||
|
||||
import akka.actor.testkit.typed.javadsl.LogCapturing;
|
||||
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
|
||||
import akka.actor.testkit.typed.javadsl.TestProbe;
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.actor.typed.javadsl.AbstractBehavior;
|
||||
import akka.actor.typed.javadsl.ActorContext;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.actor.typed.javadsl.Receive;
|
||||
import akka.cluster.MemberStatus;
|
||||
import akka.cluster.sharding.typed.javadsl.Entity;
|
||||
import akka.cluster.sharding.typed.javadsl.EntityRef;
|
||||
import akka.cluster.typed.Cluster;
|
||||
import akka.cluster.typed.Join;
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin;
|
||||
import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.javadsl.*;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.scalatestplus.junit.JUnitSuite;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static akka.cluster.sharding.typed.ReplicatedShardingTest.ProxyActor.ALL_REPLICAS;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class ReplicatedShardingTest extends JUnitSuite {
|
||||
|
||||
static class MyReplicatedStringSet
|
||||
extends ReplicatedEventSourcedBehavior<MyReplicatedStringSet.Command, String, Set<String>> {
|
||||
interface Command {}
|
||||
|
||||
static class Add implements Command {
|
||||
public final String text;
|
||||
|
||||
public Add(String text) {
|
||||
this.text = text;
|
||||
}
|
||||
}
|
||||
|
||||
static class GetTexts implements Command {
|
||||
public final ActorRef<Texts> replyTo;
|
||||
|
||||
public GetTexts(ActorRef<Texts> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static class Texts {
|
||||
public final Set<String> texts;
|
||||
|
||||
public Texts(Set<String> texts) {
|
||||
this.texts = texts;
|
||||
}
|
||||
}
|
||||
|
||||
static Behavior<Command> create(ReplicationId replicationId) {
|
||||
return ReplicatedEventSourcing.commonJournalConfig(
|
||||
replicationId,
|
||||
ALL_REPLICAS,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
MyReplicatedStringSet::new);
|
||||
}
|
||||
|
||||
private MyReplicatedStringSet(ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> emptyState() {
|
||||
return new HashSet<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandHandler<Command, String, Set<String>> commandHandler() {
|
||||
return newCommandHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onCommand(Add.class, add -> Effect().persist(add.text))
|
||||
.onCommand(
|
||||
GetTexts.class,
|
||||
(state, get) -> {
|
||||
// protective copy
|
||||
get.replyTo.tell(new Texts(new HashSet<>(state)));
|
||||
return Effect().none();
|
||||
})
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler<Set<String>, String> eventHandler() {
|
||||
return newEventHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onAnyEvent(
|
||||
(state, text) -> {
|
||||
state.add(text);
|
||||
return state;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
public static class ProxyActor extends AbstractBehavior<ProxyActor.Command> {
|
||||
interface Command {}
|
||||
|
||||
public static final class ForwardToRandom implements Command {
|
||||
public final String entityId;
|
||||
public final MyReplicatedStringSet.Command message;
|
||||
|
||||
public ForwardToRandom(String entityId, MyReplicatedStringSet.Command message) {
|
||||
this.entityId = entityId;
|
||||
this.message = message;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class ForwardToAll implements Command {
|
||||
public final String entityId;
|
||||
public final MyReplicatedStringSet.Command message;
|
||||
|
||||
public ForwardToAll(String entityId, MyReplicatedStringSet.Command message) {
|
||||
this.entityId = entityId;
|
||||
this.message = message;
|
||||
}
|
||||
}
|
||||
|
||||
public static Behavior<Command> create() {
|
||||
return Behaviors.setup(ProxyActor::new);
|
||||
}
|
||||
|
||||
public static final Set<ReplicaId> ALL_REPLICAS =
|
||||
Collections.unmodifiableSet(
|
||||
new HashSet<>(
|
||||
Arrays.asList(
|
||||
new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C"))));
|
||||
|
||||
private final ReplicatedSharding<MyReplicatedStringSet.Command> replicatedSharding;
|
||||
|
||||
private ProxyActor(ActorContext<Command> context) {
|
||||
super(context);
|
||||
|
||||
// #bootstrap
|
||||
ReplicatedEntityProvider<MyReplicatedStringSet.Command> replicatedEntityProvider =
|
||||
ReplicatedEntityProvider.create(
|
||||
MyReplicatedStringSet.Command.class,
|
||||
"StringSet",
|
||||
ALL_REPLICAS,
|
||||
// factory for replicated entity for a given replica
|
||||
(entityTypeKey, replicaId) ->
|
||||
ReplicatedEntity.create(
|
||||
replicaId,
|
||||
// use the replica id as typekey for sharding to get one sharding instance
|
||||
// per replica
|
||||
Entity.of(
|
||||
entityTypeKey,
|
||||
entityContext ->
|
||||
// factory for the entity for a given entity in that replica
|
||||
MyReplicatedStringSet.create(
|
||||
ReplicationId.fromString(entityContext.getEntityId())))
|
||||
// potentially use replica id as role or dc in Akka multi dc for the
|
||||
// sharding instance
|
||||
// to control where replicas will live
|
||||
// .withDataCenter(replicaId.id()))
|
||||
.withRole(replicaId.id())));
|
||||
|
||||
ReplicatedShardingExtension extension =
|
||||
ReplicatedShardingExtension.get(getContext().getSystem());
|
||||
ReplicatedSharding<MyReplicatedStringSet.Command> replicatedSharding =
|
||||
extension.init(replicatedEntityProvider);
|
||||
// #bootstrap
|
||||
|
||||
this.replicatedSharding = replicatedSharding;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Receive<Command> createReceive() {
|
||||
return newReceiveBuilder()
|
||||
.onMessage(ForwardToRandom.class, this::onForwardToRandom)
|
||||
.onMessage(ForwardToAll.class, this::onForwardToAll)
|
||||
.build();
|
||||
}
|
||||
|
||||
private Behavior<Command> onForwardToRandom(ForwardToRandom forwardToRandom) {
|
||||
Map<ReplicaId, EntityRef<MyReplicatedStringSet.Command>> refs =
|
||||
replicatedSharding.getEntityRefsFor(forwardToRandom.entityId);
|
||||
int chosenIdx = ThreadLocalRandom.current().nextInt(refs.size());
|
||||
new ArrayList<>(refs.values()).get(chosenIdx).tell(forwardToRandom.message);
|
||||
return this;
|
||||
}
|
||||
|
||||
private Behavior<Command> onForwardToAll(ForwardToAll forwardToAll) {
|
||||
// #all-entity-refs
|
||||
Map<ReplicaId, EntityRef<MyReplicatedStringSet.Command>> refs =
|
||||
replicatedSharding.getEntityRefsFor(forwardToAll.entityId);
|
||||
refs.forEach((replicaId, ref) -> ref.tell(forwardToAll.message));
|
||||
// #all-entity-refs
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@ClassRule
|
||||
public static final TestKitJunitResource testKit =
|
||||
new TestKitJunitResource(
|
||||
ConfigFactory.parseString(
|
||||
" akka.loglevel = DEBUG\n"
|
||||
+ " akka.loggers = [\"akka.testkit.SilenceAllTestEventListener\"]\n"
|
||||
+ " akka.actor.provider = \"cluster\"\n"
|
||||
+ " # pretend we're a node in all dc:s\n"
|
||||
+ " akka.cluster.roles = [\"DC-A\", \"DC-B\", \"DC-C\"]\n"
|
||||
+ " akka.remote.classic.netty.tcp.port = 0\n"
|
||||
+ " akka.remote.artery.canonical.port = 0")
|
||||
.withFallback(PersistenceTestKitPlugin.getInstance().config()));
|
||||
|
||||
@Rule public final LogCapturing logCapturing = new LogCapturing();
|
||||
|
||||
@Test
|
||||
public void formClusterAndInteractWithReplicas() {
|
||||
// join ourselves to form a one node cluster
|
||||
Cluster node = Cluster.get(testKit.system());
|
||||
node.manager().tell(new Join(node.selfMember().address()));
|
||||
TestProbe<Object> testProbe = testKit.createTestProbe();
|
||||
testProbe.awaitAssert(
|
||||
() -> {
|
||||
assertEquals(MemberStatus.up(), node.selfMember().status());
|
||||
return null;
|
||||
});
|
||||
|
||||
// forward messages to replicas
|
||||
ActorRef<ProxyActor.Command> proxy = testKit.spawn(ProxyActor.create());
|
||||
|
||||
proxy.tell(new ProxyActor.ForwardToAll("id1", new MyReplicatedStringSet.Add("to-all")));
|
||||
proxy.tell(new ProxyActor.ForwardToRandom("id1", new MyReplicatedStringSet.Add("to-random")));
|
||||
|
||||
testProbe.awaitAssert(
|
||||
() -> {
|
||||
TestProbe<MyReplicatedStringSet.Texts> responseProbe = testKit.createTestProbe();
|
||||
proxy.tell(
|
||||
new ProxyActor.ForwardToAll(
|
||||
"id1", new MyReplicatedStringSet.GetTexts(responseProbe.ref())));
|
||||
List<MyReplicatedStringSet.Texts> responses = responseProbe.receiveSeveralMessages(3);
|
||||
Set<String> uniqueTexts =
|
||||
responses.stream().flatMap(res -> res.texts.stream()).collect(Collectors.toSet());
|
||||
assertEquals(new HashSet<>(Arrays.asList("to-all", "to-random")), uniqueTexts);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.cluster.sharding.typed;
|
||||
|
||||
import akka.actor.typed.ActorSystem;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.cluster.sharding.typed.*;
|
||||
import akka.cluster.sharding.typed.javadsl.Entity;
|
||||
import akka.cluster.sharding.typed.javadsl.EntityRef;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
public class ReplicatedShardingCompileOnlySpec {
|
||||
|
||||
private static ActorSystem<?> system = null;
|
||||
|
||||
interface Command {}
|
||||
|
||||
public static Behavior<Command> myEventSourcedBehavior(ReplicationId replicationId) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public static final Set<ReplicaId> ALL_REPLICAS =
|
||||
Collections.unmodifiableSet(
|
||||
new HashSet<>(
|
||||
Arrays.asList(new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C"))));
|
||||
|
||||
public static ReplicatedEntityProvider<Command> provider() {
|
||||
// #bootstrap
|
||||
return ReplicatedEntityProvider.create(
|
||||
Command.class,
|
||||
"MyReplicatedType",
|
||||
ALL_REPLICAS,
|
||||
(entityTypeKey, replicaId) ->
|
||||
ReplicatedEntity.create(
|
||||
replicaId,
|
||||
Entity.of(
|
||||
entityTypeKey,
|
||||
entityContext ->
|
||||
myEventSourcedBehavior(
|
||||
ReplicationId.fromString(entityContext.getEntityId())))));
|
||||
|
||||
// #bootstrap
|
||||
}
|
||||
|
||||
public static void dc() {
|
||||
// #bootstrap-dc
|
||||
ReplicatedEntityProvider.create(
|
||||
Command.class,
|
||||
"MyReplicatedType",
|
||||
ALL_REPLICAS,
|
||||
(entityTypeKey, replicaId) ->
|
||||
ReplicatedEntity.create(
|
||||
replicaId,
|
||||
Entity.of(
|
||||
entityTypeKey,
|
||||
entityContext ->
|
||||
myEventSourcedBehavior(
|
||||
ReplicationId.fromString(entityContext.getEntityId())))
|
||||
.withDataCenter(replicaId.id())));
|
||||
|
||||
// #bootstrap-dc
|
||||
}
|
||||
|
||||
public static ReplicatedEntityProvider<Command> role() {
|
||||
// #bootstrap-role
|
||||
return ReplicatedEntityProvider.create(
|
||||
Command.class,
|
||||
"MyReplicatedType",
|
||||
ALL_REPLICAS,
|
||||
(entityTypeKey, replicaId) ->
|
||||
ReplicatedEntity.create(
|
||||
replicaId,
|
||||
Entity.of(
|
||||
entityTypeKey,
|
||||
entityContext ->
|
||||
myEventSourcedBehavior(
|
||||
ReplicationId.fromString(entityContext.getEntityId())))
|
||||
.withRole(replicaId.id())));
|
||||
|
||||
// #bootstrap-role
|
||||
}
|
||||
|
||||
public static void sendingMessages() {
|
||||
// #sending-messages
|
||||
ReplicatedShardingExtension extension = ReplicatedShardingExtension.get(system);
|
||||
|
||||
ReplicatedSharding<Command> replicatedSharding = extension.init(provider());
|
||||
|
||||
Map<ReplicaId, EntityRef<Command>> myEntityId =
|
||||
replicatedSharding.getEntityRefsFor("myEntityId");
|
||||
// #sending-messages
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.eventstream.EventStream
|
||||
import akka.persistence.typed
|
||||
import akka.persistence.typed.PublishedEvent
|
||||
import akka.persistence.typed.internal.{ PublishedEventImpl, ReplicatedPublishedEventMetaData, VersionVector }
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
|
||||
class ReplicatedShardingDirectReplicationSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
|
||||
|
||||
"Replicated sharding direct replication" must {
|
||||
|
||||
"replicate published events to all sharding proxies" in {
|
||||
val replicaAProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]()
|
||||
val replicaBProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]()
|
||||
val replicaCProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]()
|
||||
|
||||
val replicationActor = spawn(
|
||||
ShardingDirectReplication(
|
||||
Some(typed.ReplicaId("ReplicaA")),
|
||||
replicaShardingProxies = Map(
|
||||
ReplicaId("ReplicaA") -> replicaAProbe.ref,
|
||||
ReplicaId("ReplicaB") -> replicaBProbe.ref,
|
||||
ReplicaId("ReplicaC") -> replicaCProbe.ref)))
|
||||
|
||||
val upProbe = createTestProbe[Done]()
|
||||
replicationActor ! ShardingDirectReplication.VerifyStarted(upProbe.ref)
|
||||
upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough
|
||||
|
||||
val event = PublishedEventImpl(
|
||||
ReplicationId("ReplicatedShardingSpec", "pid", ReplicaId("ReplicaA")).persistenceId,
|
||||
1L,
|
||||
"event",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(ReplicaId("ReplicaA"), VersionVector.empty)))
|
||||
system.eventStream ! EventStream.Publish(event)
|
||||
|
||||
replicaBProbe.receiveMessage().message should equal(event)
|
||||
replicaCProbe.receiveMessage().message should equal(event)
|
||||
replicaAProbe.expectNoMessage() // no publishing to the replica emitting it
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,294 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
|
||||
import akka.actor.testkit.typed.scaladsl.ActorTestKit
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.cluster.MemberStatus
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.DataCenter
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.Normal
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.ReplicationType
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.Role
|
||||
import akka.cluster.sharding.typed.scaladsl.Entity
|
||||
import akka.cluster.typed.Cluster
|
||||
import akka.cluster.typed.Join
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
import akka.actor.typed.scaladsl.LoggerOps
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.MyReplicatedIntSet
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingSpec.MyReplicatedStringSet
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import com.typesafe.config.Config
|
||||
import akka.util.ccompat._
|
||||
@ccompatUsedUntil213
|
||||
object ReplicatedShardingSpec {
|
||||
def commonConfig = ConfigFactory.parseString("""
|
||||
akka.loglevel = INFO
|
||||
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
|
||||
akka.actor.provider = "cluster"
|
||||
akka.remote.classic.netty.tcp.port = 0
|
||||
akka.remote.artery.canonical.port = 0""").withFallback(PersistenceTestKitPlugin.config)
|
||||
|
||||
def roleAConfig = ConfigFactory.parseString("""
|
||||
akka.cluster.roles = ["DC-A"]
|
||||
""".stripMargin).withFallback(commonConfig)
|
||||
|
||||
def roleBConfig = ConfigFactory.parseString("""
|
||||
akka.cluster.roles = ["DC-B"]
|
||||
""".stripMargin).withFallback(commonConfig)
|
||||
|
||||
def dcAConfig = ConfigFactory.parseString("""
|
||||
akka.cluster.multi-data-center.self-data-center = "DC-A"
|
||||
""").withFallback(commonConfig)
|
||||
|
||||
def dcBConfig = ConfigFactory.parseString("""
|
||||
akka.cluster.multi-data-center.self-data-center = "DC-B"
|
||||
""").withFallback(commonConfig)
|
||||
|
||||
sealed trait ReplicationType
|
||||
case object Role extends ReplicationType
|
||||
case object DataCenter extends ReplicationType
|
||||
case object Normal extends ReplicationType
|
||||
|
||||
val AllReplicas = Set(ReplicaId("DC-A"), ReplicaId("DC-B"))
|
||||
|
||||
object MyReplicatedStringSet {
|
||||
trait Command extends CborSerializable
|
||||
case class Add(text: String) extends Command
|
||||
case class GetTexts(replyTo: ActorRef[Texts]) extends Command
|
||||
|
||||
case class Texts(texts: Set[String]) extends CborSerializable
|
||||
|
||||
def apply(replicationId: ReplicationId): Behavior[Command] =
|
||||
ReplicatedEventSourcing.commonJournalConfig( // it isn't really shared as it is in memory
|
||||
replicationId,
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, String, Set[String]](
|
||||
replicationContext.persistenceId,
|
||||
Set.empty[String],
|
||||
(state, command) =>
|
||||
command match {
|
||||
case Add(text) =>
|
||||
Effect.persist(text)
|
||||
case GetTexts(replyTo) =>
|
||||
replyTo ! Texts(state)
|
||||
Effect.none
|
||||
},
|
||||
(state, event) => state + event)
|
||||
.withJournalPluginId(PersistenceTestKitPlugin.PluginId)
|
||||
.withEventPublishing(true)
|
||||
}
|
||||
|
||||
def provider(replicationType: ReplicationType) =
|
||||
ReplicatedEntityProvider[MyReplicatedStringSet.Command](
|
||||
// all replicas
|
||||
"StringSet",
|
||||
AllReplicas) { (entityTypeKey, replicaId) =>
|
||||
// factory for replicated entity for a given replica
|
||||
val entity = {
|
||||
val e = Entity(entityTypeKey) { entityContext =>
|
||||
MyReplicatedStringSet(ReplicationId.fromString(entityContext.entityId))
|
||||
}
|
||||
replicationType match {
|
||||
case Role =>
|
||||
e.withRole(replicaId.id)
|
||||
case DataCenter =>
|
||||
e.withDataCenter(replicaId.id)
|
||||
case Normal =>
|
||||
e
|
||||
}
|
||||
}
|
||||
ReplicatedEntity(replicaId, entity)
|
||||
}.withDirectReplication(true)
|
||||
|
||||
}
|
||||
|
||||
object MyReplicatedIntSet {
|
||||
trait Command extends CborSerializable
|
||||
case class Add(text: Int) extends Command
|
||||
case class GetInts(replyTo: ActorRef[Ints]) extends Command
|
||||
case class Ints(ints: Set[Int]) extends CborSerializable
|
||||
|
||||
def apply(id: ReplicationId, allReplicas: Set[ReplicaId]): Behavior[Command] =
|
||||
ReplicatedEventSourcing.commonJournalConfig( // it isn't really shared as it is in memory
|
||||
id,
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, Int, Set[Int]](
|
||||
replicationContext.persistenceId,
|
||||
Set.empty[Int],
|
||||
(state, command) =>
|
||||
command match {
|
||||
case Add(int) =>
|
||||
Effect.persist(int)
|
||||
case GetInts(replyTo) =>
|
||||
replyTo ! Ints(state)
|
||||
Effect.none
|
||||
},
|
||||
(state, event) => state + event)
|
||||
.withJournalPluginId(PersistenceTestKitPlugin.PluginId)
|
||||
.withEventPublishing(true)
|
||||
}
|
||||
|
||||
def provider(replicationType: ReplicationType) =
|
||||
ReplicatedEntityProvider[MyReplicatedIntSet.Command]("IntSet", AllReplicas) { (entityTypeKey, replicaId) =>
|
||||
val entity = {
|
||||
val e = Entity(entityTypeKey) { entityContext =>
|
||||
val replicationId = ReplicationId.fromString(entityContext.entityId)
|
||||
MyReplicatedIntSet(replicationId, AllReplicas)
|
||||
}
|
||||
replicationType match {
|
||||
case Role =>
|
||||
e.withRole(replicaId.id)
|
||||
case DataCenter =>
|
||||
e.withDataCenter(replicaId.id)
|
||||
case Normal =>
|
||||
e
|
||||
}
|
||||
}
|
||||
ReplicatedEntity(replicaId, entity)
|
||||
}.withDirectReplication(true)
|
||||
}
|
||||
}
|
||||
|
||||
object ProxyActor {
|
||||
sealed trait Command
|
||||
case class ForwardToRandomString(entityId: String, msg: MyReplicatedStringSet.Command) extends Command
|
||||
case class ForwardToAllString(entityId: String, msg: MyReplicatedStringSet.Command) extends Command
|
||||
case class ForwardToRandomInt(entityId: String, msg: MyReplicatedIntSet.Command) extends Command
|
||||
case class ForwardToAllInt(entityId: String, msg: MyReplicatedIntSet.Command) extends Command
|
||||
|
||||
def apply(replicationType: ReplicationType): Behavior[Command] = Behaviors.setup { context =>
|
||||
val replicatedShardingStringSet =
|
||||
ReplicatedShardingExtension(context.system).init(MyReplicatedStringSet.provider(replicationType))
|
||||
val replicatedShardingIntSet =
|
||||
ReplicatedShardingExtension(context.system).init(MyReplicatedIntSet.provider(replicationType))
|
||||
|
||||
Behaviors.setup { ctx =>
|
||||
Behaviors.receiveMessage {
|
||||
case ForwardToAllString(entityId, cmd) =>
|
||||
val entityRefs = replicatedShardingStringSet.entityRefsFor(entityId)
|
||||
|
||||
ctx.log.infoN("Entity refs {}", entityRefs)
|
||||
|
||||
entityRefs.foreach {
|
||||
case (replica, ref) =>
|
||||
ctx.log.infoN("Forwarding to replica {} ref {}", replica, ref)
|
||||
ref ! cmd
|
||||
}
|
||||
Behaviors.same
|
||||
case ForwardToRandomString(entityId, cmd) =>
|
||||
val refs = replicatedShardingStringSet.entityRefsFor(entityId)
|
||||
val chosenIdx = ThreadLocalRandom.current().nextInt(refs.size)
|
||||
val chosen = refs.values.toIndexedSeq(chosenIdx)
|
||||
ctx.log.info("Forwarding to {}", chosen)
|
||||
chosen ! cmd
|
||||
Behaviors.same
|
||||
case ForwardToAllInt(entityId, cmd) =>
|
||||
replicatedShardingIntSet.entityRefsFor(entityId).foreach {
|
||||
case (_, ref) => ref ! cmd
|
||||
}
|
||||
Behaviors.same
|
||||
case ForwardToRandomInt(entityId, cmd) =>
|
||||
val refs =
|
||||
replicatedShardingIntSet.entityRefsFor(entityId)
|
||||
val chosenIdx = ThreadLocalRandom.current().nextInt(refs.size)
|
||||
refs.values.toIndexedSeq(chosenIdx) ! cmd
|
||||
Behaviors.same
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class NormalReplicatedShardingSpec
|
||||
extends ReplicatedShardingSpec(Normal, ReplicatedShardingSpec.commonConfig, ReplicatedShardingSpec.commonConfig)
|
||||
class RoleReplicatedShardingSpec
|
||||
extends ReplicatedShardingSpec(Role, ReplicatedShardingSpec.roleAConfig, ReplicatedShardingSpec.roleBConfig)
|
||||
class DataCenterReplicatedShardingSpec
|
||||
extends ReplicatedShardingSpec(DataCenter, ReplicatedShardingSpec.dcAConfig, ReplicatedShardingSpec.dcBConfig)
|
||||
|
||||
abstract class ReplicatedShardingSpec(replicationType: ReplicationType, configA: Config, configB: Config)
|
||||
extends ScalaTestWithActorTestKit(configA)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
|
||||
val system2 = ActorSystem(Behaviors.ignore[Any], name = system.name, config = configB)
|
||||
|
||||
override protected def afterAll(): Unit = {
|
||||
super.afterAll()
|
||||
ActorTestKit.shutdown(
|
||||
system2,
|
||||
testKitSettings.DefaultActorSystemShutdownTimeout,
|
||||
testKitSettings.ThrowOnShutdownTimeout)
|
||||
}
|
||||
|
||||
"Replicated sharding" should {
|
||||
|
||||
"form a one node cluster" in {
|
||||
Cluster(system).manager ! Join(Cluster(system).selfMember.address)
|
||||
Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
|
||||
|
||||
eventually {
|
||||
Cluster(system).state.members.unsorted.size should ===(2)
|
||||
Cluster(system).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
||||
}
|
||||
eventually {
|
||||
Cluster(system2).state.members.unsorted.size should ===(2)
|
||||
Cluster(system2).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
||||
}
|
||||
}
|
||||
|
||||
"start replicated sharding on both nodes" in {
|
||||
def start(sys: ActorSystem[_]) = {
|
||||
ReplicatedShardingExtension(sys).init(MyReplicatedStringSet.provider(replicationType))
|
||||
ReplicatedShardingExtension(sys).init(MyReplicatedIntSet.provider(replicationType))
|
||||
}
|
||||
start(system)
|
||||
start(system2)
|
||||
}
|
||||
|
||||
"forward to replicas" in {
|
||||
val proxy = spawn(ProxyActor(replicationType))
|
||||
|
||||
proxy ! ProxyActor.ForwardToAllString("id1", MyReplicatedStringSet.Add("to-all"))
|
||||
proxy ! ProxyActor.ForwardToRandomString("id1", MyReplicatedStringSet.Add("to-random"))
|
||||
|
||||
eventually {
|
||||
val probe = createTestProbe[MyReplicatedStringSet.Texts]()
|
||||
proxy ! ProxyActor.ForwardToAllString("id1", MyReplicatedStringSet.GetTexts(probe.ref))
|
||||
val responses: Seq[MyReplicatedStringSet.Texts] = probe.receiveMessages(2)
|
||||
val uniqueTexts = responses.flatMap(res => res.texts).toSet
|
||||
uniqueTexts should ===(Set("to-all", "to-random"))
|
||||
}
|
||||
|
||||
proxy ! ProxyActor.ForwardToAllInt("id1", MyReplicatedIntSet.Add(10))
|
||||
proxy ! ProxyActor.ForwardToRandomInt("id1", MyReplicatedIntSet.Add(11))
|
||||
|
||||
eventually {
|
||||
val probe = createTestProbe[MyReplicatedIntSet.Ints]()
|
||||
proxy ! ProxyActor.ForwardToAllInt("id1", MyReplicatedIntSet.GetInts(probe.ref))
|
||||
val responses: Seq[MyReplicatedIntSet.Ints] = probe.receiveMessages(2)
|
||||
val uniqueTexts = responses.flatMap(res => res.ints).toSet
|
||||
uniqueTexts should ===(Set(10, 11))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.akka.cluster.sharding.typed
|
||||
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.cluster.sharding.typed.ReplicatedEntity
|
||||
import akka.cluster.sharding.typed.ReplicatedEntityProvider
|
||||
import akka.cluster.sharding.typed.ReplicatedSharding
|
||||
import akka.cluster.sharding.typed.ReplicatedShardingExtension
|
||||
import akka.cluster.sharding.typed.scaladsl.Entity
|
||||
import akka.cluster.sharding.typed.scaladsl.EntityRef
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent("never used")
|
||||
object ReplicatedShardingCompileOnlySpec {
|
||||
|
||||
sealed trait Command
|
||||
|
||||
val system: ActorSystem[_] = ???
|
||||
|
||||
object MyEventSourcedBehavior {
|
||||
def apply(replicationId: ReplicationId): Behavior[Command] = ???
|
||||
}
|
||||
|
||||
//#bootstrap
|
||||
ReplicatedEntityProvider[Command]("MyEntityType", Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) {
|
||||
(entityTypeKey, replicaId) =>
|
||||
ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext =>
|
||||
// the sharding entity id contains the business entityId, entityType, and replica id
|
||||
// which you'll need to create a ReplicatedEventSourcedBehavior
|
||||
val replicationId = ReplicationId.fromString(entityContext.entityId)
|
||||
MyEventSourcedBehavior(replicationId)
|
||||
})
|
||||
}
|
||||
//#bootstrap
|
||||
|
||||
//#bootstrap-dc
|
||||
ReplicatedEntityProvider.perDataCenter("MyEntityType", Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { replicationId =>
|
||||
MyEventSourcedBehavior(replicationId)
|
||||
}
|
||||
//#bootstrap-dc
|
||||
|
||||
//#bootstrap-role
|
||||
val provider = ReplicatedEntityProvider.perRole("MyEntityType", Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) {
|
||||
replicationId =>
|
||||
MyEventSourcedBehavior(replicationId)
|
||||
}
|
||||
//#bootstrap-role
|
||||
|
||||
//#sending-messages
|
||||
val myReplicatedSharding: ReplicatedSharding[Command] =
|
||||
ReplicatedShardingExtension(system).init(provider)
|
||||
|
||||
val entityRefs: Map[ReplicaId, EntityRef[Command]] = myReplicatedSharding.entityRefsFor("myEntityId")
|
||||
//#sending-messages
|
||||
}
|
||||
|
|
@ -632,7 +632,8 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
|
|||
case null =>
|
||||
proxies.get(proxyName(typeName, None)) match {
|
||||
case null =>
|
||||
throw new IllegalStateException(s"Shard type [$typeName] must be started first")
|
||||
throw new IllegalStateException(
|
||||
s"Shard type [$typeName] must be started first. Started ${regions.keySet()} proxies ${proxies.keySet()}")
|
||||
case ref => ref
|
||||
}
|
||||
case ref => ref
|
||||
|
|
|
|||
|
|
@ -358,7 +358,7 @@ private[akka] object Shard {
|
|||
// only called once during handoff
|
||||
def activeEntities(): Set[ActorRef] = byRef.keySet.asScala.toSet
|
||||
|
||||
def nrActiveEntities: Int = byRef.size
|
||||
def nrActiveEntities(): Int = byRef.size()
|
||||
|
||||
// only called for getting shard stats
|
||||
def activeEntityIds(): Set[EntityId] = byRef.values.asScala.toSet
|
||||
|
|
@ -1061,7 +1061,7 @@ private[akka] class Shard(
|
|||
* of active entities.
|
||||
*/
|
||||
@InternalStableApi
|
||||
def entityCreated(@unused id: EntityId): Int = entities.nrActiveEntities
|
||||
def entityCreated(@unused id: EntityId): Int = entities.nrActiveEntities()
|
||||
|
||||
// ===== buffering while busy saving a start or stop when remembering entities =====
|
||||
def appendToMessageBuffer(id: EntityId, msg: Any, snd: ActorRef): Unit = {
|
||||
|
|
|
|||
|
|
@ -41,9 +41,9 @@ class Member private[cluster] (
|
|||
}
|
||||
override def toString =
|
||||
if (dataCenter == ClusterSettings.DefaultDataCenter)
|
||||
s"Member(address = $address, status = $status)"
|
||||
s"Member(address = $address, status = $status, roles = $roles)"
|
||||
else
|
||||
s"Member(address = $address, dataCenter = $dataCenter, status = $status)"
|
||||
s"Member(address = $address, dataCenter = $dataCenter, status = $status, roles = $roles)"
|
||||
|
||||
def hasRole(role: String): Boolean = roles.contains(role)
|
||||
|
||||
|
|
|
|||
|
|
@ -566,9 +566,11 @@ Scala
|
|||
Java
|
||||
: @@snip [LambdaPersistenceDocTest.java](/akka-docs/src/test/java/jdocs/persistence/LambdaPersistenceDocTest.java) { #save-snapshot }
|
||||
|
||||
where `metadata` is of type `SnapshotMetadata`:
|
||||
where `metadata` is of type `SnapshotMetadata` and contains:
|
||||
|
||||
@@snip [SnapshotProtocol.scala](/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala) { #snapshot-metadata }
|
||||
* persistenceId
|
||||
* sequenceNr
|
||||
* timestamp
|
||||
|
||||
During recovery, the persistent actor is offered the latest saved snapshot via a `SnapshotOffer` message from
|
||||
which it can initialize internal state.
|
||||
|
|
|
|||
|
|
@ -61,11 +61,7 @@ from the events, or publish the events to other services.
|
|||
|
||||
## Multi-DC Persistence
|
||||
|
||||
@java[@extref[Multi-DC Persistence example project](samples:akka-samples-persistence-dc-java)]
|
||||
@scala[@extref[Multi-DC Persistence example project](samples:akka-samples-persistence-dc-scala)]
|
||||
|
||||
Illustrates how to use Lightbend's [Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
|
||||
with active-active persistent entities across data centers.
|
||||
This commercial feature has now been superseded by @ref[Replicated Event Sourcing](../typed/replicated-eventsourcing.md)
|
||||
|
||||
## Cluster with Docker
|
||||
|
||||
|
|
|
|||
|
|
@ -458,6 +458,20 @@ For the `jackson-cbor` and custom bindings other than `jackson-json` compression
|
|||
but can be enabled in the same way as the configuration shown above but replacing `jackson-json` with
|
||||
the binding name (for example `jackson-cbor`).
|
||||
|
||||
## Using Akka Serialization for embedded types
|
||||
|
||||
For types that already have an Akka Serializer defined that are embedded in types serialized with Jackson the @apidoc[AkkaSerializationSerializer] and
|
||||
@apidoc[AkkaSerializationDeserializer] can be used to Akka Serialization for individual fields.
|
||||
|
||||
The serializer/deserializer are not enabled automatically. The `@JsonSerialize` and `@JsonDeserialize` annotation needs to be added
|
||||
to the fields containing the types to be serialized with Akka Serialization.
|
||||
|
||||
The type will be embedded as an object with the fields:
|
||||
|
||||
* serId - the serializer id
|
||||
* serManifest - the manifest for the type
|
||||
* payload - base64 encoded bytes
|
||||
|
||||
## Additional configuration
|
||||
|
||||
### Configuration per binding
|
||||
|
|
|
|||
|
|
@ -192,8 +192,8 @@ other data centers.
|
|||
Especially when used together with Akka Persistence that is based on the single-writer principle
|
||||
it is important to avoid running the same entity at multiple locations at the same time with a
|
||||
shared data store. That would result in corrupt data since the events stored by different instances
|
||||
may be interleaved and would be interpreted differently in a later replay. For active active persistent
|
||||
entities see Lightbend's [Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
|
||||
may be interleaved and would be interpreted differently in a later replay. For replicated persistent
|
||||
entities see @ref[Replicated Event Sourcing](replicated-eventsourcing.md).
|
||||
|
||||
If you need global entities you have to pick one data center to host that entity type and only start
|
||||
`ClusterSharding` on nodes of that data center. If the data center is unreachable from another data center the
|
||||
|
|
|
|||
|
|
@ -21,12 +21,10 @@ With a [Lightbend Platform Subscription](https://www.lightbend.com/lightbend-sub
|
|||
* [Configuration Checker](https://doc.akka.io/docs/akka-enhancements/current/config-checker.html) — Checks for potential configuration issues and logs suggestions.
|
||||
* [Diagnostics Recorder](https://doc.akka.io/docs/akka-enhancements/current/diagnostics-recorder.html) — Captures configuration and system information in a format that makes it easy to troubleshoot issues during development and production.
|
||||
* [Thread Starvation Detector](https://doc.akka.io/docs/akka-enhancements/current/starvation-detector.html) — Monitors an Akka system dispatcher and logs warnings if it becomes unresponsive.
|
||||
* [Kubernetes Lease](https://doc.akka.io/docs/akka-enhancements/current/kubernetes-lease.html) — Monitors an Akka system dispatcher and logs warnings if it becomes unresponsive.
|
||||
* [Fast Failover](https://doc.akka.io/docs/akka-enhancements/current/fast-failover.html) — Fast failover for Cluster Sharding.
|
||||
|
||||
[Akka Persistence Enhancements](https://doc.akka.io/docs/akka-enhancements/current/akka-persistence-enhancements.html):
|
||||
|
||||
* [Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html) — For active-active persistent entities across multiple data centers.
|
||||
* [GDPR for Akka Persistence](https://doc.akka.io/docs/akka-enhancements/current/gdpr/index.html) — Data shredding can be used to forget information in events.
|
||||
|
||||
This page does not list all available modules, but overviews the main functionality and gives you an idea of the level of sophistication you can reach when you start building systems on top of Akka.
|
||||
|
|
|
|||
BIN
akka-docs/src/main/paradox/typed/images/causality.png
Normal file
BIN
akka-docs/src/main/paradox/typed/images/causality.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 283 KiB |
BIN
akka-docs/src/main/paradox/typed/images/lww.png
Normal file
BIN
akka-docs/src/main/paradox/typed/images/lww.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 104 KiB |
BIN
akka-docs/src/main/paradox/typed/images/replicated-events1.png
Normal file
BIN
akka-docs/src/main/paradox/typed/images/replicated-events1.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 57 KiB |
BIN
akka-docs/src/main/paradox/typed/images/replicated-events2.png
Normal file
BIN
akka-docs/src/main/paradox/typed/images/replicated-events2.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 69 KiB |
|
|
@ -9,6 +9,7 @@ project.description: Event Sourcing with Akka Persistence enables actors to pers
|
|||
@@@ index
|
||||
|
||||
* [persistence](persistence.md)
|
||||
* [replicated-eventsourcing](replicated-eventsourcing.md)
|
||||
* [cqrs](cqrs.md)
|
||||
* [persistence-style](persistence-style.md)
|
||||
* [persistence-snapshot](persistence-snapshot.md)
|
||||
|
|
@ -19,5 +20,6 @@ project.description: Event Sourcing with Akka Persistence enables actors to pers
|
|||
* [persistence-query-leveldb](../persistence-query-leveldb.md)
|
||||
* [persistence-plugins](../persistence-plugins.md)
|
||||
* [persistence-journals](../persistence-journals.md)
|
||||
* [replicated-eventsourcing-examples](replicated-eventsourcing-examples.md)
|
||||
|
||||
@@@
|
||||
|
|
|
|||
|
|
@ -657,9 +657,9 @@ cluster and address them by id.
|
|||
Akka Persistence is based on the single-writer principle. For a particular `PersistenceId` only one `EventSourcedBehavior`
|
||||
instance should be active at one time. If multiple instances were to persist events at the same time, the events would
|
||||
be interleaved and might not be interpreted correctly on replay. Cluster Sharding ensures that there is only one
|
||||
active entity (`EventSourcedBehavior`) for each id within a data center. Lightbend's
|
||||
[Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
|
||||
supports active-active persistent entities across data centers.
|
||||
active entity (`EventSourcedBehavior`) for each id within a data center.
|
||||
@ref:[Replicated Event Sourcing](replicated-eventsourcing.md) supports active-active persistent entities across
|
||||
data centers.
|
||||
|
||||
## Configuration
|
||||
|
||||
|
|
@ -684,5 +684,5 @@ from the events, or publish the events to other services.
|
|||
|
||||
@java[@extref[Multi-DC Persistence example project](samples:akka-samples-persistence-dc-java)]
|
||||
@scala[@extref[Multi-DC Persistence example project](samples:akka-samples-persistence-dc-scala)]
|
||||
illustrates how to use Lightbend's [Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
|
||||
with active-active persistent entities across data centers.
|
||||
illustrates how to use @ref:[Replicated Event Sourcing](replicated-eventsourcing.md) that supports
|
||||
active-active persistent entities across data centers.
|
||||
|
|
|
|||
|
|
@ -0,0 +1,121 @@
|
|||
# Auction example
|
||||
|
||||
In this example we want to show that real-world applications can be implemented by designing events in a way that they
|
||||
don't conflict. In the end, you will end up with a solution based on a custom CRDT.
|
||||
|
||||
We are building a small auction service. It has the following operations:
|
||||
|
||||
* Place a bid
|
||||
* Get the highest bid
|
||||
* Finish the auction
|
||||
|
||||
We model those operations as commands to be sent to the auction actor:
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #commands }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java) { #commands }
|
||||
|
||||
The events:
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #events }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java) { #events }
|
||||
|
||||
The winner does not have to pay the highest bid but only enough to beat the second highest, so the `highestCounterOffer` is in the `AuctionFinished` event.
|
||||
|
||||
Let's have a look at the auction entity that will handle incoming commands:
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #command-handler }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java) { #command-handler }
|
||||
|
||||
There is nothing specific to Replicated Event Sourcing about the command handler. It is the same as a command handler for a standard `EventSourcedBehavior`.
|
||||
For `OfferBid` and `AuctionFinished` we do nothing more than to emit
|
||||
events corresponding to the command. For `GetHighestBid` we respond with details from the state. Note, that we overwrite the actual
|
||||
offer of the highest bid here with the amount of the `highestCounterOffer`. This is done to follow the popular auction style where
|
||||
the actual highest bid is never publicly revealed.
|
||||
|
||||
The auction entity is started with the initial parameters for the auction.
|
||||
The minimum bid is modelled as an `initialBid`.
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #setup }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java) { #setup }
|
||||
|
||||
@@@ div { .group-scala }
|
||||
|
||||
The auction moves through the following phases:
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #phase }
|
||||
|
||||
@@@
|
||||
|
||||
The closing and closed states are to model waiting for all replicas to see the result of the auction before
|
||||
actually closing the action.
|
||||
|
||||
Let's have a look at our state class, `AuctionState` which also represents the CRDT in our example.
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #state }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java) { #state }
|
||||
|
||||
The state consists of a flag that keeps track of whether the auction is still active, the currently highest bid,
|
||||
and the highest counter offer so far.
|
||||
|
||||
In the `eventHandler`, we handle persisted events to drive the state change. When a new bid is registered,
|
||||
|
||||
* it needs to be decided whether the new bid is the winning bid or not
|
||||
* the state needs to be updated accordingly
|
||||
|
||||
The point of CRDTs is that the state must be end up being the same regardless of the order the events have been processed.
|
||||
We can see how this works in the auction example: we are only interested in the highest bid, so, if we can define an
|
||||
ordering on all bids, it should suffice to compare the new bid with currently highest to eventually end up with the globally
|
||||
highest regardless of the order in which the events come in.
|
||||
|
||||
The ordering between bids is crucial, therefore. We need to ensure that it is deterministic and does not depend on local state
|
||||
outside of our state class so that all replicas come to the same result. We define the ordering as this:
|
||||
|
||||
* A higher bid wins.
|
||||
* If there's a tie between the two highest bids, the bid that was registered earlier wins. For that we keep track of the
|
||||
(local) timestamp the bid was registered.
|
||||
* We need to make sure that no timestamp is used twice in the same replica (missing in this example).
|
||||
* If there's a tie between the timestamp, we define an arbitrary but deterministic ordering on the replicas, in our case
|
||||
we just compare the name strings of the replicas. That's why we need to keep the identifier of the replica where a bid was registered
|
||||
for every `Bid`.
|
||||
|
||||
If the new bid was higher, we keep this one as the new highest and keep the amount of the former highest as the `highestCounterOffer`.
|
||||
If the new bid was lower, we just update the `highestCounterOffer` if necessary.
|
||||
|
||||
Using those rules, the order of incoming does not matter. Replicas will eventually converge to the same result.
|
||||
|
||||
## Triggering closing
|
||||
|
||||
In the auction we want to ensure that all bids are seen before declaring a winner. That means that an auction can only be closed once
|
||||
all replicas have seen all bids.
|
||||
|
||||
In the event handler above, when recovery is not running, it calls `eventTriggers`.
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #event-triggers }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java) { #event-triggers }
|
||||
|
||||
The event trigger uses the `ReplicationContext` to decide when to trigger the Finish of the action.
|
||||
When a replica saves the `AuctionFinished` event it checks whether it should close the auction.
|
||||
For the close to happen the replica must be the one designated to close and all replicas must have
|
||||
reported that they have finished.
|
||||
|
||||
|
||||
|
||||
|
|
@ -0,0 +1,22 @@
|
|||
# Shopping cart example
|
||||
|
||||
The provided CRDT data structures can be used as the root state of a replicated `EventSourcedBehavior` but they can
|
||||
also be nested inside another data structure. This requires a bit more careful thinking about the eventual consistency.
|
||||
|
||||
In this sample we model a shopping cart as a map of product ids and the number of that product added or removed in the
|
||||
shopping cart. By using the @apidoc[Counter] CRDT and persisting its `Update` in our events we can be sure that an
|
||||
add or remove of items in any data center will eventually lead to all data centers ending up with the same number of
|
||||
each product.
|
||||
|
||||
Scala
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedShoppingCartExampleSpec.scala) { #shopping-cart }
|
||||
|
||||
Java
|
||||
: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedShoppingCartExample.java) { #shopping-cart }
|
||||
|
||||
With this model we cannot have a `ClearCart` command as that could give different states in different data centers.
|
||||
It is quite easy to imagine such a scenario: commands arriving in the order `ClearCart`, `AddItem('a', 5)` in one
|
||||
data center and the order `AddItem('a', 5), ClearCart` in another.
|
||||
|
||||
To clear a cart a client would instead have to remove as many items of each product as it sees in the cart at the time
|
||||
of removal.
|
||||
|
|
@ -0,0 +1,13 @@
|
|||
# Replicated Event Sourcing Examples
|
||||
|
||||
The following are more realistic examples of building systems with Replicated Event Sourcing.
|
||||
|
||||
@@toc { depth=1 }
|
||||
|
||||
@@@ index
|
||||
|
||||
* [auction](replicated-eventsourcing-auction.md)
|
||||
* [shopping cart](replicated-eventsourcing-cart.md)
|
||||
|
||||
@@@
|
||||
|
||||
416
akka-docs/src/main/paradox/typed/replicated-eventsourcing.md
Normal file
416
akka-docs/src/main/paradox/typed/replicated-eventsourcing.md
Normal file
|
|
@ -0,0 +1,416 @@
|
|||
# Replicated Event Sourcing
|
||||
|
||||
@@@ warning
|
||||
|
||||
This module is marked as @ref:[may change](../common/may-change.md) because it is a new feature that
|
||||
needs feedback from real usage before finalizing the API. This means that API or semantics can change without
|
||||
warning or deprecation period. It is also not recommended to use this module in production just yet.
|
||||
|
||||
@@@
|
||||
|
||||
@ref[Event sourcing](./persistence.md) with `EventSourcedBehavior`s is based on the single writer principle, which means that there can only be one active instance of a `EventSourcedBehavior`
|
||||
with a given `persistenceId`. Otherwise, multiple instances would store interleaving events based on different states, and when these events would later be replayed it would not be possible to reconstruct the correct state.
|
||||
|
||||
This restriction means that in the event of network partitions, and for a short time during rolling re-deploys, some
|
||||
`EventSourcedBehavior` actors are unavailable.
|
||||
|
||||
Replicated Event Sourcing enables running multiple replicas of each entity.
|
||||
There is automatic replication of every event persisted to all replicas.
|
||||
|
||||
For instance, a replica can be run per:
|
||||
|
||||
* Data Center
|
||||
* Availability zone or rack
|
||||
|
||||
The motivations are:
|
||||
|
||||
* Redundancy to tolerate failures in one location and still be operational
|
||||
* Serve requests from a location near the user to provide better responsiveness
|
||||
* Balance the load over many servers
|
||||
|
||||
However, the event handler must be able to **handle concurrent events** as when replication is enabled
|
||||
the single-writer guarantee is not maintained like it is with a normal `EventSourcedBehavior`.
|
||||
|
||||
The state of a replicated `EventSourcedBehavior` is **eventually consistent**. Event replication may be delayed
|
||||
due to network partitions and outages, which means that the event handler and those reading the state must be designed
|
||||
to handle this.
|
||||
|
||||
To be able to use Replicated Event Sourcing the journal and snapshot store used is required to have specific support for the metadata that the replication needs (see @ref[Journal Support](#journal-support)).
|
||||
|
||||
## Relaxing the single-writer principle for availability
|
||||
|
||||
Taking the example of using Replicated Event Sourcing to run a replica per data center.
|
||||
|
||||
When there is no network partitions and no concurrent writes the events stored by an `EventSourcedBehavior` at one replica can be replicated and consumed by another (corresponding) replica in another data center without any concerns. Such replicated events can simply be applied to the local state.
|
||||
|
||||

|
||||
|
||||
The interesting part begins when there are concurrent writes by `EventSourcedBehavior` replicas. That is more likely to happen when there is a network partition, but it can also happen when there are no network issues. They simply write at the "same time" before the events from the other side have been replicated and consumed.
|
||||
|
||||

|
||||
|
||||
The event handler logic for applying events to the state of the entity must be aware of that such concurrent updates can occur, and it must be modeled to handle such conflicts. This means that it should typically have the same characteristics as a Conflict Free Replicated Data Type (CRDT). With a CRDT there are by definition no conflicts, the events can always be applied. The library provides some general purpose CRDTs, but the logic of how to apply events can also be defined by an application specific function.
|
||||
|
||||
For example, sometimes it's enough to use application specific timestamps to decide which update should win.
|
||||
|
||||
To assist in implementing the event handler the Replicated Event Sourcing detects these conflicts.
|
||||
|
||||
## API
|
||||
|
||||
@scala[The same API as regular `EventSourcedBehavior`s]@java[A very similar API to the regular `EventSourcedBehavior`] is used to define the logic.
|
||||
|
||||
To enable an entity for Replicated Event Sourcing
|
||||
@java[let it extend `ReplicatedEventSourcedBehavior` instead of `EventSourcedBehavior` and] use the factory methods on @scala[`akka.persistence.typed.scaladsl.ReplicatedEventSourcing`]@java[`akka.persistence.typed.javadsl.ReplicatedEventSourcing`].
|
||||
|
||||
All replicas need to be known up front:
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #replicas }
|
||||
|
||||
Java
|
||||
: @@snip [MyReplicatedBehavior.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java) { #replicas }
|
||||
|
||||
|
||||
Then to enable replication create the event sourced behavior with the factory method:
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #factory }
|
||||
|
||||
Java
|
||||
: @@snip [MyReplicatedBehavior.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java) { #factory }
|
||||
|
||||
The factory takes in:
|
||||
|
||||
* `entityId`: this will be used as part of the underlying persistenceId
|
||||
* `replicaId`: Which replica this instance is
|
||||
* `allReplicasAndQueryPlugins`: All Replicas and the query plugin used to read their events
|
||||
* A factory function to create an instance of the @scala[`EventSourcedBehavior`]@java[`ReplicatedEventSourcedBehavior`]
|
||||
|
||||
In this scenario each replica reads from each other's database effectively providing cross region replication for any database that has an Akka Persistence plugin. Alternatively if all the replicas use the same journal, e.g. for testing or if it is a distributed database such as Cassandra, the `withSharedJournal` factory can be used.
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #factory-shared}
|
||||
|
||||
Java
|
||||
: @@snip [MyReplicatedBehavior.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java) { #factory-shared }
|
||||
|
||||
|
||||
@@@ div { .group-scala }
|
||||
|
||||
The function passed to both factory methods return an `EventSourcedBehavior` and provide access to the @apidoc[ReplicationContext] that has the following methods:
|
||||
|
||||
* `entityId`
|
||||
* `replicaId`
|
||||
* `allReplicas`
|
||||
* `persistenceId` - to provide to the `EventSourcedBehavior` factory. This **must be used**.
|
||||
|
||||
As well as methods that **can only be** used in the event handler. The values these methods return relate to the event that is being processed.
|
||||
|
||||
@@@
|
||||
|
||||
@@@ div { .group-java }
|
||||
|
||||
The function passed to both factory methods is invoked with a special @apidoc[ReplicationContext] that needs to be passed to the
|
||||
concrete `ReplicatedEventSourcedBehavior` and on to the super constructor.
|
||||
|
||||
The context gives access to:
|
||||
|
||||
* `entityId`
|
||||
* `replicaId`
|
||||
* `allReplicas`
|
||||
* `persistenceId`
|
||||
|
||||
As well as methods that **can only be** used in the event handler, accessed through `getReplicationContext`. The values these methods return relate to the event that is being processed.
|
||||
|
||||
@@@
|
||||
|
||||
* `origin`: The ReplicaId that originally created the event
|
||||
* `concurrent`: Whether the event was concurrent with another event as in the second diagram above
|
||||
* `recoveryRunning`: Whether a recovery is running. Can be used to send commands back to self for side effects that should only happen once.
|
||||
* `currentTimeMillis`: similar to `System.currentTimeMillis` but guaranteed never to go backwards
|
||||
|
||||
The factory returns a `Behavior` that can be spawned like any other behavior.
|
||||
|
||||
## Resolving conflicting updates
|
||||
|
||||
### Conflict free replicated data types
|
||||
|
||||
Writing code to resolve conflicts can be complicated to get right.
|
||||
One well-understood technique to create eventually-consistent systems is to
|
||||
model your state as a Conflict Free Replicated Data Type, a CRDT. There are two types of CRDTs;
|
||||
operation-based and state-based. For Replicated Event Sourcing the operation-based is a good fit,
|
||||
since the events represent the operations. Note that this is distinct from the CRDT's implemented
|
||||
in @ref:[Akka Distributed Data](distributed-data.md), which are state-based rather than operation-based.
|
||||
|
||||
The rule for operation-based CRDT's is that the operations must be commutative — in other words, applying the same events
|
||||
(which represent the operations) in any order should always produce the same final state. You may assume each event
|
||||
is applied only once, with @ref:[causal delivery order](#causal-delivery-order).
|
||||
|
||||
The following CRDTs are included that can you can use as the state or part of the state in the entity:
|
||||
|
||||
* @apidoc[LwwTime]
|
||||
* @apidoc[Counter]
|
||||
* @apidoc[akka.persistence.typed.crdt.ORSet]
|
||||
|
||||
Akka serializers are included for all these types and can be used to serialize when
|
||||
@ref[embedded in Jackson](../serialization-jackson.md#using-akka-serialization-for-embedded-types).
|
||||
|
||||
An example would be a movies watch list that is represented by the general purpose
|
||||
@apidoc[akka.persistence.typed.crdt.ORSet] CRDT. `ORSet` is short for Observed Remove Set. Elements can be added and
|
||||
removed any number of times. Concurrent add wins over remove. It is an operation based CRDT where the delta of an
|
||||
operation (add/remove) can be represented as an event.
|
||||
|
||||
Such movies watch list example:
|
||||
|
||||
Scala
|
||||
: @@snip [movie](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedMovieWatchListExampleSpec.scala) { #movie-entity }
|
||||
|
||||
Java
|
||||
: @@snip [movie](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedMovieExample.java) { #movie-entity }
|
||||
|
||||
The @ref[Auction example](replicated-eventsourcing-auction.md) is a more comprehensive example that illustrates how application-specific
|
||||
rules can be used to implement an entity with CRDT semantics.
|
||||
|
||||
### Last writer wins
|
||||
|
||||
Sometimes it is enough to use timestamps to decide which update should win. Such approach relies on synchronized clocks, and clocks of different machines will always be slightly out of sync. Timestamps should therefore only be used when the choice of value is not important for concurrent updates occurring within the clock skew.
|
||||
|
||||
In general, last writer wins means that the event is used if the timestamp of the event is later (higher) than the timestamp of previous local update, otherwise the event is discarded. There is no built-in support for last writer wins, because it must often be combined with more application specific aspects.
|
||||
|
||||

|
||||
|
||||
There is a small utility class @apidoc[LwwTime] that can be useful for implementing last writer wins semantics.
|
||||
It contains a timestamp representing current time when the event was persisted and an identifier of the
|
||||
replica that persisted it. When comparing two @apidoc[LwwTime] the greatest timestamp wins. The replica
|
||||
identifier is used if the two timestamps are equal, and then the one from the `replicaId` sorted first in
|
||||
alphanumeric order wins.
|
||||
|
||||
Scala
|
||||
: @@snip [blog](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala) { #event-handler }
|
||||
|
||||
Java
|
||||
: @@snip [blog](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java) { #event-handler }
|
||||
|
||||
When creating the `LwwTime` it is good to have a monotonically increasing timestamp, and for that the `increase`
|
||||
method in `LwwTime` can be used:
|
||||
|
||||
Scala
|
||||
: @@snip [blog](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala) { #command-handler }
|
||||
|
||||
Java
|
||||
: @@snip [blog](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java) { #command-handler }
|
||||
|
||||
The nature of last writer wins means that if you only have one timestamp for the state the events must represent an
|
||||
update of the full state. Otherwise, there is a risk that the state in different replicas will be different and
|
||||
not eventually converge.
|
||||
|
||||
An example of that would be an entity representing a blog post and the fields `author` and `title` could be updated
|
||||
separately with events @scala[`AuthorChanged(newAuthor: String)`]@java[`new AuthorChanged(newAuthor)`] and @scala[`TitleChanged(newTitle: String)`]@java[`new TitleChanged(newTitle)`].
|
||||
|
||||
Let's say the blog post is created and the initial state of `title=Akka, author=unknown` is in sync in both replicas `DC-A` and `DC-B.
|
||||
|
||||
In `DC-A` author is changed to "Bob" at time `100`. Before that event has been replicated over to `DC-B` the
|
||||
title is updated to "Akka News" at time `101` in `DC-B`. When the events have been replicated the result will be:
|
||||
|
||||
`DC-A`: The title update is later so the event is used and new state is `title=Akka News, author=Bob`
|
||||
|
||||
`DC-B`: The author update is earlier so the event is discarded and state is `title=Akka News, author=unknown`
|
||||
|
||||
The problem here is that the partial update of the state is not applied on both sides, so the states have diverged and will not become the same.
|
||||
|
||||
To solve this with last writer wins the events must carry the full state, such as @scala[`AuthorChanged(newContent: PostContent)`]@java[`new AuthorChanged(newContent)`] and @scala[`TitleChanged(newContent: PostContent)`]@java[`new TitleChanged(newContent)`]. Then the result would eventually be `title=Akka News, author=unknown` on both sides.
|
||||
The author update is lost but that is because the changes were performed concurrently. More important is that the state
|
||||
is eventually consistent.
|
||||
|
||||
Including the full state in each event is often not desired. An event typically represent a change, a delta. Then one can use several timestamps, one for each set of fields that can be updated together. In the above example one could use one timestamp for the title and another for the author. Then the events could represent changes to parts of the full state, such as @scala[`AuthorChanged(newAuthor: String)`]@java[`new AuthorChanged(newAuthor)`] and @scala[`TitleChanged(newTitle: String)`]@java[`new TitleChanged(newTitle)`].
|
||||
|
||||
## Side effects
|
||||
|
||||
In most cases it is recommended to do side effects as @ref[described for `EventSourcedBehavior`s](./persistence.md#effects-and-side-effects).
|
||||
|
||||
Side effects from the event handler are generally discouraged because the event handlers are also used during replay and when consuming replicated events and that would
|
||||
result in undesired re-execution of the side effects.
|
||||
|
||||
Uses cases for doing side effects in the event handler:
|
||||
|
||||
* Doing a side effect only in a single replica
|
||||
* Doing a side effect once all replicas have seen an event
|
||||
* A side effect for a replicated event
|
||||
* A side effect when a conflict has occurred
|
||||
|
||||
There is no built in support for knowing an event has been replicated to all replicas but it can be modelled in your state.
|
||||
For some use cases you may need to trigger side effects after consuming replicated events. For example when an auction has been closed in
|
||||
all data centers and all bids have been replicated.
|
||||
|
||||
The @api[ReplicationContext] contains the current replica, the origin replica for the event processes, and if a recovery is running. These can be used to
|
||||
implement side effects that take place once events are fully replicated. If the side effect should happen only once then a particular replica can be
|
||||
designated to do it. The @ref[Auction example](replicated-eventsourcing-auction.md) uses these techniques.
|
||||
|
||||
|
||||
## How it works
|
||||
|
||||
You don’t have to read this section to be able to use the feature, but to use the abstraction efficiently and for the right type of use cases it can be good to understand how it’s implemented. For example, it should give you the right expectations of the overhead that the solution introduces compared to using just `EventSourcedBehavior`s.
|
||||
|
||||
### Causal delivery order
|
||||
|
||||
Causal delivery order means that events persisted in one replica are read in the same order in other replicas. The order of concurrent events is undefined, which should be no problem
|
||||
when using @ref:[CRDT's](#conflict-free-replicated-data-types)
|
||||
and otherwise will be detected via the `ReplicationContext` concurrent method.
|
||||
|
||||
For example:
|
||||
|
||||
```
|
||||
DC-1: write e1
|
||||
DC-2: read e1, write e2
|
||||
DC-1: read e2, write e3
|
||||
```
|
||||
|
||||
In the above example the causality is `e1 -> e2 -> e3`. Also in a third replica DC-3 these events will be read in the same order e1, e2, e3.
|
||||
|
||||
Another example with concurrent events:
|
||||
|
||||
```
|
||||
DC1: write e1
|
||||
DC2: read e1, write e2
|
||||
DC1: write e3 (e2 and e3 are concurrent)
|
||||
DC1: read e2
|
||||
DC2: read e3
|
||||
```
|
||||
|
||||
e2 and e3 are concurrent, i.e. they don't have a causal relation: DC1 sees them in the order "e1, e3, e2", while DC2 sees them as "e1, e2, e3".
|
||||
|
||||
A third replica may also see the events as either "e1, e3, e2" or as "e1, e2, e3".
|
||||
|
||||
### Concurrent updates
|
||||
|
||||
Replicated Event Sourcing automatically tracks causality between events from different replicas using [version vectors](https://en.wikipedia.org/wiki/Version_vector).
|
||||
|
||||

|
||||
|
||||
Each replica "owns" a slot in the version vector and increases its counter when an event is persisted. The version vector is stored with the event, and when a replicated event is consumed the version vector of the event is merged with the local version vector.
|
||||
|
||||
When comparing two version vectors `v1` and `v2`:
|
||||
|
||||
* `v1` is SAME as `v2` iff for all i `v1(i) == v2(i)`
|
||||
* `v1`is BEFORE `v2` iff for all i `v1(i) <= v2(i)` and there exist a j such that `v1(j) < v2(j)`
|
||||
* `v1`is AFTER `v2` iff for all i `v1(i) >= v2(i)` and there exist a j such that `v1(j) > v2(j)`
|
||||
* `v1`is CONCURRENT with `v2` otherwise
|
||||
|
||||
|
||||
## Sharded Replicated Event Sourced entities
|
||||
|
||||
There are three ways to integrate replicated event sourced entities with sharding:
|
||||
|
||||
* Ensure that each replica has a unique entity id by using the replica id as part of the entity id
|
||||
* Use @ref[multi datacenter](cluster-dc.md) to run a full copy of sharding per replica
|
||||
* Use roles to run a full copy of sharding per replica
|
||||
|
||||
|
||||
To simplify all three cases the @apidoc[ReplicatedShardingExtension] is available from the
|
||||
`akka-cluster-sharding-typed` module.
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala) { #bootstrap }
|
||||
|
||||
Java
|
||||
: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java) { #bootstrap }
|
||||
|
||||
This will run an instance of sharding and per replica and each entity id contains the replica id and the type name.
|
||||
Replicas could be on the same node if they end up in the same shard or if the shards get allocated to the same node.
|
||||
|
||||
To prevent this roles can be used. You could for instance add a cluster role per availability zone / rack and have a replica per rack.
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala) { #bootstrap-role }
|
||||
|
||||
Java
|
||||
: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java) { #bootstrap-role }
|
||||
|
||||
Lastly if your Akka Cluster is setup across DCs you can run a replica per DC.
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala) { #bootstrap-dc }
|
||||
|
||||
Java
|
||||
: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java) { #bootstrap-dc }
|
||||
|
||||
Regardless of which replication strategy you use sending messages to the replicated entities is the same.
|
||||
|
||||
`init` returns an @apidoc[ReplicatedSharding] instance which gives access to @apidoc[EntityRef]s for each of the replicas for arbitrary routing logic:
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala) { #sending-messages }
|
||||
|
||||
Java
|
||||
: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java) { #sending-messages }
|
||||
|
||||
More advanced routing among the replicas is currently left as an exercise for the reader (or may be covered in a future release [#29281](https://github.com/akka/akka/issues/29281), [#29319](https://github.com/akka/akka/issues/29319)).
|
||||
|
||||
## Tagging events and running projections
|
||||
|
||||
Just like for regular `EventSourcedBehavior`s it is possible to tag events along with persisting them.
|
||||
This is useful for later retrival of events for a given tag. The same @ref[API for tagging provided for EventSourcedBehavior](persistence.md#tagging) can
|
||||
be used for replicated event sourced behaviors as well.
|
||||
Tagging is useful in practice to build queries that lead to other data representations or aggregations of these event
|
||||
streams that can more directly serve user queries – known as building the “read side” in @ref[CQRS](cqrs.md) based applications.
|
||||
|
||||
Creating read side projections is possible through [Akka Projection](https://doc.akka.io/docs/akka-projection/current/)
|
||||
or through direct usage of the @ref[events by tag queries](../persistence-query.md#eventsbytag-and-currenteventsbytag).
|
||||
|
||||
The tagging is invoked in each replicas, which requires some special care in using tags, or else the same event will be
|
||||
tagged one time for each replica and show up in the event by tag stream one time for each replica. In addition to this
|
||||
the tags will be written in the respective journal of the replicas, which means that unless they all share a single journal
|
||||
the tag streams will be local to the replica even if the same tag is used on multiple replicas.
|
||||
|
||||
One strategy for dealing with this is to include the replica id in the tag name, this means there will be a tagged stream of events
|
||||
per replica that contains all replicated events, but since the events can arrive in different order, they can also come in different
|
||||
order per replica tag.
|
||||
|
||||
Another strategy would be to tag only the events that are local to the replica and not events that are replicated. Either
|
||||
using a tag that will be the same for all replicas, leading to a single stream of tagged events where the events from each
|
||||
replica is present only once, or with a tag including the replica id meaning that there will be a stream of tagged events
|
||||
with the events accepted locally for each replica.
|
||||
|
||||
Determining the replica id of the replicated actor itself and the origin replica id of an event is possible through the
|
||||
@apidoc[ReplicationContext] when the tagger callback is invoked like this:
|
||||
|
||||
Scala
|
||||
: @@snip [ReplicatedEventSourcingTaggingSpec.scala](/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingTaggingSpec.scala) { #tagging }
|
||||
|
||||
Java
|
||||
: @@snip [ReplicatedStringSet.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedStringSet.java) { #tagging }
|
||||
|
||||
In this sample we are using a shared journal, and single tag but only tagging local events to it and therefore ending up
|
||||
with a single stream of tagged events from all replicas without duplicates.
|
||||
|
||||
## Direct Replication of Events
|
||||
|
||||
In addition to reading each replica's events from the database the replicated events are published across the Akka cluster to the replicas when used with Cluster Sharding.
|
||||
The query is still needed as delivery is not guaranteed, but can be configured to poll the database less often since most
|
||||
events will arrive at the replicas through the cluster.
|
||||
|
||||
This feature is enabled by default when using sharding.
|
||||
To disable this feature you first need to disable event publishing on the @scala[`EventSourcedBehavior`]@java[`ReplicatedEventSourcedBehavior`] with `withEventPublishing`
|
||||
and then disable direct replication through `withDirectReplication(true)` on @apidoc[ReplicatedEntityProvider]
|
||||
|
||||
The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written,
|
||||
|
||||
## Hot Standby
|
||||
|
||||
If all writes occur to one replica and the other replicas are not started there might be many replicated events to catch up with when they are later started. Therefore it can be good to activate all replicas when there is some activity.
|
||||
|
||||
This can be achieved automatically when `ReplicatedSharding` is used and direct replication of events is enabled as described in @ref[Direct Replication of Events](#direct-replication-of-events). When each written event is forwarded to the other replicas it will trigger them to start if they are not already started.
|
||||
|
||||
## Examples
|
||||
|
||||
More examples can be found in @ref[Replicated Event Sourcing Examples](./replicated-eventsourcing-examples.md)
|
||||
|
||||
## Journal Support
|
||||
|
||||
For a journal plugin to support replication it needs to store and read metadata for each event if it is defined in the @apiref[PersistentRepr]
|
||||
`metadata` field. To attach the metadata after writing it, `PersistentRepr.withMetadata` is used. The @apidoc[JournalSpec] in the Persistence TCK provides
|
||||
a capability flag `supportsMetadata` to toggle verification that metadata is handled correctly.
|
||||
|
||||
For a snapshot plugin to support replication it needs to store and read metadata for the snapshot if it is defined in the @apiref[akka.persistence.SnapshotMetadata] `metadata` field.
|
||||
To attach the metadata when reading the snapshot the `akka.persistence.SnapshotMetadata.apply` factory overload taking a `metadata` parameter is used.
|
||||
The @apidoc[SnapshotStoreSpec] in the Persistence TCK provides a capability flag `supportsMetadata` to toggle verification that metadata is handled correctly.
|
||||
|
|
@ -4,14 +4,26 @@
|
|||
|
||||
package akka.persistence.query
|
||||
|
||||
import scala.runtime.AbstractFunction4
|
||||
import java.util.Optional
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
import scala.runtime.AbstractFunction4
|
||||
import akka.util.HashCode
|
||||
|
||||
// for binary compatibility (used to be a case class)
|
||||
object EventEnvelope extends AbstractFunction4[Offset, String, Long, Any, EventEnvelope] {
|
||||
def apply(offset: Offset, persistenceId: String, sequenceNr: Long, event: Any, timestamp: Long): EventEnvelope =
|
||||
new EventEnvelope(offset, persistenceId, sequenceNr, event, timestamp)
|
||||
new EventEnvelope(offset, persistenceId, sequenceNr, event, timestamp, None)
|
||||
|
||||
def apply(
|
||||
offset: Offset,
|
||||
persistenceId: String,
|
||||
sequenceNr: Long,
|
||||
event: Any,
|
||||
timestamp: Long,
|
||||
meta: Option[Any]): EventEnvelope =
|
||||
new EventEnvelope(offset, persistenceId, sequenceNr, event, timestamp, meta)
|
||||
|
||||
@deprecated("for binary compatibility", "2.6.2")
|
||||
override def apply(offset: Offset, persistenceId: String, sequenceNr: Long, event: Any): EventEnvelope =
|
||||
|
|
@ -34,13 +46,26 @@ final class EventEnvelope(
|
|||
val persistenceId: String,
|
||||
val sequenceNr: Long,
|
||||
val event: Any,
|
||||
val timestamp: Long)
|
||||
val timestamp: Long,
|
||||
val eventMetadata: Option[Any])
|
||||
extends Product4[Offset, String, Long, Any]
|
||||
with Serializable {
|
||||
|
||||
@deprecated("for binary compatibility", "2.6.2")
|
||||
def this(offset: Offset, persistenceId: String, sequenceNr: Long, event: Any) =
|
||||
this(offset, persistenceId, sequenceNr, event, 0L)
|
||||
this(offset, persistenceId, sequenceNr, event, 0L, None)
|
||||
|
||||
// bin compat 2.6.7
|
||||
def this(offset: Offset, persistenceId: String, sequenceNr: Long, event: Any, timestamp: Long) =
|
||||
this(offset, persistenceId, sequenceNr, event, timestamp, None)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getEventMetaData(): Optional[Any] = {
|
||||
import scala.compat.java8.OptionConverters._
|
||||
eventMetadata.asJava
|
||||
}
|
||||
|
||||
override def hashCode(): Int = {
|
||||
var result = HashCode.SEED
|
||||
|
|
@ -54,12 +79,12 @@ final class EventEnvelope(
|
|||
override def equals(obj: Any): Boolean = obj match {
|
||||
case other: EventEnvelope =>
|
||||
offset == other.offset && persistenceId == other.persistenceId && sequenceNr == other.sequenceNr &&
|
||||
event == other.event // timestamp not included in equals for backwards compatibility
|
||||
event == other.event // timestamp && metadata not included in equals for backwards compatibility
|
||||
case _ => false
|
||||
}
|
||||
|
||||
override def toString: String =
|
||||
s"EventEnvelope($offset,$persistenceId,$sequenceNr,$event,$timestamp)"
|
||||
s"EventEnvelope($offset,$persistenceId,$sequenceNr,$event,$timestamp,$eventMetadata)"
|
||||
|
||||
// for binary compatibility (used to be a case class)
|
||||
def copy(
|
||||
|
|
@ -67,7 +92,11 @@ final class EventEnvelope(
|
|||
persistenceId: String = this.persistenceId,
|
||||
sequenceNr: Long = this.sequenceNr,
|
||||
event: Any = this.event): EventEnvelope =
|
||||
new EventEnvelope(offset, persistenceId, sequenceNr, event, timestamp)
|
||||
new EventEnvelope(offset, persistenceId, sequenceNr, event, timestamp, this.eventMetadata)
|
||||
|
||||
@InternalApi
|
||||
private[akka] def withMetadata(metadata: Any): EventEnvelope =
|
||||
new EventEnvelope(offset, persistenceId, sequenceNr, event, timestamp, Some(metadata))
|
||||
|
||||
// Product4, for binary compatibility (used to be a case class)
|
||||
override def productPrefix = "EventEnvelope"
|
||||
|
|
|
|||
|
|
@ -5,7 +5,6 @@
|
|||
package akka.persistence.query.journal.leveldb
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.JournalProtocol.RecoverySuccess
|
||||
|
|
|
|||
|
|
@ -5,7 +5,6 @@
|
|||
package akka.persistence.query.journal.leveldb
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.JournalProtocol.RecoverySuccess
|
||||
|
|
|
|||
|
|
@ -29,7 +29,7 @@ class TestActor(override val persistenceId: String) extends PersistentActor {
|
|||
|
||||
case cmd: String =>
|
||||
persist(cmd) { evt =>
|
||||
sender() ! evt + "-done"
|
||||
sender() ! s"$evt-done"
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -52,6 +52,12 @@ trait JournalCapabilityFlags extends CapabilityFlags {
|
|||
*/
|
||||
protected def supportsSerialization: CapabilityFlag
|
||||
|
||||
/**
|
||||
* When `true` enables tests which check if the Journal stores and returns
|
||||
* metadata for an event
|
||||
*/
|
||||
protected def supportsMetadata: CapabilityFlag
|
||||
|
||||
}
|
||||
//#journal-flags
|
||||
|
||||
|
|
@ -63,5 +69,11 @@ trait SnapshotStoreCapabilityFlags extends CapabilityFlags {
|
|||
* deserialize snapshots.
|
||||
*/
|
||||
protected def supportsSerialization: CapabilityFlag
|
||||
|
||||
/**
|
||||
* When `true` enables tests which check if the snapshot store properly stores and
|
||||
* loads metadata (needed for replication) along with the snapshots
|
||||
*/
|
||||
protected def supportsMetadata: CapabilityFlag
|
||||
}
|
||||
//#snapshot-store-flags
|
||||
|
|
|
|||
|
|
@ -54,6 +54,8 @@ abstract class JournalSpec(config: Config)
|
|||
|
||||
override protected def supportsSerialization: CapabilityFlag = true
|
||||
|
||||
override protected def supportsMetadata: CapabilityFlag = false
|
||||
|
||||
override protected def beforeEach(): Unit = {
|
||||
super.beforeEach()
|
||||
senderProbe = TestProbe()
|
||||
|
|
@ -79,7 +81,7 @@ abstract class JournalSpec(config: Config)
|
|||
extension.journalFor(null)
|
||||
|
||||
def replayedMessage(snr: Long, deleted: Boolean = false): ReplayedMessage =
|
||||
ReplayedMessage(PersistentImpl(s"a-${snr}", snr, pid, "", deleted, Actor.noSender, writerUuid, 0L))
|
||||
ReplayedMessage(PersistentImpl(s"a-${snr}", snr, pid, "", deleted, Actor.noSender, writerUuid, 0L, None))
|
||||
|
||||
def writeMessages(fromSnr: Int, toSnr: Int, pid: String, sender: ActorRef, writerUuid: String): Unit = {
|
||||
|
||||
|
|
@ -112,7 +114,7 @@ abstract class JournalSpec(config: Config)
|
|||
probe.expectMsg(WriteMessagesSuccessful)
|
||||
(fromSnr to toSnr).foreach { i =>
|
||||
probe.expectMsgPF() {
|
||||
case WriteMessageSuccess(PersistentImpl(payload, `i`, `pid`, _, _, `sender`, `writerUuid`, _), _) =>
|
||||
case WriteMessageSuccess(PersistentImpl(payload, `i`, `pid`, _, _, `sender`, `writerUuid`, _, _), _) =>
|
||||
payload should be(s"a-${i}")
|
||||
}
|
||||
}
|
||||
|
|
@ -263,15 +265,15 @@ abstract class JournalSpec(config: Config)
|
|||
val Pid = pid
|
||||
val WriterUuid = writerUuid
|
||||
probe.expectMsgPF() {
|
||||
case WriteMessageSuccess(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _), _) =>
|
||||
case WriteMessageSuccess(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _, _), _) =>
|
||||
payload should be(s"b-6")
|
||||
}
|
||||
probe.expectMsgPF() {
|
||||
case WriteMessageRejected(PersistentImpl(payload, 7L, Pid, _, _, Actor.noSender, WriterUuid, _), _, _) =>
|
||||
case WriteMessageRejected(PersistentImpl(payload, 7L, Pid, _, _, Actor.noSender, WriterUuid, _, _), _, _) =>
|
||||
payload should be(notSerializableEvent)
|
||||
}
|
||||
probe.expectMsgPF() {
|
||||
case WriteMessageSuccess(PersistentImpl(payload, 8L, Pid, _, _, Actor.noSender, WriterUuid, _), _) =>
|
||||
case WriteMessageSuccess(PersistentImpl(payload, 8L, Pid, _, _, Actor.noSender, WriterUuid, _, _), _) =>
|
||||
payload should be(s"b-8")
|
||||
}
|
||||
}
|
||||
|
|
@ -296,13 +298,13 @@ abstract class JournalSpec(config: Config)
|
|||
val Pid = pid
|
||||
val WriterUuid = writerUuid
|
||||
probe.expectMsgPF() {
|
||||
case WriteMessageSuccess(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _), _) =>
|
||||
case WriteMessageSuccess(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _, _), _) =>
|
||||
payload should be(event)
|
||||
}
|
||||
|
||||
journal ! ReplayMessages(6, Long.MaxValue, Long.MaxValue, pid, receiverProbe.ref)
|
||||
receiverProbe.expectMsgPF() {
|
||||
case ReplayedMessage(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _)) =>
|
||||
case ReplayedMessage(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _, _)) =>
|
||||
payload should be(event)
|
||||
}
|
||||
receiverProbe.expectMsgPF() {
|
||||
|
|
@ -310,5 +312,42 @@ abstract class JournalSpec(config: Config)
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
optional(flag = supportsMetadata) {
|
||||
|
||||
"return metadata" in {
|
||||
val probe = TestProbe()
|
||||
val event = TestPayload(probe.ref)
|
||||
val meta = "meta-data"
|
||||
val aw =
|
||||
AtomicWrite(
|
||||
PersistentRepr(
|
||||
payload = event,
|
||||
sequenceNr = 6L,
|
||||
persistenceId = pid,
|
||||
sender = Actor.noSender,
|
||||
writerUuid = writerUuid).withMetadata(meta))
|
||||
|
||||
journal ! WriteMessages(List(aw), probe.ref, actorInstanceId)
|
||||
probe.expectMsg(WriteMessagesSuccessful)
|
||||
|
||||
val Pid = pid
|
||||
val WriterUuid = writerUuid
|
||||
probe.expectMsgPF() {
|
||||
case WriteMessageSuccess(
|
||||
PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _, Some(`meta`)),
|
||||
_) =>
|
||||
payload should be(event)
|
||||
}
|
||||
|
||||
journal ! ReplayMessages(6, 6, 1, Pid, receiverProbe.ref)
|
||||
receiverProbe.expectMsgPF() {
|
||||
case ReplayedMessage(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _, Some(`meta`))) =>
|
||||
payload should be(event)
|
||||
}
|
||||
receiverProbe.expectMsg(RecoverySuccess(6L))
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -52,6 +52,7 @@ abstract class SnapshotStoreSpec(config: Config)
|
|||
private var metadata: Seq[SnapshotMetadata] = Nil
|
||||
|
||||
override protected def supportsSerialization: CapabilityFlag = true
|
||||
override protected def supportsMetadata: CapabilityFlag = false
|
||||
|
||||
override protected def beforeEach(): Unit = {
|
||||
super.beforeEach()
|
||||
|
|
@ -199,5 +200,27 @@ abstract class SnapshotStoreSpec(config: Config)
|
|||
}
|
||||
}
|
||||
}
|
||||
optional(flag = supportsMetadata) {
|
||||
"store metadata" in {
|
||||
// we do not have the actual ReplicatedSnapshot metadata on classpath, but since
|
||||
// the plugin should defer to serialization defined by Akka, so in general the type
|
||||
// should not really be important to the plugin
|
||||
val fictionalMeta = "fictional metadata"
|
||||
val metadata = SnapshotMetadata(pid, 100).withMetadata(fictionalMeta)
|
||||
val snap = "snap"
|
||||
snapshotStore.tell(SaveSnapshot(metadata, snap), senderProbe.ref)
|
||||
senderProbe.expectMsgPF() { case SaveSnapshotSuccess(md) => md }
|
||||
|
||||
val Pid = pid
|
||||
snapshotStore.tell(LoadSnapshot(pid, SnapshotSelectionCriteria.Latest, Long.MaxValue), senderProbe.ref)
|
||||
senderProbe.expectMsgPF() {
|
||||
case LoadSnapshotResult(
|
||||
Some(SelectedSnapshot(meta @ SnapshotMetadata(Pid, 100, _), payload)),
|
||||
Long.MaxValue) =>
|
||||
payload should be(snap)
|
||||
meta.metadata should ===(Some(fictionalMeta))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -28,3 +28,7 @@ akka.persistence.testkit {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
akka.persistence.testkit.query {
|
||||
class = "akka.persistence.testkit.query.PersistenceTestKitReadJournalProvider"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,9 +8,9 @@ import java.util.{ List => JList }
|
|||
|
||||
import scala.collection.immutable
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.PersistentRepr
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies
|
||||
import akka.persistence.testkit.internal.TestKitStorage
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
|
|
@ -31,10 +31,12 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, Per
|
|||
// and therefore must be done at the same time with the update, not before
|
||||
updateOrSetNew(key, v => v ++ mapAny(key, elems).toVector)
|
||||
|
||||
override def reprToSeqNum(repr: PersistentRepr): Long = repr.sequenceNr
|
||||
override def reprToSeqNum(repr: (PersistentRepr)): Long = repr.sequenceNr
|
||||
|
||||
def add(elems: immutable.Seq[PersistentRepr]): Unit =
|
||||
elems.groupBy(_.persistenceId).foreach(gr => add(gr._1, gr._2))
|
||||
elems.groupBy(_.persistenceId).foreach { gr =>
|
||||
add(gr._1, gr._2)
|
||||
}
|
||||
|
||||
override protected val DefaultPolicy = JournalPolicies.PassAll
|
||||
|
||||
|
|
@ -45,7 +47,11 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, Per
|
|||
val grouped = elems.groupBy(_.persistenceId)
|
||||
|
||||
val processed = grouped.map {
|
||||
case (pid, els) => currentPolicy.tryProcess(pid, WriteEvents(els.map(_.payload)))
|
||||
case (pid, els) =>
|
||||
currentPolicy.tryProcess(pid, WriteEvents(els.map(_.payload match {
|
||||
case Tagged(payload, _) => payload
|
||||
case nonTagged => nonTagged
|
||||
})))
|
||||
}
|
||||
|
||||
val reduced: ProcessingResult =
|
||||
|
|
@ -80,6 +86,23 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, Per
|
|||
}
|
||||
}
|
||||
|
||||
def tryReadByTag(tag: String): immutable.Seq[PersistentRepr] = {
|
||||
val batch = readAll()
|
||||
.filter(repr =>
|
||||
repr.payload match {
|
||||
case Tagged(_, tags) => tags.contains(tag)
|
||||
case _ => false
|
||||
})
|
||||
.toVector
|
||||
.sortBy(_.timestamp)
|
||||
|
||||
currentPolicy.tryProcess(tag, ReadEvents(batch)) match {
|
||||
case ProcessingSuccess => batch
|
||||
case Reject(ex) => throw ex
|
||||
case StorageFailure(ex) => throw ex
|
||||
}
|
||||
}
|
||||
|
||||
def tryReadSeqNumber(persistenceId: String): Long = {
|
||||
currentPolicy.tryProcess(persistenceId, ReadSeqNum) match {
|
||||
case ProcessingSuccess => getHighestSeqNumber(persistenceId)
|
||||
|
|
@ -104,9 +127,7 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, Per
|
|||
}
|
||||
|
||||
object EventStorage {
|
||||
|
||||
object JournalPolicies extends DefaultPolicies[JournalOperation]
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -4,17 +4,19 @@
|
|||
|
||||
package akka.persistence.testkit
|
||||
|
||||
import akka.actor.ActorLogging
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Future
|
||||
import scala.util.Try
|
||||
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence._
|
||||
import akka.persistence.journal.{ AsyncWriteJournal, Tagged }
|
||||
import akka.persistence.journal.AsyncWriteJournal
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.snapshot.SnapshotStore
|
||||
import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorageEmulatorExtension }
|
||||
import akka.util.unused
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -22,26 +24,49 @@ import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorag
|
|||
* Persistence testkit plugin for events.
|
||||
*/
|
||||
@InternalApi
|
||||
class PersistenceTestKitPlugin extends AsyncWriteJournal {
|
||||
class PersistenceTestKitPlugin(@unused cfg: Config, cfgPath: String) extends AsyncWriteJournal with ActorLogging {
|
||||
|
||||
private final val storage = InMemStorageExtension(context.system)
|
||||
private final val storage = {
|
||||
log.debug("Using in memory storage [{}] for test kit journal", cfgPath)
|
||||
InMemStorageExtension(context.system).storageFor(cfgPath)
|
||||
}
|
||||
private val eventStream = context.system.eventStream
|
||||
|
||||
override def asyncWriteMessages(messages: immutable.Seq[AtomicWrite]): Future[immutable.Seq[Try[Unit]]] =
|
||||
override def asyncWriteMessages(messages: immutable.Seq[AtomicWrite]): Future[immutable.Seq[Try[Unit]]] = {
|
||||
Future.fromTry(Try(messages.map(aw => {
|
||||
val data = aw.payload.map(pl =>
|
||||
pl.payload match {
|
||||
case Tagged(p, _) => pl.withPayload(p)
|
||||
case _ => pl
|
||||
case _ => pl.withTimestamp(System.currentTimeMillis())
|
||||
})
|
||||
storage.tryAdd(data)
|
||||
|
||||
val result: Try[Unit] = storage.tryAdd(data)
|
||||
result.foreach { _ =>
|
||||
messages.foreach { aw =>
|
||||
eventStream.publish(PersistenceTestKitPlugin.Write(aw.persistenceId, aw.highestSequenceNr))
|
||||
}
|
||||
}
|
||||
result
|
||||
})))
|
||||
}
|
||||
|
||||
override def asyncDeleteMessagesTo(persistenceId: String, toSequenceNr: Long): Future[Unit] =
|
||||
Future.fromTry(Try(storage.tryDelete(persistenceId, toSequenceNr)))
|
||||
|
||||
override def asyncReplayMessages(persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, max: Long)(
|
||||
recoveryCallback: PersistentRepr => Unit): Future[Unit] =
|
||||
Future.fromTry(Try(storage.tryRead(persistenceId, fromSequenceNr, toSequenceNr, max).foreach(recoveryCallback)))
|
||||
Future.fromTry(
|
||||
Try(
|
||||
storage
|
||||
.tryRead(persistenceId, fromSequenceNr, toSequenceNr, max)
|
||||
.map { repr =>
|
||||
// we keep the tags in the repr, so remove those here
|
||||
repr.payload match {
|
||||
case Tagged(payload, _) => repr.withPayload(payload)
|
||||
case _ => repr
|
||||
}
|
||||
|
||||
}
|
||||
.foreach(recoveryCallback)))
|
||||
|
||||
override def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] =
|
||||
Future.fromTry(Try {
|
||||
|
|
@ -53,7 +78,7 @@ class PersistenceTestKitPlugin extends AsyncWriteJournal {
|
|||
|
||||
object PersistenceTestKitPlugin {
|
||||
|
||||
val PluginId = "akka.persistence.testkit.journal.pluginid"
|
||||
val PluginId = "akka.persistence.testkit.journal"
|
||||
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
|
||||
|
|
@ -64,6 +89,8 @@ object PersistenceTestKitPlugin {
|
|||
"akka.persistence.journal.plugin" -> PluginId,
|
||||
s"$PluginId.class" -> s"${classOf[PersistenceTestKitPlugin].getName}").asJava)
|
||||
|
||||
private[testkit] case class Write(persistenceId: String, toSequenceNr: Long)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -6,6 +6,8 @@ package akka.persistence.testkit
|
|||
|
||||
import akka.annotation.{ ApiMayChange, InternalApi }
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
/**
|
||||
* Policies allow to emulate behavior of the storage (failures and rejections).
|
||||
*
|
||||
|
|
@ -150,7 +152,7 @@ object ExpectedRejection extends ExpectedRejection {
|
|||
|
||||
}
|
||||
|
||||
sealed abstract class ExpectedFailure extends Throwable
|
||||
sealed abstract class ExpectedFailure extends Throwable with NoStackTrace
|
||||
|
||||
object ExpectedFailure extends ExpectedFailure {
|
||||
|
||||
|
|
|
|||
|
|
@ -4,8 +4,9 @@
|
|||
|
||||
package akka.persistence.testkit
|
||||
|
||||
import scala.util.Success
|
||||
import akka.actor.Extension
|
||||
|
||||
import scala.util.Success
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.{ SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria }
|
||||
import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies
|
||||
|
|
@ -15,7 +16,9 @@ import akka.persistence.testkit.internal.TestKitStorage
|
|||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[testkit] trait SnapshotStorage extends TestKitStorage[SnapshotOperation, (SnapshotMetadata, Any)] {
|
||||
private[testkit] trait SnapshotStorage
|
||||
extends TestKitStorage[SnapshotOperation, (SnapshotMetadata, Any)]
|
||||
with Extension {
|
||||
|
||||
import SnapshotStorage._
|
||||
|
||||
|
|
|
|||
|
|
@ -4,26 +4,55 @@
|
|||
|
||||
package akka.persistence.testkit.internal
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import akka.actor.Extension
|
||||
import akka.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.testkit.EventStorage
|
||||
import akka.persistence.testkit.JournalOperation
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.ProcessingPolicy
|
||||
import akka.persistence.testkit.scaladsl.PersistenceTestKit
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[testkit] object InMemStorageExtension extends ExtensionId[EventStorage] with ExtensionIdProvider {
|
||||
private[testkit] object InMemStorageExtension extends ExtensionId[InMemStorageExtension] with ExtensionIdProvider {
|
||||
|
||||
override def get(system: ActorSystem): EventStorage = super.get(system)
|
||||
override def get(system: ActorSystem): InMemStorageExtension = super.get(system)
|
||||
|
||||
override def createExtension(system: ExtendedActorSystem) =
|
||||
if (PersistenceTestKit.Settings(system).serialize) {
|
||||
new SerializedEventStorageImpl(system)
|
||||
} else {
|
||||
new SimpleEventStorageImpl
|
||||
}
|
||||
override def createExtension(system: ExtendedActorSystem): InMemStorageExtension =
|
||||
new InMemStorageExtension(system)
|
||||
|
||||
override def lookup = InMemStorageExtension
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
final class InMemStorageExtension(system: ExtendedActorSystem) extends Extension {
|
||||
|
||||
private val stores = new ConcurrentHashMap[String, EventStorage]()
|
||||
|
||||
def defaultStorage(): EventStorage = storageFor(PersistenceTestKitPlugin.PluginId)
|
||||
|
||||
// shortcuts for default policy
|
||||
def currentPolicy: ProcessingPolicy[JournalOperation] = defaultStorage().currentPolicy
|
||||
def setPolicy(policy: ProcessingPolicy[JournalOperation]): Unit = defaultStorage().setPolicy(policy)
|
||||
def resetPolicy(): Unit = defaultStorage().resetPolicy()
|
||||
|
||||
def storageFor(key: String): EventStorage =
|
||||
stores.computeIfAbsent(key, _ => {
|
||||
// we don't really care about the key here, we just want separate instances
|
||||
if (PersistenceTestKit.Settings(system).serialize) {
|
||||
new SerializedEventStorageImpl(system)
|
||||
} else {
|
||||
new SimpleEventStorageImpl
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,40 +4,69 @@
|
|||
|
||||
package akka.persistence.testkit.internal
|
||||
|
||||
import scala.util.Try
|
||||
|
||||
import akka.actor.{ ActorSystem, ExtendedActorSystem }
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.PersistentRepr
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.testkit.EventStorage
|
||||
import akka.serialization.{ Serialization, SerializationExtension }
|
||||
import akka.persistence.testkit.internal.SerializedEventStorageImpl.Serialized
|
||||
import akka.serialization.{ Serialization, SerializationExtension, Serializers }
|
||||
|
||||
@InternalApi
|
||||
private[testkit] object SerializedEventStorageImpl {
|
||||
case class Serialized(
|
||||
persistenceId: String,
|
||||
sequenceNr: Long,
|
||||
payloadSerId: Int,
|
||||
payloadSerManifest: String,
|
||||
writerUuid: String,
|
||||
payload: Array[Byte],
|
||||
tags: Set[String],
|
||||
metadata: Option[Any])
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* FIXME, once we add serializers for metadata serialize the metadata payload if present
|
||||
*/
|
||||
@InternalApi
|
||||
private[testkit] class SerializedEventStorageImpl(system: ActorSystem) extends EventStorage {
|
||||
|
||||
override type InternalRepr = (Int, Array[Byte])
|
||||
override type InternalRepr = Serialized
|
||||
|
||||
private lazy val serialization = SerializationExtension(system)
|
||||
|
||||
/**
|
||||
* @return (serializer id, serialized bytes)
|
||||
*/
|
||||
override def toInternal(repr: PersistentRepr): (Int, Array[Byte]) =
|
||||
override def toInternal(pr: PersistentRepr): Serialized =
|
||||
Serialization.withTransportInformation(system.asInstanceOf[ExtendedActorSystem]) { () =>
|
||||
val s = serialization.findSerializerFor(repr)
|
||||
(s.identifier, s.toBinary(repr))
|
||||
val (payload, tags) = pr.payload match {
|
||||
case Tagged(event: AnyRef, tags) => (event, tags)
|
||||
case event: AnyRef => (event, Set.empty[String])
|
||||
}
|
||||
val s = serialization.findSerializerFor(payload)
|
||||
val manifest = Serializers.manifestFor(s, payload)
|
||||
Serialized(
|
||||
pr.persistenceId,
|
||||
pr.sequenceNr,
|
||||
s.identifier,
|
||||
manifest,
|
||||
pr.writerUuid,
|
||||
s.toBinary(payload),
|
||||
tags,
|
||||
pr.metadata)
|
||||
}
|
||||
|
||||
/**
|
||||
* @param internal (serializer id, serialized bytes)
|
||||
*/
|
||||
override def toRepr(internal: (Int, Array[Byte])): PersistentRepr =
|
||||
serialization
|
||||
.deserialize(internal._2, internal._1, PersistentRepr.Undefined)
|
||||
.flatMap(r => Try(r.asInstanceOf[PersistentRepr]))
|
||||
.get
|
||||
override def toRepr(internal: Serialized): PersistentRepr = {
|
||||
val event = serialization.deserialize(internal.payload, internal.payloadSerId, internal.payloadSerManifest).get
|
||||
val eventForRepr =
|
||||
if (internal.tags.isEmpty) event
|
||||
else Tagged(event, internal.tags)
|
||||
val pr = PersistentRepr(eventForRepr, internal.sequenceNr, internal.persistenceId, writerUuid = internal.writerUuid)
|
||||
internal.metadata.fold(pr)(meta => pr.withMetadata(meta))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,7 +4,8 @@
|
|||
|
||||
package akka.persistence.testkit.internal
|
||||
|
||||
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
|
||||
import akka.actor.Extension
|
||||
import akka.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.testkit.SnapshotStorage
|
||||
import akka.persistence.testkit.scaladsl.SnapshotTestKit
|
||||
|
|
|
|||
|
|
@ -8,8 +8,8 @@ import java.util.concurrent.ConcurrentHashMap
|
|||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
|
||||
import akka.actor.Extension
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.testkit.ProcessingPolicy
|
||||
|
||||
|
|
@ -87,6 +87,10 @@ sealed trait InMemStorage[K, R] extends InternalReprSupport[R] {
|
|||
def read(key: K): Option[Vector[R]] =
|
||||
Option(eventsMap.get(key)).map(_._2.map(toRepr))
|
||||
|
||||
def readAll(): Iterable[R] = {
|
||||
eventsMap.values().asScala.flatMap { case (_, events) => events }.map(toRepr)
|
||||
}
|
||||
|
||||
def clearAll(): Unit =
|
||||
eventsMap.clear()
|
||||
|
||||
|
|
@ -151,4 +155,4 @@ sealed trait PolicyOps[U] {
|
|||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[testkit] trait TestKitStorage[P, R] extends InMemStorage[String, R] with PolicyOps[P] with Extension
|
||||
private[testkit] trait TestKitStorage[P, R] extends InMemStorage[String, R] with PolicyOps[P]
|
||||
|
|
|
|||
|
|
@ -0,0 +1,18 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.query
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.persistence.query.ReadJournalProvider
|
||||
import com.typesafe.config.Config
|
||||
|
||||
class PersistenceTestKitReadJournalProvider(system: ExtendedActorSystem, config: Config, configPath: String)
|
||||
extends ReadJournalProvider {
|
||||
|
||||
override def scaladslReadJournal(): scaladsl.PersistenceTestKitReadJournal =
|
||||
new scaladsl.PersistenceTestKitReadJournal(system, config, configPath)
|
||||
|
||||
override def javadslReadJournal(): javadsl.PersistenceTestKitReadJournal =
|
||||
new javadsl.PersistenceTestKitReadJournal(scaladslReadJournal())
|
||||
}
|
||||
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.query.internal
|
||||
import akka.actor.ActorRef
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.query.{ EventEnvelope, Sequence }
|
||||
import akka.persistence.testkit.{ EventStorage, PersistenceTestKitPlugin }
|
||||
import akka.stream.{ Attributes, Outlet, SourceShape }
|
||||
import akka.stream.stage.{ GraphStage, GraphStageLogic, GraphStageLogicWithLogging, OutHandler }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
final private[akka] class EventsByPersistenceIdStage(
|
||||
persistenceId: String,
|
||||
fromSequenceNr: Long,
|
||||
toSequenceNr: Long,
|
||||
storage: EventStorage)
|
||||
extends GraphStage[SourceShape[EventEnvelope]] {
|
||||
val out: Outlet[EventEnvelope] = Outlet("EventsByPersistenceIdSource")
|
||||
override def shape: SourceShape[EventEnvelope] = SourceShape(out)
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = {
|
||||
new GraphStageLogicWithLogging(shape) with OutHandler {
|
||||
private var currentSequenceNr = math.max(fromSequenceNr, 1)
|
||||
private var stageActorRef: ActorRef = null
|
||||
override def preStart(): Unit = {
|
||||
stageActorRef = getStageActor(receiveNotifications).ref
|
||||
materializer.system.eventStream.subscribe(stageActorRef, classOf[PersistenceTestKitPlugin.Write])
|
||||
}
|
||||
|
||||
private def receiveNotifications(in: (ActorRef, Any)): Unit = {
|
||||
val (_, msg) = in
|
||||
msg match {
|
||||
case PersistenceTestKitPlugin.Write(pid, toSequenceNr) if pid == persistenceId =>
|
||||
if (toSequenceNr >= currentSequenceNr) {
|
||||
tryPush()
|
||||
}
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
|
||||
private def tryPush(): Unit = {
|
||||
if (isAvailable(out)) {
|
||||
val event = storage.tryRead(persistenceId, currentSequenceNr, currentSequenceNr, 1)
|
||||
log.debug("tryPush available. Query for {} {} result {}", currentSequenceNr, currentSequenceNr, event)
|
||||
event.headOption match {
|
||||
case Some(pr) =>
|
||||
push(out, EventEnvelope(Sequence(pr.sequenceNr), pr.persistenceId, pr.sequenceNr, pr.payload match {
|
||||
case Tagged(payload, _) => payload
|
||||
case payload => payload
|
||||
}, pr.timestamp, pr.metadata))
|
||||
if (currentSequenceNr == toSequenceNr) {
|
||||
completeStage()
|
||||
} else {
|
||||
currentSequenceNr += 1
|
||||
}
|
||||
case None =>
|
||||
}
|
||||
} else {
|
||||
log.debug("tryPush, no demand")
|
||||
}
|
||||
}
|
||||
|
||||
override def onPull(): Unit = {
|
||||
tryPush()
|
||||
}
|
||||
|
||||
setHandler(out, this)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.query.javadsl
|
||||
import akka.NotUsed
|
||||
import akka.persistence.query.EventEnvelope
|
||||
import akka.persistence.query.Offset
|
||||
import akka.persistence.query.javadsl.{
|
||||
CurrentEventsByPersistenceIdQuery,
|
||||
CurrentEventsByTagQuery,
|
||||
EventsByPersistenceIdQuery,
|
||||
ReadJournal
|
||||
}
|
||||
import akka.stream.javadsl.Source
|
||||
import akka.persistence.testkit.query.scaladsl
|
||||
|
||||
object PersistenceTestKitReadJournal {
|
||||
val Identifier = "akka.persistence.testkit.query"
|
||||
}
|
||||
|
||||
final class PersistenceTestKitReadJournal(delegate: scaladsl.PersistenceTestKitReadJournal)
|
||||
extends ReadJournal
|
||||
with EventsByPersistenceIdQuery
|
||||
with CurrentEventsByPersistenceIdQuery
|
||||
with CurrentEventsByTagQuery {
|
||||
|
||||
override def eventsByPersistenceId(
|
||||
persistenceId: String,
|
||||
fromSequenceNr: Long,
|
||||
toSequenceNr: Long): Source[EventEnvelope, NotUsed] =
|
||||
delegate.eventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr).asJava
|
||||
|
||||
override def currentEventsByPersistenceId(
|
||||
persistenceId: String,
|
||||
fromSequenceNr: Long,
|
||||
toSequenceNr: Long): Source[EventEnvelope, NotUsed] =
|
||||
delegate.currentEventsByPersistenceId(persistenceId, fromSequenceNr, toSequenceNr).asJava
|
||||
|
||||
override def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] =
|
||||
delegate.currentEventsByTag(tag, offset).asJava
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.query.scaladsl
|
||||
import akka.NotUsed
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.query.NoOffset
|
||||
import akka.persistence.query.Offset
|
||||
import akka.persistence.query.scaladsl.CurrentEventsByTagQuery
|
||||
import akka.persistence.query.{ EventEnvelope, Sequence }
|
||||
import akka.persistence.query.scaladsl.{ CurrentEventsByPersistenceIdQuery, EventsByPersistenceIdQuery, ReadJournal }
|
||||
import akka.persistence.testkit.EventStorage
|
||||
import akka.persistence.testkit.internal.InMemStorageExtension
|
||||
import akka.persistence.testkit.query.internal.EventsByPersistenceIdStage
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.util.unused
|
||||
import com.typesafe.config.Config
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
object PersistenceTestKitReadJournal {
|
||||
val Identifier = "akka.persistence.testkit.query"
|
||||
}
|
||||
|
||||
final class PersistenceTestKitReadJournal(system: ExtendedActorSystem, @unused config: Config, configPath: String)
|
||||
extends ReadJournal
|
||||
with EventsByPersistenceIdQuery
|
||||
with CurrentEventsByPersistenceIdQuery
|
||||
with CurrentEventsByTagQuery {
|
||||
|
||||
private val log = LoggerFactory.getLogger(getClass)
|
||||
|
||||
private val storage: EventStorage = {
|
||||
// use shared path up to before `query` to identify which inmem journal we are addressing
|
||||
val storagePluginId = configPath.replaceAll("""query$""", "journal")
|
||||
log.debug("Using in memory storage [{}] for test kit read journal", storagePluginId)
|
||||
InMemStorageExtension(system).storageFor(storagePluginId)
|
||||
}
|
||||
|
||||
private def unwrapTaggedPayload(payload: Any): Any = payload match {
|
||||
case Tagged(payload, _) => payload
|
||||
case payload => payload
|
||||
}
|
||||
|
||||
override def eventsByPersistenceId(
|
||||
persistenceId: String,
|
||||
fromSequenceNr: Long,
|
||||
toSequenceNr: Long): Source[EventEnvelope, NotUsed] = {
|
||||
Source.fromGraph(new EventsByPersistenceIdStage(persistenceId, fromSequenceNr, toSequenceNr, storage))
|
||||
}
|
||||
|
||||
override def currentEventsByPersistenceId(
|
||||
persistenceId: String,
|
||||
fromSequenceNr: Long,
|
||||
toSequenceNr: Long): Source[EventEnvelope, NotUsed] = {
|
||||
Source(storage.tryRead(persistenceId, fromSequenceNr, toSequenceNr, Long.MaxValue)).map { pr =>
|
||||
EventEnvelope(
|
||||
Sequence(pr.sequenceNr),
|
||||
persistenceId,
|
||||
pr.sequenceNr,
|
||||
unwrapTaggedPayload(pr.payload),
|
||||
pr.timestamp,
|
||||
pr.metadata)
|
||||
}
|
||||
}
|
||||
|
||||
override def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] = {
|
||||
offset match {
|
||||
case NoOffset =>
|
||||
case _ =>
|
||||
throw new UnsupportedOperationException("Offsets not supported for persistence test kit currentEventsByTag yet")
|
||||
}
|
||||
Source(storage.tryReadByTag(tag)).map { pr =>
|
||||
EventEnvelope(
|
||||
Sequence(pr.timestamp),
|
||||
pr.persistenceId,
|
||||
pr.sequenceNr,
|
||||
unwrapTaggedPayload(pr.payload),
|
||||
pr.timestamp,
|
||||
pr.metadata)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -7,9 +7,7 @@ package akka.persistence.testkit.scaladsl
|
|||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.Try
|
||||
|
||||
import com.typesafe.config.Config
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.ClassicActorSystemProvider
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
|
@ -20,6 +18,7 @@ import akka.annotation.ApiMayChange
|
|||
import akka.persistence.Persistence
|
||||
import akka.persistence.PersistentRepr
|
||||
import akka.persistence.SnapshotMetadata
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.testkit._
|
||||
import akka.persistence.testkit.internal.InMemStorageExtension
|
||||
import akka.persistence.testkit.internal.SnapshotStorageEmulatorExtension
|
||||
|
|
@ -433,7 +432,7 @@ class PersistenceTestKit(system: ActorSystem)
|
|||
|
||||
import PersistenceTestKit._
|
||||
|
||||
override protected val storage = InMemStorageExtension(system)
|
||||
override protected val storage = InMemStorageExtension(system).storageFor(PersistenceTestKitPlugin.PluginId)
|
||||
|
||||
private final lazy val settings = Settings(system)
|
||||
|
||||
|
|
@ -495,7 +494,10 @@ class PersistenceTestKit(system: ActorSystem)
|
|||
def persistedInStorage(persistenceId: String): immutable.Seq[Any] =
|
||||
storage.read(persistenceId).getOrElse(List.empty).map(reprToAny)
|
||||
|
||||
override private[testkit] def reprToAny(repr: PersistentRepr): Any = repr.payload
|
||||
override private[testkit] def reprToAny(repr: PersistentRepr): Any = repr.payload match {
|
||||
case Tagged(payload, _) => payload
|
||||
case payload => payload
|
||||
}
|
||||
}
|
||||
|
||||
@ApiMayChange
|
||||
|
|
|
|||
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.query
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.persistence.query.NoOffset
|
||||
import akka.persistence.query.PersistenceQuery
|
||||
import akka.persistence.testkit.query.EventsByPersistenceIdSpec.Command
|
||||
import akka.persistence.testkit.query.EventsByPersistenceIdSpec.testBehaviour
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.stream.scaladsl.Sink
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
class CurrentEventsByTagSpec
|
||||
extends ScalaTestWithActorTestKit(EventsByPersistenceIdSpec.config)
|
||||
with LogCapturing
|
||||
with AnyWordSpecLike {
|
||||
|
||||
implicit val classic = system.classicSystem
|
||||
|
||||
val queries =
|
||||
PersistenceQuery(system).readJournalFor[PersistenceTestKitReadJournal](PersistenceTestKitReadJournal.Identifier)
|
||||
|
||||
def setup(persistenceId: String): ActorRef[Command] = {
|
||||
val probe = createTestProbe[Done]()
|
||||
val ref = setupEmpty(persistenceId)
|
||||
ref ! Command(s"$persistenceId-1", probe.ref)
|
||||
ref ! Command(s"$persistenceId-2", probe.ref)
|
||||
ref ! Command(s"$persistenceId-3", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
probe.expectMessage(Done)
|
||||
probe.expectMessage(Done)
|
||||
ref
|
||||
}
|
||||
|
||||
def setupEmpty(persistenceId: String): ActorRef[Command] = {
|
||||
spawn(
|
||||
testBehaviour(persistenceId).withTagger(evt =>
|
||||
if (evt.indexOf('-') > 0) Set(evt.split('-')(1), "all")
|
||||
else Set("all")))
|
||||
}
|
||||
|
||||
"Persistent test kit currentEventsByTag query" must {
|
||||
|
||||
"find tagged events ordered by insert time" in {
|
||||
val probe = createTestProbe[Done]()
|
||||
val ref1 = setupEmpty("taggedpid-1")
|
||||
val ref2 = setupEmpty("taggedpid-2")
|
||||
ref1 ! Command("evt-1", probe.ref)
|
||||
ref1 ! Command("evt-2", probe.ref)
|
||||
ref1 ! Command("evt-3", probe.ref)
|
||||
probe.receiveMessages(3)
|
||||
ref2 ! Command("evt-4", probe.ref)
|
||||
probe.receiveMessage()
|
||||
ref1 ! Command("evt-5", probe.ref)
|
||||
probe.receiveMessage()
|
||||
|
||||
queries.currentEventsByTag("all", NoOffset).runWith(Sink.seq).futureValue.map(_.event) should ===(
|
||||
Seq("evt-1", "evt-2", "evt-3", "evt-4", "evt-5"))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,137 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.query
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit }
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.persistence.query.{ EventEnvelope, PersistenceQuery }
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.PersistenceId
|
||||
import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior }
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
object EventsByPersistenceIdSpec {
|
||||
val config = PersistenceTestKitPlugin.config.withFallback(
|
||||
ConfigFactory.parseString("""
|
||||
akka.loglevel = DEBUG
|
||||
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
|
||||
akka.persistence.testkit.events.serialize = off
|
||||
"""))
|
||||
|
||||
case class Command(evt: String, ack: ActorRef[Done])
|
||||
case class State()
|
||||
|
||||
def testBehaviour(persistenceId: String) = {
|
||||
EventSourcedBehavior[Command, String, State](
|
||||
PersistenceId.ofUniqueId(persistenceId),
|
||||
State(),
|
||||
(_, command) =>
|
||||
Effect.persist(command.evt).thenRun { _ =>
|
||||
command.ack ! Done
|
||||
},
|
||||
(state, _) => state)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class EventsByPersistenceIdSpec
|
||||
extends ScalaTestWithActorTestKit(EventsByPersistenceIdSpec.config)
|
||||
with LogCapturing
|
||||
with AnyWordSpecLike {
|
||||
import EventsByPersistenceIdSpec._
|
||||
|
||||
implicit val classic = system.classicSystem
|
||||
|
||||
val queries =
|
||||
PersistenceQuery(system).readJournalFor[PersistenceTestKitReadJournal](PersistenceTestKitReadJournal.Identifier)
|
||||
|
||||
def setup(persistenceId: String): ActorRef[Command] = {
|
||||
val probe = createTestProbe[Done]()
|
||||
val ref = setupEmpty(persistenceId)
|
||||
ref ! Command(s"$persistenceId-1", probe.ref)
|
||||
ref ! Command(s"$persistenceId-2", probe.ref)
|
||||
ref ! Command(s"$persistenceId-3", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
probe.expectMessage(Done)
|
||||
probe.expectMessage(Done)
|
||||
ref
|
||||
}
|
||||
|
||||
def setupEmpty(persistenceId: String): ActorRef[Command] = {
|
||||
spawn(testBehaviour(persistenceId))
|
||||
}
|
||||
|
||||
"Persistent test kit live query EventsByPersistenceId" must {
|
||||
"find new events" in {
|
||||
val ackProbe = createTestProbe[Done]()
|
||||
val ref = setup("c")
|
||||
val src = queries.eventsByPersistenceId("c", 0L, Long.MaxValue)
|
||||
val probe = src.map(_.event).runWith(TestSink.probe[Any]).request(5).expectNext("c-1", "c-2", "c-3")
|
||||
|
||||
ref ! Command("c-4", ackProbe.ref)
|
||||
ackProbe.expectMessage(Done)
|
||||
|
||||
probe.expectNext("c-4")
|
||||
}
|
||||
|
||||
"find new events up to a sequence number" in {
|
||||
val ackProbe = createTestProbe[Done]()
|
||||
val ref = setup("d")
|
||||
val src = queries.eventsByPersistenceId("d", 0L, 4L)
|
||||
val probe = src.map(_.event).runWith(TestSink.probe[Any]).request(5).expectNext("d-1", "d-2", "d-3")
|
||||
|
||||
ref ! Command("d-4", ackProbe.ref)
|
||||
ackProbe.expectMessage(Done)
|
||||
|
||||
probe.expectNext("d-4").expectComplete()
|
||||
}
|
||||
|
||||
"find new events after demand request" in {
|
||||
val ackProbe = createTestProbe[Done]()
|
||||
val ref = setup("e")
|
||||
val src = queries.eventsByPersistenceId("e", 0L, Long.MaxValue)
|
||||
val probe =
|
||||
src.map(_.event).runWith(TestSink.probe[Any]).request(2).expectNext("e-1", "e-2").expectNoMessage(100.millis)
|
||||
|
||||
ref ! Command("e-4", ackProbe.ref)
|
||||
ackProbe.expectMessage(Done)
|
||||
|
||||
probe.expectNoMessage(100.millis).request(5).expectNext("e-3").expectNext("e-4")
|
||||
}
|
||||
|
||||
"include timestamp in EventEnvelope" in {
|
||||
setup("n")
|
||||
|
||||
val src = queries.eventsByPersistenceId("n", 0L, Long.MaxValue)
|
||||
val probe = src.runWith(TestSink.probe[EventEnvelope])
|
||||
|
||||
probe.request(5)
|
||||
probe.expectNext().timestamp should be > 0L
|
||||
probe.expectNext().timestamp should be > 0L
|
||||
probe.cancel()
|
||||
}
|
||||
|
||||
"not complete for empty persistence id" in {
|
||||
val ackProbe = createTestProbe[Done]()
|
||||
val src = queries.eventsByPersistenceId("o", 0L, Long.MaxValue)
|
||||
val probe =
|
||||
src.map(_.event).runWith(TestSink.probe[Any]).request(2)
|
||||
|
||||
probe.expectNoMessage(200.millis) // must not complete
|
||||
|
||||
val ref = setupEmpty("o")
|
||||
ref ! Command("o-1", ackProbe.ref)
|
||||
ackProbe.expectMessage(Done)
|
||||
|
||||
probe.cancel()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -234,7 +234,7 @@ class EventSourcedBehaviorTestKitSpec
|
|||
val eventSourcedTestKit = createTestKit()
|
||||
|
||||
val exc = intercept[IllegalArgumentException] {
|
||||
eventSourcedTestKit.runCommand(TestCounter.IncrementWithNotSerializableReply(_))
|
||||
eventSourcedTestKit.runCommand(TestCounter.IncrementWithNotSerializableReply)
|
||||
}
|
||||
(exc.getMessage should include).regex("Reply.*isn't serializable")
|
||||
exc.getCause.getClass should ===(classOf[NotSerializableException])
|
||||
|
|
|
|||
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.testkit.scaladsl
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.query.PersistenceQuery
|
||||
import akka.persistence.query.scaladsl.CurrentEventsByPersistenceIdQuery
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.typed.PersistenceId
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.stream.scaladsl.Sink
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object MultipleJournalsSpec {
|
||||
|
||||
object ListActor {
|
||||
sealed trait Command
|
||||
case class Save(text: String, replyTo: ActorRef[Done]) extends Command
|
||||
case class ShowMeWhatYouGot(replyTo: ActorRef[Set[String]]) extends Command
|
||||
case object Stop extends Command
|
||||
|
||||
def apply(persistenceId: String, journal: String): Behavior[Command] =
|
||||
EventSourcedBehavior[Command, String, Set[String]](
|
||||
PersistenceId.ofUniqueId(persistenceId),
|
||||
Set.empty[String],
|
||||
(state, cmd) =>
|
||||
cmd match {
|
||||
case Save(text, replyTo) =>
|
||||
Effect.persist(text).thenRun(_ => replyTo ! Done)
|
||||
case ShowMeWhatYouGot(replyTo) =>
|
||||
replyTo ! state
|
||||
Effect.none
|
||||
case Stop =>
|
||||
Effect.stop()
|
||||
},
|
||||
(state, evt) => state + evt).withJournalPluginId(journal)
|
||||
|
||||
}
|
||||
|
||||
def config = ConfigFactory.parseString(s"""
|
||||
journal1 {
|
||||
# journal and query expected to be next to each other under config path
|
||||
journal.class = "${classOf[PersistenceTestKitPlugin].getName}"
|
||||
query = $${akka.persistence.testkit.query}
|
||||
}
|
||||
journal2 {
|
||||
journal.class = "${classOf[PersistenceTestKitPlugin].getName}"
|
||||
query = $${akka.persistence.testkit.query}
|
||||
}
|
||||
""").withFallback(ConfigFactory.load()).resolve()
|
||||
|
||||
}
|
||||
|
||||
class MultipleJournalsSpec
|
||||
extends ScalaTestWithActorTestKit(MultipleJournalsSpec.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
|
||||
import MultipleJournalsSpec._
|
||||
|
||||
"The testkit journal and query plugin" must {
|
||||
|
||||
"be possible to configure and use in multiple isolated instances" in {
|
||||
val probe = createTestProbe[Any]()
|
||||
|
||||
{
|
||||
// one actor in each journal with same id
|
||||
val j1 = spawn(ListActor("id1", "journal1.journal"))
|
||||
val j2 = spawn(ListActor("id1", "journal2.journal"))
|
||||
j1 ! ListActor.Save("j1m1", probe.ref)
|
||||
probe.receiveMessage()
|
||||
j2 ! ListActor.Save("j2m1", probe.ref)
|
||||
probe.receiveMessage()
|
||||
|
||||
j1 ! ListActor.Stop
|
||||
probe.expectTerminated(j1)
|
||||
j2 ! ListActor.Stop
|
||||
probe.expectTerminated(j2)
|
||||
}
|
||||
|
||||
{
|
||||
// new incarnations in each journal with same id
|
||||
val j1 = spawn(ListActor("id1", "journal1.journal"))
|
||||
val j2 = spawn(ListActor("id1", "journal2.journal"))
|
||||
|
||||
// does not see each others events
|
||||
j1 ! ListActor.ShowMeWhatYouGot(probe.ref)
|
||||
probe.expectMessage(Set("j1m1"))
|
||||
j2 ! ListActor.ShowMeWhatYouGot(probe.ref)
|
||||
probe.expectMessage(Set("j2m1"))
|
||||
}
|
||||
|
||||
val readJournal1 = PersistenceQuery(system).readJournalFor[CurrentEventsByPersistenceIdQuery]("journal1.query")
|
||||
val readJournal2 = PersistenceQuery(system).readJournalFor[CurrentEventsByPersistenceIdQuery]("journal2.query")
|
||||
|
||||
val eventsForJournal1 =
|
||||
readJournal1.currentEventsByPersistenceId("id1", 0L, Long.MaxValue).runWith(Sink.seq).futureValue
|
||||
eventsForJournal1.map(_.event) should ===(Seq("j1m1"))
|
||||
|
||||
val eventsForJournal2 =
|
||||
readJournal2.currentEventsByPersistenceId("id1", 0L, Long.MaxValue).runWith(Sink.seq).futureValue
|
||||
eventsForJournal2.map(_.event) should ===(Seq("j2m1"))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -37,6 +37,7 @@ class PersistenceTestKitJournalCompatSpec extends JournalSpec(config = Persisten
|
|||
}
|
||||
|
||||
override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = true
|
||||
override protected def supportsMetadata: CapabilityFlag = true
|
||||
}
|
||||
|
||||
class PersistenceTestKitSnapshotStoreCompatSpec
|
||||
|
|
|
|||
|
|
@ -0,0 +1,206 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed;
|
||||
|
||||
import akka.Done;
|
||||
import akka.actor.testkit.typed.javadsl.LogCapturing;
|
||||
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
|
||||
import akka.actor.testkit.typed.javadsl.TestProbe;
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin;
|
||||
import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.javadsl.*;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.scalatestplus.junit.JUnitSuite;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
import static akka.Done.done;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class ReplicatedEventSourcingTest extends JUnitSuite {
|
||||
|
||||
static final class TestBehavior
|
||||
extends ReplicatedEventSourcedBehavior<TestBehavior.Command, String, Set<String>> {
|
||||
interface Command {}
|
||||
|
||||
static final class GetState implements Command {
|
||||
final ActorRef<State> replyTo;
|
||||
|
||||
public GetState(ActorRef<State> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static final class StoreMe implements Command {
|
||||
final String text;
|
||||
final ActorRef<Done> replyTo;
|
||||
|
||||
public StoreMe(String text, ActorRef<Done> replyTo) {
|
||||
this.text = text;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static final class StoreUs implements Command {
|
||||
final List<String> texts;
|
||||
final ActorRef<Done> replyTo;
|
||||
|
||||
public StoreUs(List<String> texts, ActorRef<Done> replyTo) {
|
||||
this.texts = texts;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static final class GetReplica implements Command {
|
||||
final ActorRef<ReplicaId> replyTo;
|
||||
|
||||
public GetReplica(ActorRef<ReplicaId> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static final class State {
|
||||
final Set<String> texts;
|
||||
|
||||
public State(Set<String> texts) {
|
||||
this.texts = texts;
|
||||
}
|
||||
}
|
||||
|
||||
enum Stop implements Command {
|
||||
INSTANCE
|
||||
}
|
||||
|
||||
public static Behavior<Command> create(
|
||||
String entityId, ReplicaId replicaId, Set<ReplicaId> allReplicas) {
|
||||
return ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("ReplicatedEventSourcingTest", entityId, replicaId),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
TestBehavior::new);
|
||||
}
|
||||
|
||||
private TestBehavior(ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String journalPluginId() {
|
||||
return PersistenceTestKitPlugin.PluginId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> emptyState() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandHandler<Command, String, Set<String>> commandHandler() {
|
||||
return newCommandHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onCommand(
|
||||
StoreMe.class,
|
||||
(StoreMe cmd) -> Effect().persist(cmd.text).thenRun(__ -> cmd.replyTo.tell(done())))
|
||||
.onCommand(
|
||||
StoreUs.class,
|
||||
(StoreUs cmd) -> Effect().persist(cmd.texts).thenRun(__ -> cmd.replyTo.tell(done())))
|
||||
.onCommand(
|
||||
GetState.class,
|
||||
(GetState get) ->
|
||||
Effect()
|
||||
.none()
|
||||
.thenRun(state -> get.replyTo.tell(new State(new HashSet<>(state)))))
|
||||
.onCommand(
|
||||
GetReplica.class,
|
||||
(GetReplica cmd) ->
|
||||
Effect()
|
||||
.none()
|
||||
.thenRun(() -> cmd.replyTo.tell(getReplicationContext().replicaId())))
|
||||
.onCommand(Stop.class, __ -> Effect().stop())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler<Set<String>, String> eventHandler() {
|
||||
return newEventHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onAnyEvent(
|
||||
(state, text) -> {
|
||||
// FIXME mutable - state I don't remember if we support or not so defensive copy for
|
||||
// now
|
||||
Set<String> newSet = new HashSet<>(state);
|
||||
newSet.add(text);
|
||||
return newSet;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@ClassRule
|
||||
public static final TestKitJunitResource testKit =
|
||||
new TestKitJunitResource(
|
||||
ConfigFactory.parseString(
|
||||
"akka.loglevel = INFO\n" + "akka.loggers = [\"akka.testkit.TestEventListener\"]")
|
||||
.withFallback(PersistenceTestKitPlugin.getInstance().config()));
|
||||
|
||||
@Rule public final LogCapturing logCapturing = new LogCapturing();
|
||||
|
||||
// minimal test, full coverage over in ReplicatedEventSourcingSpec
|
||||
@Test
|
||||
public void replicatedEventSourcingReplicationTest() {
|
||||
ReplicaId dcA = new ReplicaId("DC-A");
|
||||
ReplicaId dcB = new ReplicaId("DC-B");
|
||||
ReplicaId dcC = new ReplicaId("DC-C");
|
||||
Set<ReplicaId> allReplicas = new HashSet<>(Arrays.asList(dcA, dcB, dcC));
|
||||
|
||||
ActorRef<TestBehavior.Command> replicaA =
|
||||
testKit.spawn(TestBehavior.create("id1", dcA, allReplicas));
|
||||
ActorRef<TestBehavior.Command> replicaB =
|
||||
testKit.spawn(TestBehavior.create("id1", dcB, allReplicas));
|
||||
ActorRef<TestBehavior.Command> replicaC =
|
||||
testKit.spawn(TestBehavior.create("id1", dcC, allReplicas));
|
||||
|
||||
TestProbe<Object> probe = testKit.createTestProbe();
|
||||
replicaA.tell(new TestBehavior.GetReplica(probe.ref().narrow()));
|
||||
assertEquals("DC-A", probe.expectMessageClass(ReplicaId.class).id());
|
||||
|
||||
replicaA.tell(new TestBehavior.StoreMe("stored-to-a", probe.ref().narrow()));
|
||||
replicaB.tell(new TestBehavior.StoreMe("stored-to-b", probe.ref().narrow()));
|
||||
replicaC.tell(new TestBehavior.StoreMe("stored-to-c", probe.ref().narrow()));
|
||||
probe.receiveSeveralMessages(3);
|
||||
|
||||
probe.awaitAssert(
|
||||
() -> {
|
||||
replicaA.tell(new TestBehavior.GetState(probe.ref().narrow()));
|
||||
TestBehavior.State reply = probe.expectMessageClass(TestBehavior.State.class);
|
||||
assertEquals(
|
||||
reply.texts,
|
||||
new HashSet<String>(Arrays.asList("stored-to-a", "stored-to-b", "stored-to-c")));
|
||||
return null;
|
||||
});
|
||||
probe.awaitAssert(
|
||||
() -> {
|
||||
replicaB.tell(new TestBehavior.GetState(probe.ref().narrow()));
|
||||
TestBehavior.State reply = probe.expectMessageClass(TestBehavior.State.class);
|
||||
assertEquals(
|
||||
reply.texts,
|
||||
new HashSet<String>(Arrays.asList("stored-to-a", "stored-to-b", "stored-to-c")));
|
||||
return null;
|
||||
});
|
||||
probe.awaitAssert(
|
||||
() -> {
|
||||
replicaC.tell(new TestBehavior.GetState(probe.ref().narrow()));
|
||||
TestBehavior.State reply = probe.expectMessageClass(TestBehavior.State.class);
|
||||
assertEquals(
|
||||
reply.texts,
|
||||
new HashSet<String>(Arrays.asList("stored-to-a", "stored-to-b", "stored-to-c")));
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.persistence.typed;
|
||||
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.javadsl.*;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
// #factory
|
||||
public class MyReplicatedBehavior
|
||||
extends ReplicatedEventSourcedBehavior<
|
||||
MyReplicatedBehavior.Command, MyReplicatedBehavior.Event, MyReplicatedBehavior.State> {
|
||||
// #factory
|
||||
interface Command {}
|
||||
|
||||
interface State {}
|
||||
|
||||
interface Event {}
|
||||
|
||||
// #replicas
|
||||
public static final ReplicaId DCA = new ReplicaId("DCA");
|
||||
public static final ReplicaId DCB = new ReplicaId("DCB");
|
||||
|
||||
public static final Set<ReplicaId> ALL_REPLICAS =
|
||||
Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DCA, DCB)));
|
||||
// #replicas
|
||||
|
||||
// #factory-shared
|
||||
public static Behavior<Command> create(
|
||||
String entityId, ReplicaId replicaId, String queryPluginId) {
|
||||
return ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("MyReplicatedEntity", entityId, replicaId),
|
||||
ALL_REPLICAS,
|
||||
queryPluginId,
|
||||
MyReplicatedBehavior::new);
|
||||
}
|
||||
// #factory-shared
|
||||
|
||||
// #factory
|
||||
public static Behavior<Command> create(String entityId, ReplicaId replicaId) {
|
||||
Map<ReplicaId, String> allReplicasAndQueryPlugins = new HashMap<>();
|
||||
allReplicasAndQueryPlugins.put(DCA, "journalForDCA");
|
||||
allReplicasAndQueryPlugins.put(DCB, "journalForDCB");
|
||||
|
||||
return ReplicatedEventSourcing.perReplicaJournalConfig(
|
||||
new ReplicationId("MyReplicatedEntity", entityId, replicaId),
|
||||
allReplicasAndQueryPlugins,
|
||||
MyReplicatedBehavior::new);
|
||||
}
|
||||
|
||||
private MyReplicatedBehavior(ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
}
|
||||
// #factory
|
||||
|
||||
@Override
|
||||
public State emptyState() {
|
||||
throw new UnsupportedOperationException("dummy for example");
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandHandler<Command, Event, State> commandHandler() {
|
||||
throw new UnsupportedOperationException("dummy for example");
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler<State, Event> eventHandler() {
|
||||
throw new UnsupportedOperationException("dummy for example");
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,445 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.persistence.typed;
|
||||
|
||||
import akka.actor.testkit.typed.javadsl.LogCapturing;
|
||||
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
|
||||
import akka.actor.testkit.typed.javadsl.TestProbe;
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.actor.typed.javadsl.ActorContext;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.actor.typed.javadsl.TimerScheduler;
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin;
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.RecoveryCompleted;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.javadsl.CommandHandler;
|
||||
import akka.persistence.typed.javadsl.CommandHandlerBuilder;
|
||||
import akka.persistence.typed.javadsl.EventHandler;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcing;
|
||||
import akka.persistence.typed.javadsl.ReplicationContext;
|
||||
import akka.persistence.typed.javadsl.SignalHandler;
|
||||
import akka.serialization.jackson.CborSerializable;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.scalatestplus.junit.JUnitSuite;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static jdocs.akka.persistence.typed.AuctionEntity.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class ReplicatedAuctionExampleTest extends JUnitSuite {
|
||||
@ClassRule
|
||||
public static final TestKitJunitResource testKit =
|
||||
new TestKitJunitResource(PersistenceTestKitPlugin.getInstance().config());
|
||||
|
||||
@Rule public final LogCapturing logCapturing = new LogCapturing();
|
||||
|
||||
@Test
|
||||
public void auctionExample() {
|
||||
String auctionName = "old-skis";
|
||||
Bid initialBid = new Bid("chbatey", 12, Instant.now(), R1);
|
||||
Instant closeAt = Instant.now().plusSeconds(10);
|
||||
|
||||
ActorRef<Command> replicaA =
|
||||
testKit.spawn(AuctionEntity.create(R1, auctionName, initialBid, closeAt, true));
|
||||
ActorRef<Command> replicaB =
|
||||
testKit.spawn(AuctionEntity.create(R2, auctionName, initialBid, closeAt, false));
|
||||
|
||||
replicaA.tell(new OfferBid("me", 100));
|
||||
replicaA.tell(new OfferBid("me", 99));
|
||||
replicaA.tell(new OfferBid("me", 202));
|
||||
|
||||
TestProbe<Bid> replyProbe = testKit.createTestProbe();
|
||||
replyProbe.awaitAssert(
|
||||
() -> {
|
||||
replicaA.tell(new GetHighestBid(replyProbe.ref()));
|
||||
Bid bid = replyProbe.expectMessageClass(Bid.class);
|
||||
assertEquals(bid.offer, 202);
|
||||
return bid;
|
||||
});
|
||||
|
||||
replicaA.tell(Finish.INSTANCE);
|
||||
|
||||
TestProbe<Boolean> finishProbe = testKit.createTestProbe();
|
||||
finishProbe.awaitAssert(
|
||||
() -> {
|
||||
replicaA.tell(new IsClosed(finishProbe.ref()));
|
||||
return finishProbe.expectMessage(true);
|
||||
});
|
||||
finishProbe.awaitAssert(
|
||||
() -> {
|
||||
replicaB.tell(new IsClosed(finishProbe.ref()));
|
||||
return finishProbe.expectMessage(true);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
// #setup
|
||||
class AuctionEntity
|
||||
extends ReplicatedEventSourcedBehavior<
|
||||
AuctionEntity.Command, AuctionEntity.Event, AuctionEntity.AuctionState> {
|
||||
|
||||
public static ReplicaId R1 = new ReplicaId("R1");
|
||||
public static ReplicaId R2 = new ReplicaId("R2");
|
||||
|
||||
public static Set<ReplicaId> ALL_REPLICAS = new HashSet<>(Arrays.asList(R1, R2));
|
||||
|
||||
private final ActorContext<Command> context;
|
||||
private final TimerScheduler<Command> timers;
|
||||
private final Bid initialBid;
|
||||
private final Instant closingAt;
|
||||
private final boolean responsibleForClosing;
|
||||
|
||||
// #setup
|
||||
|
||||
// #commands
|
||||
public static final class Bid {
|
||||
public final String bidder;
|
||||
public final int offer;
|
||||
public final Instant timestamp;
|
||||
public final ReplicaId originReplica;
|
||||
|
||||
public Bid(String bidder, int offer, Instant timestamp, ReplicaId originReplica) {
|
||||
this.bidder = bidder;
|
||||
this.offer = offer;
|
||||
this.timestamp = timestamp;
|
||||
this.originReplica = originReplica;
|
||||
}
|
||||
}
|
||||
|
||||
interface Command extends CborSerializable {}
|
||||
|
||||
public enum Finish implements Command {
|
||||
INSTANCE
|
||||
}
|
||||
|
||||
public static final class OfferBid implements Command {
|
||||
public final String bidder;
|
||||
public final int offer;
|
||||
|
||||
public OfferBid(String bidder, int offer) {
|
||||
this.bidder = bidder;
|
||||
this.offer = offer;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class GetHighestBid implements Command {
|
||||
public final ActorRef<Bid> replyTo;
|
||||
|
||||
public GetHighestBid(ActorRef<Bid> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class IsClosed implements Command {
|
||||
public final ActorRef<Boolean> replyTo;
|
||||
|
||||
public IsClosed(ActorRef<Boolean> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
private enum Close implements Command {
|
||||
INSTANCE
|
||||
}
|
||||
// #commands
|
||||
|
||||
// #events
|
||||
interface Event extends CborSerializable {}
|
||||
|
||||
public static final class BidRegistered implements Event {
|
||||
public final Bid bid;
|
||||
|
||||
@JsonCreator
|
||||
public BidRegistered(Bid bid) {
|
||||
this.bid = bid;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class AuctionFinished implements Event {
|
||||
public final ReplicaId atReplica;
|
||||
|
||||
@JsonCreator
|
||||
public AuctionFinished(ReplicaId atReplica) {
|
||||
this.atReplica = atReplica;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class WinnerDecided implements Event {
|
||||
public final ReplicaId atReplica;
|
||||
public final Bid winningBid;
|
||||
public final int amount;
|
||||
|
||||
public WinnerDecided(ReplicaId atReplica, Bid winningBid, int amount) {
|
||||
this.atReplica = atReplica;
|
||||
this.winningBid = winningBid;
|
||||
this.amount = amount;
|
||||
}
|
||||
}
|
||||
// #events
|
||||
|
||||
// #state
|
||||
static class AuctionState implements CborSerializable {
|
||||
|
||||
final boolean stillRunning;
|
||||
final Bid highestBid;
|
||||
final int highestCounterOffer;
|
||||
final Set<String> finishedAtDc;
|
||||
|
||||
AuctionState(
|
||||
boolean stillRunning, Bid highestBid, int highestCounterOffer, Set<String> finishedAtDc) {
|
||||
this.stillRunning = stillRunning;
|
||||
this.highestBid = highestBid;
|
||||
this.highestCounterOffer = highestCounterOffer;
|
||||
this.finishedAtDc = finishedAtDc;
|
||||
}
|
||||
|
||||
AuctionState withNewHighestBid(Bid bid) {
|
||||
assert (stillRunning);
|
||||
assert (isHigherBid(bid, highestBid));
|
||||
return new AuctionState(
|
||||
stillRunning, bid, highestBid.offer, finishedAtDc); // keep last highest bid around
|
||||
}
|
||||
|
||||
AuctionState withTooLowBid(Bid bid) {
|
||||
assert (stillRunning);
|
||||
assert (isHigherBid(highestBid, bid));
|
||||
return new AuctionState(
|
||||
stillRunning, highestBid, Math.max(highestCounterOffer, bid.offer), finishedAtDc);
|
||||
}
|
||||
|
||||
static Boolean isHigherBid(Bid first, Bid second) {
|
||||
return first.offer > second.offer
|
||||
|| (first.offer == second.offer && first.timestamp.isBefore(second.timestamp))
|
||||
|| // if equal, first one wins
|
||||
// If timestamps are equal, choose by dc where the offer was submitted
|
||||
// In real auctions, this last comparison should be deterministic but unpredictable, so
|
||||
// that submitting to a
|
||||
// particular DC would not be an advantage.
|
||||
(first.offer == second.offer
|
||||
&& first.timestamp.equals(second.timestamp)
|
||||
&& first.originReplica.id().compareTo(second.originReplica.id()) < 0);
|
||||
}
|
||||
|
||||
AuctionState addFinishedAtReplica(String replica) {
|
||||
Set<String> s = new HashSet<>(finishedAtDc);
|
||||
s.add(replica);
|
||||
return new AuctionState(
|
||||
false, highestBid, highestCounterOffer, Collections.unmodifiableSet(s));
|
||||
}
|
||||
|
||||
public AuctionState close() {
|
||||
return new AuctionState(false, highestBid, highestCounterOffer, Collections.emptySet());
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return !stillRunning && finishedAtDc.isEmpty();
|
||||
}
|
||||
}
|
||||
// #state
|
||||
|
||||
// #setup
|
||||
public static Behavior<Command> create(
|
||||
ReplicaId replica,
|
||||
String name,
|
||||
Bid initialBid,
|
||||
Instant closingAt,
|
||||
boolean responsibleForClosing) {
|
||||
return Behaviors.setup(
|
||||
ctx ->
|
||||
Behaviors.withTimers(
|
||||
timers ->
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("Auction", name, replica),
|
||||
ALL_REPLICAS,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
replicationCtx ->
|
||||
new AuctionEntity(
|
||||
ctx,
|
||||
replicationCtx,
|
||||
timers,
|
||||
initialBid,
|
||||
closingAt,
|
||||
responsibleForClosing))));
|
||||
}
|
||||
|
||||
private AuctionEntity(
|
||||
ActorContext<Command> context,
|
||||
ReplicationContext replicationContext,
|
||||
TimerScheduler<Command> timers,
|
||||
Bid initialBid,
|
||||
Instant closingAt,
|
||||
boolean responsibleForClosing) {
|
||||
super(replicationContext);
|
||||
this.context = context;
|
||||
this.timers = timers;
|
||||
this.initialBid = initialBid;
|
||||
this.closingAt = closingAt;
|
||||
this.responsibleForClosing = responsibleForClosing;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AuctionState emptyState() {
|
||||
return new AuctionState(true, initialBid, initialBid.offer, Collections.emptySet());
|
||||
}
|
||||
|
||||
@Override
|
||||
public SignalHandler<AuctionState> signalHandler() {
|
||||
return newSignalHandlerBuilder()
|
||||
.onSignal(RecoveryCompleted.instance(), this::onRecoveryCompleted)
|
||||
.build();
|
||||
}
|
||||
|
||||
private void onRecoveryCompleted(AuctionState state) {
|
||||
if (shouldClose(state)) {
|
||||
context.getSelf().tell(Close.INSTANCE);
|
||||
}
|
||||
|
||||
long millisUntilClosing =
|
||||
closingAt.toEpochMilli() - getReplicationContext().currentTimeMillis();
|
||||
timers.startSingleTimer(Finish.INSTANCE, Duration.ofMillis(millisUntilClosing));
|
||||
}
|
||||
|
||||
// #setup
|
||||
|
||||
// #command-handler
|
||||
@Override
|
||||
public CommandHandler<Command, Event, AuctionState> commandHandler() {
|
||||
|
||||
CommandHandlerBuilder<Command, Event, AuctionState> builder = newCommandHandlerBuilder();
|
||||
|
||||
// running
|
||||
builder
|
||||
.forState(state -> state.stillRunning)
|
||||
.onCommand(
|
||||
OfferBid.class,
|
||||
(state, bid) ->
|
||||
Effect()
|
||||
.persist(
|
||||
new BidRegistered(
|
||||
new Bid(
|
||||
bid.bidder,
|
||||
bid.offer,
|
||||
Instant.ofEpochMilli(
|
||||
this.getReplicationContext().currentTimeMillis()),
|
||||
this.getReplicationContext().replicaId()))))
|
||||
.onCommand(
|
||||
GetHighestBid.class,
|
||||
(state, get) -> {
|
||||
get.replyTo.tell(state.highestBid);
|
||||
return Effect().none();
|
||||
})
|
||||
.onCommand(
|
||||
Finish.class,
|
||||
(state, finish) ->
|
||||
Effect().persist(new AuctionFinished(getReplicationContext().replicaId())))
|
||||
.onCommand(Close.class, (state, close) -> Effect().unhandled())
|
||||
.onCommand(
|
||||
IsClosed.class,
|
||||
(state, get) -> {
|
||||
get.replyTo.tell(false);
|
||||
return Effect().none();
|
||||
});
|
||||
|
||||
// finished
|
||||
builder
|
||||
.forAnyState()
|
||||
.onCommand(OfferBid.class, (state, bid) -> Effect().unhandled())
|
||||
.onCommand(
|
||||
GetHighestBid.class,
|
||||
(state, get) -> {
|
||||
get.replyTo.tell(state.highestBid);
|
||||
return Effect().none();
|
||||
})
|
||||
.onCommand(
|
||||
Finish.class,
|
||||
(state, finish) ->
|
||||
Effect().persist(new AuctionFinished(getReplicationContext().replicaId())))
|
||||
.onCommand(
|
||||
Close.class,
|
||||
(state, close) ->
|
||||
Effect()
|
||||
.persist(
|
||||
new WinnerDecided(
|
||||
getReplicationContext().replicaId(),
|
||||
state.highestBid,
|
||||
state.highestCounterOffer)))
|
||||
.onCommand(
|
||||
IsClosed.class,
|
||||
(state, get) -> {
|
||||
get.replyTo.tell(state.isClosed());
|
||||
return Effect().none();
|
||||
});
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
// #command-handler
|
||||
|
||||
// #event-handler
|
||||
@Override
|
||||
public EventHandler<AuctionState, Event> eventHandler() {
|
||||
return newEventHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onEvent(
|
||||
BidRegistered.class,
|
||||
(state, event) -> {
|
||||
if (AuctionState.isHigherBid(event.bid, state.highestBid)) {
|
||||
return state.withNewHighestBid(event.bid);
|
||||
} else {
|
||||
return state.withTooLowBid(event.bid);
|
||||
}
|
||||
})
|
||||
.onEvent(
|
||||
AuctionFinished.class,
|
||||
(state, event) -> {
|
||||
AuctionState newState = state.addFinishedAtReplica(event.atReplica.id());
|
||||
if (state.isClosed()) return state; // already closed
|
||||
else if (!getReplicationContext().recoveryRunning()) {
|
||||
eventTriggers(event, newState);
|
||||
}
|
||||
return newState;
|
||||
})
|
||||
.onEvent(WinnerDecided.class, (state, event) -> state.close())
|
||||
.build();
|
||||
}
|
||||
// #event-handler
|
||||
|
||||
// #event-triggers
|
||||
private void eventTriggers(AuctionFinished event, AuctionState newState) {
|
||||
if (newState.finishedAtDc.contains(getReplicationContext().replicaId().id())) {
|
||||
if (shouldClose(newState)) {
|
||||
context.getSelf().tell(Close.INSTANCE);
|
||||
}
|
||||
} else {
|
||||
context.getSelf().tell(Finish.INSTANCE);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean shouldClose(AuctionState state) {
|
||||
return responsibleForClosing
|
||||
&& !state.isClosed()
|
||||
&& getReplicationContext().getAllReplicas().stream()
|
||||
.map(ReplicaId::id)
|
||||
.collect(Collectors.toSet())
|
||||
.equals(state.finishedAtDc);
|
||||
}
|
||||
// #event-triggers
|
||||
|
||||
// #setup
|
||||
}
|
||||
// #setup
|
||||
|
|
@ -0,0 +1,291 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.persistence.typed;
|
||||
|
||||
import akka.Done;
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.actor.typed.javadsl.ActorContext;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.crdt.LwwTime;
|
||||
import akka.persistence.typed.javadsl.CommandHandler;
|
||||
import akka.persistence.typed.javadsl.Effect;
|
||||
import akka.persistence.typed.javadsl.EventHandler;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcing;
|
||||
import akka.persistence.typed.javadsl.ReplicationContext;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
interface ReplicatedBlogExample {
|
||||
|
||||
public final class BlogEntity
|
||||
extends ReplicatedEventSourcedBehavior<
|
||||
BlogEntity.Command, BlogEntity.Event, BlogEntity.BlogState> {
|
||||
|
||||
private final ActorContext<Command> context;
|
||||
|
||||
interface Command {
|
||||
String getPostId();
|
||||
}
|
||||
|
||||
static final class AddPost implements Command {
|
||||
final String postId;
|
||||
final PostContent content;
|
||||
final ActorRef<AddPostDone> replyTo;
|
||||
|
||||
public AddPost(String postId, PostContent content, ActorRef<AddPostDone> replyTo) {
|
||||
this.postId = postId;
|
||||
this.content = content;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
|
||||
public String getPostId() {
|
||||
return postId;
|
||||
}
|
||||
}
|
||||
|
||||
static final class AddPostDone {
|
||||
final String postId;
|
||||
|
||||
AddPostDone(String postId) {
|
||||
this.postId = postId;
|
||||
}
|
||||
|
||||
public String getPostId() {
|
||||
return postId;
|
||||
}
|
||||
}
|
||||
|
||||
static final class GetPost implements Command {
|
||||
final String postId;
|
||||
final ActorRef<PostContent> replyTo;
|
||||
|
||||
public GetPost(String postId, ActorRef<PostContent> replyTo) {
|
||||
this.postId = postId;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
|
||||
public String getPostId() {
|
||||
return postId;
|
||||
}
|
||||
}
|
||||
|
||||
static final class ChangeBody implements Command {
|
||||
final String postId;
|
||||
final PostContent newContent;
|
||||
final ActorRef<Done> replyTo;
|
||||
|
||||
public ChangeBody(String postId, PostContent newContent, ActorRef<Done> replyTo) {
|
||||
this.postId = postId;
|
||||
this.newContent = newContent;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
|
||||
public String getPostId() {
|
||||
return postId;
|
||||
}
|
||||
}
|
||||
|
||||
static final class Publish implements Command {
|
||||
final String postId;
|
||||
final ActorRef<Done> replyTo;
|
||||
|
||||
public Publish(String postId, ActorRef<Done> replyTo) {
|
||||
this.postId = postId;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
|
||||
public String getPostId() {
|
||||
return postId;
|
||||
}
|
||||
}
|
||||
|
||||
interface Event {}
|
||||
|
||||
static final class PostAdded implements Event {
|
||||
final String postId;
|
||||
final PostContent content;
|
||||
final LwwTime timestamp;
|
||||
|
||||
public PostAdded(String postId, PostContent content, LwwTime timestamp) {
|
||||
this.postId = postId;
|
||||
this.content = content;
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
}
|
||||
|
||||
static final class BodyChanged implements Event {
|
||||
final String postId;
|
||||
final PostContent content;
|
||||
final LwwTime timestamp;
|
||||
|
||||
public BodyChanged(String postId, PostContent content, LwwTime timestamp) {
|
||||
this.postId = postId;
|
||||
this.content = content;
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
}
|
||||
|
||||
static final class Published implements Event {
|
||||
final String postId;
|
||||
|
||||
public Published(String postId) {
|
||||
this.postId = postId;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class PostContent {
|
||||
final String title;
|
||||
final String body;
|
||||
|
||||
public PostContent(String title, String body) {
|
||||
this.title = title;
|
||||
this.body = body;
|
||||
}
|
||||
}
|
||||
|
||||
public static class BlogState {
|
||||
|
||||
public static final BlogState EMPTY =
|
||||
new BlogState(Optional.empty(), new LwwTime(Long.MIN_VALUE, new ReplicaId("")), false);
|
||||
|
||||
final Optional<PostContent> content;
|
||||
final LwwTime contentTimestamp;
|
||||
final boolean published;
|
||||
|
||||
public BlogState(Optional<PostContent> content, LwwTime contentTimestamp, boolean published) {
|
||||
this.content = content;
|
||||
this.contentTimestamp = contentTimestamp;
|
||||
this.published = published;
|
||||
}
|
||||
|
||||
BlogState withContent(PostContent newContent, LwwTime timestamp) {
|
||||
return new BlogState(Optional.of(newContent), timestamp, this.published);
|
||||
}
|
||||
|
||||
BlogState publish() {
|
||||
if (published) {
|
||||
return this;
|
||||
} else {
|
||||
return new BlogState(content, contentTimestamp, true);
|
||||
}
|
||||
}
|
||||
|
||||
boolean isEmpty() {
|
||||
return !content.isPresent();
|
||||
}
|
||||
}
|
||||
|
||||
public static Behavior<Command> create(
|
||||
String entityId, ReplicaId replicaId, Set<ReplicaId> allReplicas) {
|
||||
return Behaviors.setup(
|
||||
context ->
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("blog", entityId, replicaId),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
replicationContext -> new BlogEntity(context, replicationContext)));
|
||||
}
|
||||
|
||||
private BlogEntity(ActorContext<Command> context, ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlogState emptyState() {
|
||||
return BlogState.EMPTY;
|
||||
}
|
||||
|
||||
// #command-handler
|
||||
@Override
|
||||
public CommandHandler<Command, Event, BlogState> commandHandler() {
|
||||
return newCommandHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onCommand(AddPost.class, this::onAddPost)
|
||||
.onCommand(ChangeBody.class, this::onChangeBody)
|
||||
.onCommand(Publish.class, this::onPublish)
|
||||
.onCommand(GetPost.class, this::onGetPost)
|
||||
.build();
|
||||
}
|
||||
|
||||
private Effect<Event, BlogState> onAddPost(BlogState state, AddPost command) {
|
||||
PostAdded evt =
|
||||
new PostAdded(
|
||||
getReplicationContext().entityId(),
|
||||
command.content,
|
||||
state.contentTimestamp.increase(
|
||||
getReplicationContext().currentTimeMillis(),
|
||||
getReplicationContext().replicaId()));
|
||||
return Effect()
|
||||
.persist(evt)
|
||||
.thenRun(() -> command.replyTo.tell(new AddPostDone(getReplicationContext().entityId())));
|
||||
}
|
||||
|
||||
private Effect<Event, BlogState> onChangeBody(BlogState state, ChangeBody command) {
|
||||
BodyChanged evt =
|
||||
new BodyChanged(
|
||||
getReplicationContext().entityId(),
|
||||
command.newContent,
|
||||
state.contentTimestamp.increase(
|
||||
getReplicationContext().currentTimeMillis(),
|
||||
getReplicationContext().replicaId()));
|
||||
return Effect().persist(evt).thenRun(() -> command.replyTo.tell(Done.getInstance()));
|
||||
}
|
||||
|
||||
private Effect<Event, BlogState> onPublish(BlogState state, Publish command) {
|
||||
Published evt = new Published(getReplicationContext().entityId());
|
||||
return Effect().persist(evt).thenRun(() -> command.replyTo.tell(Done.getInstance()));
|
||||
}
|
||||
|
||||
private Effect<Event, BlogState> onGetPost(BlogState state, GetPost command) {
|
||||
context.getLog().info("GetPost {}", state.content);
|
||||
if (state.content.isPresent()) command.replyTo.tell(state.content.get());
|
||||
return Effect().none();
|
||||
}
|
||||
// #command-handler
|
||||
|
||||
// #event-handler
|
||||
@Override
|
||||
public EventHandler<BlogState, Event> eventHandler() {
|
||||
return newEventHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onEvent(PostAdded.class, this::onPostAdded)
|
||||
.onEvent(BodyChanged.class, this::onBodyChanged)
|
||||
.onEvent(Published.class, this::onPublished)
|
||||
.build();
|
||||
}
|
||||
|
||||
private BlogState onPostAdded(BlogState state, PostAdded event) {
|
||||
if (event.timestamp.isAfter(state.contentTimestamp)) {
|
||||
BlogState s = state.withContent(event.content, event.timestamp);
|
||||
context.getLog().info("Updating content. New content is {}", s);
|
||||
return s;
|
||||
} else {
|
||||
context.getLog().info("Ignoring event as timestamp is older");
|
||||
return state;
|
||||
}
|
||||
}
|
||||
|
||||
private BlogState onBodyChanged(BlogState state, BodyChanged event) {
|
||||
if (event.timestamp.isAfter(state.contentTimestamp)) {
|
||||
return state.withContent(event.content, event.timestamp);
|
||||
} else {
|
||||
return state;
|
||||
}
|
||||
}
|
||||
|
||||
private BlogState onPublished(BlogState state, Published event) {
|
||||
return state.publish();
|
||||
}
|
||||
// #event-handler
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,104 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.persistence.typed;
|
||||
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.crdt.ORSet;
|
||||
import akka.persistence.typed.javadsl.CommandHandler;
|
||||
import akka.persistence.typed.javadsl.EventHandler;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcing;
|
||||
import akka.persistence.typed.javadsl.ReplicationContext;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
interface ReplicatedMovieExample {
|
||||
|
||||
// #movie-entity
|
||||
public final class MovieWatchList
|
||||
extends ReplicatedEventSourcedBehavior<MovieWatchList.Command, ORSet.DeltaOp, ORSet<String>> {
|
||||
|
||||
interface Command {}
|
||||
|
||||
public static class AddMovie implements Command {
|
||||
public final String movieId;
|
||||
|
||||
public AddMovie(String movieId) {
|
||||
this.movieId = movieId;
|
||||
}
|
||||
}
|
||||
|
||||
public static class RemoveMovie implements Command {
|
||||
public final String movieId;
|
||||
|
||||
public RemoveMovie(String movieId) {
|
||||
this.movieId = movieId;
|
||||
}
|
||||
}
|
||||
|
||||
public static class GetMovieList implements Command {
|
||||
public final ActorRef<MovieList> replyTo;
|
||||
|
||||
public GetMovieList(ActorRef<MovieList> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
public static class MovieList {
|
||||
public final Set<String> movieIds;
|
||||
|
||||
public MovieList(Set<String> movieIds) {
|
||||
this.movieIds = Collections.unmodifiableSet(movieIds);
|
||||
}
|
||||
}
|
||||
|
||||
public static Behavior<Command> create(
|
||||
String entityId, ReplicaId replicaId, Set<ReplicaId> allReplicas) {
|
||||
return ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("movies", entityId, replicaId),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
MovieWatchList::new);
|
||||
}
|
||||
|
||||
private MovieWatchList(ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ORSet<String> emptyState() {
|
||||
return ORSet.empty(getReplicationContext().replicaId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandHandler<Command, ORSet.DeltaOp, ORSet<String>> commandHandler() {
|
||||
return newCommandHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onCommand(
|
||||
AddMovie.class, (state, command) -> Effect().persist(state.add(command.movieId)))
|
||||
.onCommand(
|
||||
RemoveMovie.class,
|
||||
(state, command) -> Effect().persist(state.remove(command.movieId)))
|
||||
.onCommand(
|
||||
GetMovieList.class,
|
||||
(state, command) -> {
|
||||
command.replyTo.tell(new MovieList(state.getElements()));
|
||||
return Effect().none();
|
||||
})
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler<ORSet<String>, ORSet.DeltaOp> eventHandler() {
|
||||
return newEventHandlerBuilder().forAnyState().onAnyEvent(ORSet::applyOperation);
|
||||
}
|
||||
}
|
||||
// #movie-entity
|
||||
}
|
||||
|
|
@ -0,0 +1,158 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.persistence.typed;
|
||||
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.crdt.Counter;
|
||||
import akka.persistence.typed.javadsl.CommandHandler;
|
||||
import akka.persistence.typed.javadsl.Effect;
|
||||
import akka.persistence.typed.javadsl.EventHandler;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior;
|
||||
import akka.persistence.typed.javadsl.ReplicatedEventSourcing;
|
||||
import akka.persistence.typed.javadsl.ReplicationContext;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
interface ReplicatedShoppingCartExample {
|
||||
|
||||
// #shopping-cart
|
||||
public final class ShoppingCart
|
||||
extends ReplicatedEventSourcedBehavior<
|
||||
ShoppingCart.Command, ShoppingCart.Event, ShoppingCart.State> {
|
||||
|
||||
public interface Event {}
|
||||
|
||||
public static final class ItemUpdated implements Event {
|
||||
public final String productId;
|
||||
public final Counter.Updated update;
|
||||
|
||||
public ItemUpdated(String productId, Counter.Updated update) {
|
||||
this.productId = productId;
|
||||
this.update = update;
|
||||
}
|
||||
}
|
||||
|
||||
public interface Command {}
|
||||
|
||||
public static final class AddItem implements Command {
|
||||
public final String productId;
|
||||
public final int count;
|
||||
|
||||
public AddItem(String productId, int count) {
|
||||
this.productId = productId;
|
||||
this.count = count;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class RemoveItem implements Command {
|
||||
public final String productId;
|
||||
public final int count;
|
||||
|
||||
public RemoveItem(String productId, int count) {
|
||||
this.productId = productId;
|
||||
this.count = count;
|
||||
}
|
||||
}
|
||||
|
||||
public static class GetCartItems implements Command {
|
||||
public final ActorRef<CartItems> replyTo;
|
||||
|
||||
public GetCartItems(ActorRef<CartItems> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class CartItems {
|
||||
public final Map<String, Integer> items;
|
||||
|
||||
public CartItems(Map<String, Integer> items) {
|
||||
this.items = items;
|
||||
}
|
||||
}
|
||||
|
||||
public static final class State {
|
||||
public final Map<String, Counter> items = new HashMap<>();
|
||||
}
|
||||
|
||||
public static Behavior<Command> create(
|
||||
String entityId, ReplicaId replicaId, Set<ReplicaId> allReplicas) {
|
||||
return ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("blog", entityId, replicaId),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
ShoppingCart::new);
|
||||
}
|
||||
|
||||
private ShoppingCart(ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public State emptyState() {
|
||||
return new State();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandHandler<Command, Event, State> commandHandler() {
|
||||
return newCommandHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onCommand(AddItem.class, this::onAddItem)
|
||||
.onCommand(RemoveItem.class, this::onRemoveItem)
|
||||
.onCommand(GetCartItems.class, this::onGetCartItems)
|
||||
.build();
|
||||
}
|
||||
|
||||
private Effect<Event, State> onAddItem(State state, AddItem command) {
|
||||
return Effect()
|
||||
.persist(new ItemUpdated(command.productId, new Counter.Updated(command.count)));
|
||||
}
|
||||
|
||||
private Effect<Event, State> onRemoveItem(State state, RemoveItem command) {
|
||||
return Effect()
|
||||
.persist(new ItemUpdated(command.productId, new Counter.Updated(-command.count)));
|
||||
}
|
||||
|
||||
private Effect<Event, State> onGetCartItems(State state, GetCartItems command) {
|
||||
command.replyTo.tell(new CartItems(filterEmptyAndNegative(state.items)));
|
||||
return Effect().none();
|
||||
}
|
||||
|
||||
private Map<String, Integer> filterEmptyAndNegative(Map<String, Counter> cart) {
|
||||
Map<String, Integer> result = new HashMap<>();
|
||||
for (Map.Entry<String, Counter> entry : cart.entrySet()) {
|
||||
int count = entry.getValue().value().intValue();
|
||||
if (count > 0) result.put(entry.getKey(), count);
|
||||
}
|
||||
return Collections.unmodifiableMap(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler<State, Event> eventHandler() {
|
||||
return newEventHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onEvent(ItemUpdated.class, this::onItemUpdated)
|
||||
.build();
|
||||
}
|
||||
|
||||
private State onItemUpdated(State state, ItemUpdated event) {
|
||||
final Counter counterForProduct;
|
||||
if (state.items.containsKey(event.productId)) {
|
||||
counterForProduct = state.items.get(event.productId);
|
||||
} else {
|
||||
counterForProduct = Counter.empty();
|
||||
}
|
||||
state.items.put(event.productId, counterForProduct.applyOperation(event.update));
|
||||
return state;
|
||||
}
|
||||
}
|
||||
// #shopping-cart
|
||||
}
|
||||
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.akka.persistence.typed;
|
||||
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal;
|
||||
import akka.persistence.typed.ReplicaId;
|
||||
import akka.persistence.typed.ReplicationId;
|
||||
import akka.persistence.typed.javadsl.*;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
public final class ReplicatedStringSet
|
||||
extends ReplicatedEventSourcedBehavior<ReplicatedStringSet.Command, String, Set<String>> {
|
||||
interface Command {}
|
||||
|
||||
public static final class AddString implements Command {
|
||||
final String string;
|
||||
|
||||
public AddString(String string) {
|
||||
this.string = string;
|
||||
}
|
||||
}
|
||||
|
||||
public static Behavior<Command> create(
|
||||
String entityId, ReplicaId replicaId, Set<ReplicaId> allReplicas) {
|
||||
return ReplicatedEventSourcing.commonJournalConfig(
|
||||
new ReplicationId("StringSet", entityId, replicaId),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier(),
|
||||
ReplicatedStringSet::new);
|
||||
}
|
||||
|
||||
private ReplicatedStringSet(ReplicationContext replicationContext) {
|
||||
super(replicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> emptyState() {
|
||||
return new HashSet<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandHandler<Command, String, Set<String>> commandHandler() {
|
||||
return newCommandHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onCommand(
|
||||
AddString.class,
|
||||
(state, cmd) -> {
|
||||
if (!state.contains(cmd.string)) return Effect().persist(cmd.string);
|
||||
else return Effect().none();
|
||||
})
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler<Set<String>, String> eventHandler() {
|
||||
return newEventHandlerBuilder()
|
||||
.forAnyState()
|
||||
.onAnyEvent(
|
||||
(set, string) -> {
|
||||
HashSet<String> newState = new HashSet<>(set);
|
||||
newState.add(string);
|
||||
return newState;
|
||||
});
|
||||
}
|
||||
|
||||
// #tagging
|
||||
@Override
|
||||
public Set<String> tagsFor(String event) {
|
||||
// don't apply tags if event was replicated here, it already will appear in queries by tag
|
||||
// as the origin replica would have tagged it already
|
||||
if (getReplicationContext().replicaId() != getReplicationContext().origin()) {
|
||||
return new HashSet<>();
|
||||
} else {
|
||||
Set<String> tags = new HashSet<>();
|
||||
tags.add("strings");
|
||||
if (event.length() > 10) tags.add("long-strings");
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
// #tagging
|
||||
}
|
||||
|
|
@ -0,0 +1,32 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<configuration>
|
||||
<!-- Silence initial setup logging from Logback -->
|
||||
<statusListener class="ch.qos.logback.core.status.NopStatusListener" />
|
||||
|
||||
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
|
||||
<encoder>
|
||||
<pattern>%date{ISO8601} [%-5level] [%logger] [%marker] [%X{akkaSource}] [%X{persistencePhase}] [%X{persistenceId}] - %msg %n</pattern>
|
||||
</encoder>
|
||||
</appender>
|
||||
|
||||
<!--
|
||||
Logging from tests are silenced by this appender. When there is a test failure
|
||||
the captured logging events are flushed to the appenders defined for the
|
||||
akka.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
|
||||
-->
|
||||
<appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
|
||||
|
||||
<!--
|
||||
The appenders defined for this CapturingAppenderDelegate logger are used
|
||||
when there is a test failure and all logging events from the test are
|
||||
flushed to these appenders.
|
||||
-->
|
||||
<logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
|
||||
<appender-ref ref="STDOUT"/>
|
||||
</logger>
|
||||
|
||||
<root level="TRACE">
|
||||
<appender-ref ref="CapturingAppender"/>
|
||||
</root>
|
||||
|
||||
</configuration>
|
||||
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.eventstream.EventStream
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object EventPublishingSpec {
|
||||
|
||||
object WowSuchEventSourcingBehavior {
|
||||
sealed trait Command
|
||||
case class StoreThis(data: String, tagIt: Boolean, replyTo: ActorRef[Done]) extends Command
|
||||
|
||||
final case class Event(data: String, tagIt: Boolean) extends CborSerializable
|
||||
|
||||
def apply(id: PersistenceId): Behavior[Command] =
|
||||
EventSourcedBehavior[Command, Event, Set[Event]](
|
||||
id,
|
||||
Set.empty,
|
||||
(_, command) =>
|
||||
command match {
|
||||
case StoreThis(data, tagIt, replyTo) =>
|
||||
Effect.persist(Event(data, tagIt)).thenRun(_ => replyTo ! Done)
|
||||
},
|
||||
(state, event) => state + event)
|
||||
.withTagger(evt => if (evt.tagIt) Set("tag") else Set.empty)
|
||||
.withEventPublishing(enabled = true)
|
||||
}
|
||||
}
|
||||
|
||||
class EventPublishingSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
|
||||
import EventPublishingSpec._
|
||||
|
||||
"EventPublishing support" must {
|
||||
|
||||
"publish events after written for any actor" in {
|
||||
val topicProbe = createTestProbe[PublishedEvent]()
|
||||
system.eventStream ! EventStream.Subscribe(topicProbe.ref)
|
||||
// We don't verify subscription completed (no ack available), but expect the next steps to take enough time
|
||||
// for subscription to complete
|
||||
|
||||
val myId = PersistenceId.ofUniqueId("myId")
|
||||
val wowSuchActor = spawn(WowSuchEventSourcingBehavior(myId))
|
||||
|
||||
val persistProbe = createTestProbe[Any]()
|
||||
wowSuchActor ! WowSuchEventSourcingBehavior.StoreThis("great stuff", tagIt = false, replyTo = persistProbe.ref)
|
||||
persistProbe.expectMessage(Done)
|
||||
|
||||
val published1 = topicProbe.receiveMessage()
|
||||
published1.persistenceId should ===(myId)
|
||||
published1.event should ===(WowSuchEventSourcingBehavior.Event("great stuff", false))
|
||||
published1.sequenceNumber should ===(1L)
|
||||
published1.tags should ===(Set.empty)
|
||||
|
||||
val anotherId = PersistenceId.ofUniqueId("anotherId")
|
||||
val anotherActor = spawn(WowSuchEventSourcingBehavior(anotherId))
|
||||
anotherActor ! WowSuchEventSourcingBehavior.StoreThis("another event", tagIt = true, replyTo = persistProbe.ref)
|
||||
persistProbe.expectMessage(Done)
|
||||
|
||||
val published2 = topicProbe.receiveMessage()
|
||||
published2.persistenceId should ===(anotherId)
|
||||
published2.event should ===(WowSuchEventSourcingBehavior.Event("another event", true))
|
||||
published2.sequenceNumber should ===(1L)
|
||||
published2.tags should ===(Set("tag"))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.query.PersistenceQuery
|
||||
import akka.persistence.query.scaladsl.CurrentEventsByPersistenceIdQuery
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.stream.scaladsl.Sink
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object MultiJournalReplicationSpec {
|
||||
|
||||
object Actor {
|
||||
sealed trait Command
|
||||
case class GetState(replyTo: ActorRef[Set[String]]) extends Command
|
||||
case class StoreMe(text: String, ack: ActorRef[Done]) extends Command
|
||||
|
||||
private val writeJournalPerReplica = Map("R1" -> "journal1.journal", "R2" -> "journal2.journal")
|
||||
def apply(entityId: String, replicaId: String): Behavior[Command] = {
|
||||
ReplicatedEventSourcing
|
||||
.perReplicaJournalConfig(
|
||||
ReplicationId("MultiJournalSpec", entityId, ReplicaId(replicaId)),
|
||||
Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))(replicationContext =>
|
||||
EventSourcedBehavior[Command, String, Set[String]](
|
||||
replicationContext.persistenceId,
|
||||
Set.empty[String],
|
||||
(state, command) =>
|
||||
command match {
|
||||
case GetState(replyTo) =>
|
||||
replyTo ! state
|
||||
Effect.none
|
||||
case StoreMe(evt, ack) =>
|
||||
Effect.persist(evt).thenRun(_ => ack ! Done)
|
||||
},
|
||||
(state, event) => state + event))
|
||||
.withJournalPluginId(writeJournalPerReplica(replicaId))
|
||||
}
|
||||
}
|
||||
|
||||
def separateJournalsConfig: Config = ConfigFactory.parseString(s"""
|
||||
journal1 {
|
||||
journal.class = "${classOf[PersistenceTestKitPlugin].getName}"
|
||||
query = $${akka.persistence.testkit.query}
|
||||
}
|
||||
journal2 {
|
||||
journal.class = "${classOf[PersistenceTestKitPlugin].getName}"
|
||||
query = $${akka.persistence.testkit.query}
|
||||
}
|
||||
""").withFallback(ConfigFactory.load()).resolve()
|
||||
|
||||
}
|
||||
|
||||
class MultiJournalReplicationSpec
|
||||
extends ScalaTestWithActorTestKit(MultiJournalReplicationSpec.separateJournalsConfig)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing
|
||||
with Eventually {
|
||||
import MultiJournalReplicationSpec._
|
||||
val ids = new AtomicInteger(0)
|
||||
def nextEntityId = s"e-${ids.getAndIncrement()}"
|
||||
"ReplicatedEventSourcing" should {
|
||||
"support one journal per replica" in {
|
||||
|
||||
val r1 = spawn(Actor("id1", "R1"))
|
||||
val r2 = spawn(Actor("id1", "R2"))
|
||||
|
||||
val probe = createTestProbe[Any]()
|
||||
r1 ! Actor.StoreMe("r1 m1", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
r2 ! Actor.StoreMe("r2 m1", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
eventually {
|
||||
val probe = createTestProbe[Set[String]]()
|
||||
r1 ! Actor.GetState(probe.ref)
|
||||
probe.receiveMessage() should ===(Set("r1 m1", "r2 m1"))
|
||||
|
||||
r2 ! Actor.GetState(probe.ref)
|
||||
probe.receiveMessage() should ===(Set("r1 m1", "r2 m1"))
|
||||
}
|
||||
|
||||
val readJournal1 = PersistenceQuery(system).readJournalFor[CurrentEventsByPersistenceIdQuery]("journal1.query")
|
||||
val readJournal2 = PersistenceQuery(system).readJournalFor[CurrentEventsByPersistenceIdQuery]("journal2.query")
|
||||
|
||||
val eventsForJournal1 =
|
||||
readJournal1
|
||||
.currentEventsByPersistenceId("MultiJournalSpec|id1|R1", 0L, Long.MaxValue)
|
||||
.runWith(Sink.seq)
|
||||
.futureValue
|
||||
eventsForJournal1.map(_.event).toSet should ===(Set("r1 m1", "r2 m1"))
|
||||
|
||||
val eventsForJournal2 =
|
||||
readJournal2
|
||||
.currentEventsByPersistenceId("MultiJournalSpec|id1|R2", 0L, Long.MaxValue)
|
||||
.runWith(Sink.seq)
|
||||
.futureValue
|
||||
eventsForJournal2.map(_.event).toSet should ===(Set("r1 m1", "r2 m1"))
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,240 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.internal.{ ReplicatedPublishedEventMetaData, VersionVector }
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicatedEventPublishingSpec {
|
||||
|
||||
val EntityType = "EventPublishingSpec"
|
||||
|
||||
object MyReplicatedBehavior {
|
||||
trait Command
|
||||
case class Add(text: String, replyTo: ActorRef[Done]) extends Command
|
||||
case class Get(replyTo: ActorRef[Set[String]]) extends Command
|
||||
case object Stop extends Command
|
||||
|
||||
def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] =
|
||||
Behaviors.setup { ctx =>
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId(EntityType, entityId, replicaId),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier)(
|
||||
replicationContext =>
|
||||
EventSourcedBehavior[Command, String, Set[String]](
|
||||
replicationContext.persistenceId,
|
||||
Set.empty,
|
||||
(state, command) =>
|
||||
command match {
|
||||
case Add(string, replyTo) =>
|
||||
ctx.log.debug("Persisting [{}]", string)
|
||||
Effect.persist(string).thenRun { _ =>
|
||||
ctx.log.debug("Ack:ing [{}]", string)
|
||||
replyTo ! Done
|
||||
}
|
||||
case Get(replyTo) =>
|
||||
replyTo ! state
|
||||
Effect.none
|
||||
case Stop =>
|
||||
Effect.stop()
|
||||
},
|
||||
(state, string) => state + string))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicatedEventPublishingSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
|
||||
val DCA = ReplicaId("DC-A")
|
||||
val DCB = ReplicaId("DC-B")
|
||||
val DCC = ReplicaId("DC-C")
|
||||
|
||||
private var idCounter = 0
|
||||
def nextEntityId(): String = {
|
||||
idCounter += 1
|
||||
s"myId$idCounter"
|
||||
}
|
||||
|
||||
import ReplicatedEventPublishingSpec._
|
||||
|
||||
"An Replicated Event Sourced actor" must {
|
||||
"move forward when a published event from a replica is received" in {
|
||||
val id = nextEntityId()
|
||||
val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB)))
|
||||
val probe = createTestProbe[Any]()
|
||||
actor ! MyReplicatedBehavior.Add("one", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// simulate a published event from another replica
|
||||
actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCB).persistenceId,
|
||||
1L,
|
||||
"two",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
actor ! MyReplicatedBehavior.Add("three", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
actor ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one", "two", "three"))
|
||||
}
|
||||
|
||||
"ignore a published event from a replica is received but the sequence number is unexpected" in {
|
||||
val id = nextEntityId()
|
||||
val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB)))
|
||||
val probe = createTestProbe[Any]()
|
||||
actor ! MyReplicatedBehavior.Add("one", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// simulate a published event from another replica
|
||||
actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCB).persistenceId,
|
||||
2L, // missing 1L
|
||||
"two",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
actor ! MyReplicatedBehavior.Add("three", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
actor ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one", "three"))
|
||||
}
|
||||
|
||||
"ignore a published event from an unknown replica" in {
|
||||
val id = nextEntityId()
|
||||
val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB)))
|
||||
val probe = createTestProbe[Any]()
|
||||
actor ! MyReplicatedBehavior.Add("one", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// simulate a published event from another replica
|
||||
actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCC).persistenceId,
|
||||
1L,
|
||||
"two",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCC, VersionVector.empty)))
|
||||
actor ! MyReplicatedBehavior.Add("three", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
actor ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one", "three"))
|
||||
}
|
||||
|
||||
"ignore an already seen event from a replica" in {
|
||||
val id = nextEntityId()
|
||||
val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB)))
|
||||
val probe = createTestProbe[Any]()
|
||||
actor ! MyReplicatedBehavior.Add("one", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// simulate a published event from another replica
|
||||
actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, "myId4", DCB).persistenceId,
|
||||
1L,
|
||||
"two",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
// simulate another published event from that replica
|
||||
actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCB).persistenceId,
|
||||
1L,
|
||||
"two-again", // ofc this would be the same in the real world, different just so we can detect
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
|
||||
actor ! MyReplicatedBehavior.Add("three", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
actor ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one", "two", "three"))
|
||||
}
|
||||
|
||||
"handle published events after replay" in {
|
||||
val id = nextEntityId()
|
||||
val probe = createTestProbe[Any]()
|
||||
val replicatedBehavior = MyReplicatedBehavior(id, DCA, Set(DCA, DCB))
|
||||
val incarnation1 = spawn(replicatedBehavior)
|
||||
incarnation1 ! MyReplicatedBehavior.Add("one", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
incarnation1 ! MyReplicatedBehavior.Stop
|
||||
probe.expectTerminated(incarnation1)
|
||||
|
||||
val incarnation2 = spawn(replicatedBehavior)
|
||||
|
||||
incarnation2 ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one"))
|
||||
// replay completed
|
||||
|
||||
// simulate a published event from another replica
|
||||
incarnation2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCB).persistenceId,
|
||||
1L,
|
||||
"two",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
|
||||
incarnation2 ! MyReplicatedBehavior.Add("three", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
incarnation2 ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one", "two", "three"))
|
||||
}
|
||||
|
||||
"handle published events before and after replay" in {
|
||||
val id = nextEntityId()
|
||||
val probe = createTestProbe[Any]()
|
||||
val replicatedBehaviorA = MyReplicatedBehavior(id, DCA, Set(DCA, DCB))
|
||||
val incarnationA1 = spawn(replicatedBehaviorA)
|
||||
incarnationA1 ! MyReplicatedBehavior.Add("one", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// simulate a published event from another replica
|
||||
incarnationA1.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCB).persistenceId,
|
||||
1L,
|
||||
"two",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
|
||||
incarnationA1 ! MyReplicatedBehavior.Stop
|
||||
probe.expectTerminated(incarnationA1)
|
||||
|
||||
val incarnationA2 = spawn(replicatedBehaviorA)
|
||||
|
||||
// simulate a published event from another replica
|
||||
incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, id, DCB).persistenceId,
|
||||
2L,
|
||||
"three",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty)))
|
||||
|
||||
incarnationA2 ! MyReplicatedBehavior.Add("four", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
incarnationA2 ! MyReplicatedBehavior.Get(probe.ref)
|
||||
probe.expectMessage(Set("one", "two", "three", "four"))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,429 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.testkit.scaladsl.PersistenceTestKit
|
||||
import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing, ReplicationContext }
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicatedEventSourcingSpec {
|
||||
|
||||
val AllReplicas = Set(ReplicaId("R1"), ReplicaId("R2"), ReplicaId("R3"))
|
||||
|
||||
sealed trait Command
|
||||
case class GetState(replyTo: ActorRef[State]) extends Command
|
||||
case class StoreMe(description: String, replyTo: ActorRef[Done]) extends Command
|
||||
case class StoreUs(descriptions: List[String], replyTo: ActorRef[Done]) extends Command
|
||||
case class GetReplica(replyTo: ActorRef[(ReplicaId, Set[ReplicaId])]) extends Command
|
||||
case object Stop extends Command
|
||||
|
||||
case class State(all: List[String]) extends CborSerializable
|
||||
|
||||
def testBehavior(entityId: String, replicaId: String, probe: ActorRef[EventAndContext]): Behavior[Command] =
|
||||
testBehavior(entityId, replicaId, Some(probe))
|
||||
|
||||
def eventSourcedBehavior(
|
||||
replicationContext: ReplicationContext,
|
||||
probe: Option[ActorRef[EventAndContext]]): EventSourcedBehavior[Command, String, State] = {
|
||||
EventSourcedBehavior[Command, String, State](
|
||||
replicationContext.persistenceId,
|
||||
State(Nil),
|
||||
(state, command) =>
|
||||
command match {
|
||||
case GetState(replyTo) =>
|
||||
replyTo ! state
|
||||
Effect.none
|
||||
case GetReplica(replyTo) =>
|
||||
replyTo.tell((replicationContext.replicaId, replicationContext.allReplicas))
|
||||
Effect.none
|
||||
case StoreMe(evt, ack) =>
|
||||
Effect.persist(evt).thenRun(_ => ack ! Done)
|
||||
case StoreUs(evts, replyTo) =>
|
||||
Effect.persist(evts).thenRun(_ => replyTo ! Done)
|
||||
case Stop =>
|
||||
Effect.stop()
|
||||
},
|
||||
(state, event) => {
|
||||
probe.foreach(
|
||||
_ ! EventAndContext(
|
||||
event,
|
||||
replicationContext.origin,
|
||||
replicationContext.recoveryRunning,
|
||||
replicationContext.concurrent))
|
||||
state.copy(all = event :: state.all)
|
||||
})
|
||||
}
|
||||
|
||||
def testBehavior(
|
||||
entityId: String,
|
||||
replicaId: String,
|
||||
probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] =
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("ReplicatedEventSourcingSpec", entityId, ReplicaId(replicaId)),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier)(replicationContext => eventSourcedBehavior(replicationContext, probe))
|
||||
|
||||
}
|
||||
|
||||
case class EventAndContext(event: Any, origin: ReplicaId, recoveryRunning: Boolean, concurrent: Boolean)
|
||||
|
||||
class ReplicatedEventSourcingSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing
|
||||
with Eventually {
|
||||
import ReplicatedEventSourcingSpec._
|
||||
val ids = new AtomicInteger(0)
|
||||
def nextEntityId = s"e-${ids.getAndIncrement()}"
|
||||
"ReplicatedEventSourcing" should {
|
||||
"replicate events between two entities" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1"))
|
||||
val r2 = spawn(testBehavior(entityId, "R2"))
|
||||
r1 ! StoreMe("from r1", probe.ref)
|
||||
r2 ! StoreMe("from r2", probe.ref)
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r1 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from r1", "from r2")
|
||||
}
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from r1", "from r2")
|
||||
}
|
||||
}
|
||||
"get all events in recovery" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1"))
|
||||
val r2 = spawn(testBehavior(entityId, "R2"))
|
||||
r1 ! StoreMe("from r1", probe.ref)
|
||||
r2 ! StoreMe("from r2", probe.ref)
|
||||
r1 ! StoreMe("from r1 again", probe.ref)
|
||||
|
||||
val r3 = spawn(testBehavior(entityId, "R3"))
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r3 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from r1", "from r2", "from r1 again")
|
||||
}
|
||||
}
|
||||
|
||||
"continue after recovery" in {
|
||||
val entityId = nextEntityId
|
||||
val r1Behavior = testBehavior(entityId, "R1")
|
||||
val r2Behavior = testBehavior(entityId, "R2")
|
||||
val probe = createTestProbe[Done]()
|
||||
|
||||
{
|
||||
// first incarnation
|
||||
val r1 = spawn(r1Behavior)
|
||||
val r2 = spawn(r2Behavior)
|
||||
r1 ! StoreMe("1 from r1", probe.ref)
|
||||
r2 ! StoreMe("1 from r2", probe.ref)
|
||||
r1 ! Stop
|
||||
r2 ! Stop
|
||||
probe.expectTerminated(r1)
|
||||
probe.expectTerminated(r2)
|
||||
}
|
||||
|
||||
{
|
||||
// second incarnation
|
||||
val r1 = spawn(r1Behavior)
|
||||
val r2 = spawn(r2Behavior)
|
||||
|
||||
r1 ! StoreMe("2 from r1", probe.ref)
|
||||
r2 ! StoreMe("2 from r2", probe.ref)
|
||||
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r1 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("1 from r1", "1 from r2", "2 from r1", "2 from r2")
|
||||
r2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("1 from r1", "1 from r2", "2 from r1", "2 from r2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"have access to replica information" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[(ReplicaId, Set[ReplicaId])]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1"))
|
||||
r1 ! GetReplica(probe.ref)
|
||||
probe.expectMessage((ReplicaId("R1"), Set(ReplicaId("R1"), ReplicaId("R2"), ReplicaId("R3"))))
|
||||
}
|
||||
|
||||
"have access to event origin" in {
|
||||
val entityId = nextEntityId
|
||||
val replyProbe = createTestProbe[Done]()
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
val eventProbeR2 = createTestProbe[EventAndContext]()
|
||||
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
val r2 = spawn(testBehavior(entityId, "R2", eventProbeR2.ref))
|
||||
|
||||
r1 ! StoreMe("from r1", replyProbe.ref)
|
||||
eventProbeR2.expectMessage(EventAndContext("from r1", ReplicaId("R1"), false, false))
|
||||
eventProbeR1.expectMessage(EventAndContext("from r1", ReplicaId("R1"), false, false))
|
||||
|
||||
r2 ! StoreMe("from r2", replyProbe.ref)
|
||||
eventProbeR1.expectMessage(EventAndContext("from r2", ReplicaId("R2"), false, false))
|
||||
eventProbeR2.expectMessage(EventAndContext("from r2", ReplicaId("R2"), false, false))
|
||||
}
|
||||
|
||||
"set recovery running" in {
|
||||
val entityId = nextEntityId
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
val replyProbe = createTestProbe[Done]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
r1 ! StoreMe("Event", replyProbe.ref)
|
||||
eventProbeR1.expectMessage(EventAndContext("Event", ReplicaId("R1"), recoveryRunning = false, false))
|
||||
replyProbe.expectMessage(Done)
|
||||
|
||||
val recoveryProbe = createTestProbe[EventAndContext]()
|
||||
spawn(testBehavior(entityId, "R1", recoveryProbe.ref))
|
||||
recoveryProbe.expectMessage(EventAndContext("Event", ReplicaId("R1"), recoveryRunning = true, false))
|
||||
}
|
||||
|
||||
"persist all" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
val r2 = spawn(testBehavior(entityId, "R2"))
|
||||
r1 ! StoreUs("1 from r1" :: "2 from r1" :: Nil, probe.ref)
|
||||
r2 ! StoreUs("1 from r2" :: "2 from r2" :: Nil, probe.ref)
|
||||
probe.receiveMessage()
|
||||
probe.receiveMessage()
|
||||
|
||||
// events at r2 happened concurrently with events at r1
|
||||
|
||||
eventProbeR1.expectMessage(EventAndContext("1 from r1", ReplicaId("R1"), false, concurrent = false))
|
||||
eventProbeR1.expectMessage(EventAndContext("2 from r1", ReplicaId("R1"), false, concurrent = false))
|
||||
eventProbeR1.expectMessage(EventAndContext("1 from r2", ReplicaId("R2"), false, concurrent = true))
|
||||
eventProbeR1.expectMessage(EventAndContext("2 from r2", ReplicaId("R2"), false, concurrent = true))
|
||||
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r1 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("1 from r1", "2 from r1", "1 from r2", "2 from r2")
|
||||
}
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("1 from r1", "2 from r1", "1 from r2", "2 from r2")
|
||||
}
|
||||
}
|
||||
|
||||
"replicate alternate events" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
val eventProbeR2 = createTestProbe[EventAndContext]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
val r2 = spawn(testBehavior(entityId, "R2", eventProbeR2.ref))
|
||||
r1 ! StoreMe("from r1", probe.ref) // R1 0 R2 0 -> R1 1 R2 0
|
||||
r2 ! StoreMe("from r2", probe.ref) // R2 0 R1 0 -> R2 1 R1 0
|
||||
|
||||
// each gets its local event
|
||||
eventProbeR1.expectMessage(
|
||||
EventAndContext("from r1", ReplicaId("R1"), recoveryRunning = false, concurrent = false))
|
||||
eventProbeR2.expectMessage(
|
||||
EventAndContext("from r2", ReplicaId("R2"), recoveryRunning = false, concurrent = false))
|
||||
|
||||
// then the replicated remote events, which will be concurrent
|
||||
eventProbeR1.expectMessage(
|
||||
EventAndContext("from r2", ReplicaId("R2"), recoveryRunning = false, concurrent = true))
|
||||
eventProbeR2.expectMessage(
|
||||
EventAndContext("from r1", ReplicaId("R1"), recoveryRunning = false, concurrent = true))
|
||||
|
||||
// state is updated
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r1 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from r1", "from r2")
|
||||
}
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from r1", "from r2")
|
||||
}
|
||||
|
||||
// Neither of these should be concurrent, nothing happening at r2
|
||||
r1 ! StoreMe("from r1 2", probe.ref) // R1 1 R2 1
|
||||
eventProbeR1.expectMessage(EventAndContext("from r1 2", ReplicaId("R1"), false, concurrent = false))
|
||||
eventProbeR2.expectMessage(EventAndContext("from r1 2", ReplicaId("R1"), false, concurrent = false))
|
||||
r1 ! StoreMe("from r1 3", probe.ref) // R2 2 R2 1
|
||||
eventProbeR1.expectMessage(EventAndContext("from r1 3", ReplicaId("R1"), false, concurrent = false))
|
||||
eventProbeR2.expectMessage(EventAndContext("from r1 3", ReplicaId("R1"), false, concurrent = false))
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r2 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("from r1", "from r2", "from r1 2", "from r1 3")
|
||||
}
|
||||
|
||||
// not concurrent as the above asserts mean that all events are fully replicated
|
||||
r2 ! StoreMe("from r2 2", probe.ref)
|
||||
eventProbeR1.expectMessage(EventAndContext("from r2 2", ReplicaId("R2"), false, concurrent = false))
|
||||
eventProbeR2.expectMessage(EventAndContext("from r2 2", ReplicaId("R2"), false, concurrent = false))
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
r1 ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set(
|
||||
"from r1",
|
||||
"from r2",
|
||||
"from r1 2",
|
||||
"from r1 3",
|
||||
"from r2 2")
|
||||
}
|
||||
}
|
||||
|
||||
"receive each event only once" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
val eventProbeR2 = createTestProbe[EventAndContext]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
val r2 = spawn(testBehavior(entityId, "R2", eventProbeR2.ref))
|
||||
r1 ! StoreMe("from r1 1", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
r1 ! StoreMe("from r1 2", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// r2, in order because we wrote them both in r1
|
||||
eventProbeR2.expectMessage(EventAndContext("from r1 1", ReplicaId("R1"), false, false))
|
||||
eventProbeR2.expectMessage(EventAndContext("from r1 2", ReplicaId("R1"), false, false))
|
||||
|
||||
r2 ! StoreMe("from r2 1", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
r2 ! StoreMe("from r2 2", probe.ref)
|
||||
probe.expectMessage(Done)
|
||||
|
||||
// r3 should only get the events 1, not R2s stored version of them, but we don't know the
|
||||
// order they will arrive
|
||||
val eventProbeR3 = createTestProbe[EventAndContext]()
|
||||
spawn(testBehavior(entityId, "R3", eventProbeR3.ref))
|
||||
val eventAndContexts = eventProbeR3.receiveMessages(4).toSet
|
||||
eventAndContexts should ===(
|
||||
Set(
|
||||
EventAndContext("from r1 1", ReplicaId("R1"), false, false),
|
||||
EventAndContext("from r1 2", ReplicaId("R1"), false, false),
|
||||
EventAndContext("from r2 1", ReplicaId("R2"), false, false),
|
||||
EventAndContext("from r2 2", ReplicaId("R2"), false, false)))
|
||||
eventProbeR3.expectNoMessage()
|
||||
}
|
||||
|
||||
"set concurrent on replay of events" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
val r2 = spawn(testBehavior(entityId, "R2"))
|
||||
r1 ! StoreMe("from r1", probe.ref) // R1 0 R2 0 -> R1 1 R2 0
|
||||
r2 ! StoreMe("from r2", probe.ref) // R2 0 R1 0 -> R2 1 R1 0
|
||||
// local event isn't concurrent, remote event is
|
||||
eventProbeR1.expectMessage(
|
||||
EventAndContext("from r1", ReplicaId("R1"), recoveryRunning = false, concurrent = false))
|
||||
eventProbeR1.expectMessage(
|
||||
EventAndContext("from r2", ReplicaId("R2"), recoveryRunning = false, concurrent = true))
|
||||
|
||||
// take 2
|
||||
val eventProbeR1Take2 = createTestProbe[EventAndContext]()
|
||||
spawn(testBehavior(entityId, "R1", eventProbeR1Take2.ref))
|
||||
eventProbeR1Take2.expectMessage(
|
||||
EventAndContext("from r1", ReplicaId("R1"), recoveryRunning = true, concurrent = false))
|
||||
eventProbeR1Take2.expectMessage(
|
||||
EventAndContext("from r2", ReplicaId("R2"), recoveryRunning = true, concurrent = true))
|
||||
}
|
||||
|
||||
"replicate events between three entities" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
var r1 = spawn(testBehavior(entityId, "R1"))
|
||||
var r2 = spawn(testBehavior(entityId, "R2"))
|
||||
var r3 = spawn(testBehavior(entityId, "R3"))
|
||||
r1 ! StoreMe("1 from r1", probe.ref)
|
||||
r2 ! StoreMe("1 from r2", probe.ref)
|
||||
r3 ! StoreMe("1 from r3", probe.ref)
|
||||
probe.receiveMessages(3) // all writes acked
|
||||
|
||||
(r1 :: r2 :: r3 :: Nil).foreach { replica =>
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
replica ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set("1 from r1", "1 from r2", "1 from r3")
|
||||
replica ! Stop
|
||||
probe.expectTerminated(replica)
|
||||
}
|
||||
}
|
||||
|
||||
// with all replicas stopped, start and write a bit to one of them
|
||||
r1 = spawn(testBehavior(entityId, "R1"))
|
||||
r1 ! StoreMe("2 from r1", probe.ref)
|
||||
r1 ! StoreMe("3 from r1", probe.ref)
|
||||
probe.receiveMessages(2) // both writes acked
|
||||
r1 ! Stop
|
||||
probe.expectTerminated(r1)
|
||||
|
||||
// start the other two
|
||||
r1 = spawn(testBehavior(entityId, "R1"))
|
||||
r2 = spawn(testBehavior(entityId, "R2"))
|
||||
r3 = spawn(testBehavior(entityId, "R3"))
|
||||
|
||||
(r1 :: r2 :: r3 :: Nil).foreach { replica =>
|
||||
eventually {
|
||||
val probe = createTestProbe[State]()
|
||||
replica ! GetState(probe.ref)
|
||||
probe.expectMessageType[State].all.toSet shouldEqual Set(
|
||||
"1 from r1",
|
||||
"2 from r1",
|
||||
"3 from r1",
|
||||
"1 from r2",
|
||||
"1 from r3")
|
||||
replica ! Stop
|
||||
probe.expectTerminated(replica)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"restart replication stream" in {
|
||||
val testkit = PersistenceTestKit(system)
|
||||
val entityId = nextEntityId
|
||||
val stateProbe = createTestProbe[State]()
|
||||
val probe = createTestProbe[Done]()
|
||||
val eventProbeR1 = createTestProbe[EventAndContext]()
|
||||
val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref))
|
||||
val r2 = spawn(testBehavior(entityId, "R2"))
|
||||
|
||||
// ensure recovery is complete
|
||||
r1 ! GetState(stateProbe.ref)
|
||||
stateProbe.expectMessage(State(Nil))
|
||||
r2 ! GetState(stateProbe.ref)
|
||||
stateProbe.expectMessage(State(Nil))
|
||||
|
||||
// make reads fail for the replication
|
||||
testkit.failNextNReads(s"$entityId|R2", 1)
|
||||
|
||||
// should restart the replication stream
|
||||
r2 ! StoreMe("from r2", probe.ref)
|
||||
eventProbeR1.expectMessageType[EventAndContext].event shouldEqual "from r2"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.persistence.query.NoOffset
|
||||
import akka.persistence.query.scaladsl.CurrentEventsByTagQuery
|
||||
import akka.persistence.query.PersistenceQuery
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicatedEventSourcingTaggingSpec {
|
||||
|
||||
val ReplicaId1 = ReplicaId("R1")
|
||||
val ReplicaId2 = ReplicaId("R2")
|
||||
val AllReplicas = Set(ReplicaId1, ReplicaId2)
|
||||
val queryPluginId = PersistenceTestKitReadJournal.Identifier
|
||||
|
||||
object ReplicatedStringSet {
|
||||
|
||||
sealed trait Command
|
||||
case class Add(description: String, replyTo: ActorRef[Done]) extends Command
|
||||
case class GetStrings(replyTo: ActorRef[Set[String]]) extends Command
|
||||
case class State(strings: Set[String]) extends CborSerializable
|
||||
|
||||
def apply(
|
||||
entityId: String,
|
||||
replica: ReplicaId,
|
||||
allReplicas: Set[ReplicaId]): EventSourcedBehavior[Command, String, State] = {
|
||||
// #tagging
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("TaggingSpec", entityId, replica),
|
||||
allReplicas,
|
||||
queryPluginId)(
|
||||
replicationContext =>
|
||||
EventSourcedBehavior[Command, String, State](
|
||||
replicationContext.persistenceId,
|
||||
State(Set.empty),
|
||||
(state, command) =>
|
||||
command match {
|
||||
case Add(string, ack) =>
|
||||
if (state.strings.contains(string)) Effect.none.thenRun(_ => ack ! Done)
|
||||
else Effect.persist(string).thenRun(_ => ack ! Done)
|
||||
case GetStrings(replyTo) =>
|
||||
replyTo ! state.strings
|
||||
Effect.none
|
||||
},
|
||||
(state, event) => state.copy(strings = state.strings + event))
|
||||
// use withTagger to define tagging logic
|
||||
.withTagger(
|
||||
event =>
|
||||
// don't apply tags if event was replicated here, it already will appear in queries by tag
|
||||
// as the origin replica would have tagged it already
|
||||
if (replicationContext.origin != replicationContext.replicaId) Set.empty
|
||||
else if (event.length > 10) Set("long-strings", "strings")
|
||||
else Set("strings")))
|
||||
// #tagging
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class ReplicatedEventSourcingTaggingSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing
|
||||
with Eventually {
|
||||
import ReplicatedEventSourcingTaggingSpec._
|
||||
val ids = new AtomicInteger(0)
|
||||
def nextEntityId = s"e-${ids.getAndIncrement()}"
|
||||
"ReplicatedEventSourcing" should {
|
||||
"allow for tagging of events using the replication context" in {
|
||||
val entityId = nextEntityId
|
||||
val probe = createTestProbe[Done]()
|
||||
val r1 = spawn(ReplicatedStringSet(entityId, ReplicaId1, AllReplicas))
|
||||
val r2 = spawn(ReplicatedStringSet(entityId, ReplicaId2, AllReplicas))
|
||||
r1 ! ReplicatedStringSet.Add("from r1", probe.ref)
|
||||
r2 ! ReplicatedStringSet.Add("from r2", probe.ref)
|
||||
probe.receiveMessages(2)
|
||||
r1 ! ReplicatedStringSet.Add("a very long string from r1", probe.ref)
|
||||
probe.receiveMessages(1)
|
||||
|
||||
val allEvents = Set("from r1", "from r2", "a very long string from r1")
|
||||
for (replica <- r1 :: r2 :: Nil) {
|
||||
eventually {
|
||||
val probe = testKit.createTestProbe[Set[String]]()
|
||||
replica ! ReplicatedStringSet.GetStrings(probe.ref)
|
||||
probe.receiveMessage() should ===(allEvents)
|
||||
}
|
||||
}
|
||||
|
||||
val query =
|
||||
PersistenceQuery(system).readJournalFor[CurrentEventsByTagQuery](PersistenceTestKitReadJournal.Identifier)
|
||||
|
||||
val stringTaggedEvents = query.currentEventsByTag("strings", NoOffset).runWith(Sink.seq).futureValue
|
||||
stringTaggedEvents.map(_.event).toSet should equal(allEvents)
|
||||
|
||||
val longStrings = query.currentEventsByTag("long-strings", NoOffset).runWith(Sink.seq).futureValue
|
||||
longStrings should have size (1)
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit }
|
||||
import akka.persistence.testkit.{ PersistenceTestKitPlugin, PersistenceTestKitSnapshotPlugin }
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicationBaseSpec {
|
||||
val R1 = ReplicaId("R1")
|
||||
val R2 = ReplicaId("R2")
|
||||
val AllReplicas = Set(R1, R2)
|
||||
}
|
||||
|
||||
abstract class ReplicationBaseSpec
|
||||
extends ScalaTestWithActorTestKit(
|
||||
PersistenceTestKitPlugin.config.withFallback(PersistenceTestKitSnapshotPlugin.config))
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing
|
||||
with Eventually {
|
||||
|
||||
val ids = new AtomicInteger(0)
|
||||
def nextEntityId: String = s"e-${ids.getAndIncrement()}"
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,102 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit }
|
||||
import akka.actor.typed.{ ActorRef, Behavior }
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing }
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicationIllegalAccessSpec {
|
||||
|
||||
val R1 = ReplicaId("R1")
|
||||
val R2 = ReplicaId("R1")
|
||||
val AllReplicas = Set(R1, R2)
|
||||
|
||||
sealed trait Command
|
||||
case class AccessInCommandHandler(replyTo: ActorRef[Thrown]) extends Command
|
||||
case class AccessInPersistCallback(replyTo: ActorRef[Thrown]) extends Command
|
||||
|
||||
case class Thrown(exception: Option[Throwable])
|
||||
|
||||
case class State(all: List[String]) extends CborSerializable
|
||||
|
||||
def apply(entityId: String, replica: ReplicaId): Behavior[Command] = {
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("IllegalAccessSpec", entityId, replica),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier)(
|
||||
replicationContext =>
|
||||
EventSourcedBehavior[Command, String, State](
|
||||
replicationContext.persistenceId,
|
||||
State(Nil),
|
||||
(_, command) =>
|
||||
command match {
|
||||
case AccessInCommandHandler(replyTo) =>
|
||||
val exception = try {
|
||||
replicationContext.origin
|
||||
None
|
||||
} catch {
|
||||
case t: Throwable =>
|
||||
Some(t)
|
||||
}
|
||||
replyTo ! Thrown(exception)
|
||||
Effect.none
|
||||
case AccessInPersistCallback(replyTo) =>
|
||||
Effect.persist("cat").thenRun { _ =>
|
||||
val exception = try {
|
||||
replicationContext.concurrent
|
||||
None
|
||||
} catch {
|
||||
case t: Throwable =>
|
||||
Some(t)
|
||||
}
|
||||
replyTo ! Thrown(exception)
|
||||
}
|
||||
},
|
||||
(state, event) => state.copy(all = event :: state.all)))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class ReplicationIllegalAccessSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing
|
||||
with Eventually {
|
||||
import ReplicationIllegalAccessSpec._
|
||||
"ReplicatedEventSourcing" should {
|
||||
"detect illegal access to context in command handler" in {
|
||||
val probe = createTestProbe[Thrown]()
|
||||
val ref = spawn(ReplicationIllegalAccessSpec("id1", R1))
|
||||
ref ! AccessInCommandHandler(probe.ref)
|
||||
val thrown: Throwable = probe.expectMessageType[Thrown].exception.get
|
||||
thrown.getMessage should include("from the event handler")
|
||||
}
|
||||
"detect illegal access to context in persist thenRun" in {
|
||||
val probe = createTestProbe[Thrown]()
|
||||
val ref = spawn(ReplicationIllegalAccessSpec("id1", R1))
|
||||
ref ! AccessInPersistCallback(probe.ref)
|
||||
val thrown: Throwable = probe.expectMessageType[Thrown].exception.get
|
||||
thrown.getMessage should include("from the event handler")
|
||||
}
|
||||
"detect illegal access in the factory" in {
|
||||
val exception = intercept[UnsupportedOperationException] {
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("IllegalAccessSpec", "id2", R1),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
replicationContext.origin
|
||||
???
|
||||
}
|
||||
}
|
||||
exception.getMessage should include("from the event handler")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit }
|
||||
import akka.actor.typed.{ ActorRef, Behavior }
|
||||
import akka.persistence.testkit.{ PersistenceTestKitPlugin, PersistenceTestKitSnapshotPlugin }
|
||||
import akka.persistence.testkit.scaladsl.{ PersistenceTestKit, SnapshotTestKit }
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.internal.{ ReplicatedPublishedEventMetaData, VersionVector }
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicationSnapshotSpec {
|
||||
|
||||
import ReplicatedEventSourcingSpec._
|
||||
|
||||
val EntityType = "SnapshotSpec"
|
||||
|
||||
def behaviorWithSnapshotting(entityId: String, replicaId: ReplicaId): Behavior[Command] =
|
||||
behaviorWithSnapshotting(entityId, replicaId, None)
|
||||
|
||||
def behaviorWithSnapshotting(
|
||||
entityId: String,
|
||||
replicaId: ReplicaId,
|
||||
eventProbe: ActorRef[EventAndContext]): Behavior[Command] =
|
||||
behaviorWithSnapshotting(entityId, replicaId, Some(eventProbe))
|
||||
|
||||
def behaviorWithSnapshotting(
|
||||
entityId: String,
|
||||
replicaId: ReplicaId,
|
||||
probe: Option[ActorRef[EventAndContext]]): Behavior[Command] = {
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId(EntityType, entityId, replicaId),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier)(replicationContext =>
|
||||
eventSourcedBehavior(replicationContext, probe).snapshotWhen((_, _, sequenceNr) => sequenceNr % 2 == 0))
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicationSnapshotSpec
|
||||
extends ScalaTestWithActorTestKit(
|
||||
PersistenceTestKitPlugin.config.withFallback(PersistenceTestKitSnapshotPlugin.config))
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing
|
||||
with Eventually {
|
||||
import ReplicatedEventSourcingSpec._
|
||||
import ReplicationSnapshotSpec._
|
||||
|
||||
val ids = new AtomicInteger(0)
|
||||
def nextEntityId = s"e-${ids.getAndIncrement()}"
|
||||
|
||||
val snapshotTestKit = SnapshotTestKit(system)
|
||||
val persistenceTestKit = PersistenceTestKit(system)
|
||||
|
||||
val R1 = ReplicaId("R1")
|
||||
val R2 = ReplicaId("R2")
|
||||
|
||||
"ReplicatedEventSourcing" should {
|
||||
"recover state from snapshots" in {
|
||||
val entityId = nextEntityId
|
||||
val persistenceIdR1 = s"$EntityType|$entityId|R1"
|
||||
val persistenceIdR2 = s"$EntityType|$entityId|R2"
|
||||
val probe = createTestProbe[Done]()
|
||||
val r2EventProbe = createTestProbe[EventAndContext]()
|
||||
|
||||
{
|
||||
val r1 = spawn(behaviorWithSnapshotting(entityId, R1))
|
||||
val r2 = spawn(behaviorWithSnapshotting(entityId, R2, r2EventProbe.ref))
|
||||
r1 ! StoreMe("r1 1", probe.ref)
|
||||
r1 ! StoreMe("r1 2", probe.ref)
|
||||
r2EventProbe.expectMessageType[EventAndContext]
|
||||
r2EventProbe.expectMessageType[EventAndContext]
|
||||
|
||||
snapshotTestKit.expectNextPersisted(persistenceIdR1, State(List("r1 2", "r1 1")))
|
||||
snapshotTestKit.expectNextPersisted(persistenceIdR2, State(List("r1 2", "r1 1")))
|
||||
|
||||
r2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, entityId, R1).persistenceId,
|
||||
1L,
|
||||
"two-again",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(R1, VersionVector.empty)))
|
||||
|
||||
// r2 should now filter out that event if it receives it again
|
||||
r2EventProbe.expectNoMessage()
|
||||
}
|
||||
|
||||
// restart r2 from a snapshot, the event should still be filtered
|
||||
{
|
||||
val r2 = spawn(behaviorWithSnapshotting(entityId, R2, r2EventProbe.ref))
|
||||
r2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl(
|
||||
ReplicationId(EntityType, entityId, R1).persistenceId,
|
||||
1L,
|
||||
"two-again",
|
||||
System.currentTimeMillis(),
|
||||
Some(new ReplicatedPublishedEventMetaData(R1, VersionVector.empty)))
|
||||
r2EventProbe.expectNoMessage()
|
||||
|
||||
val stateProbe = createTestProbe[State]()
|
||||
r2 ! GetState(stateProbe.ref)
|
||||
stateProbe.expectMessage(State(List("r1 2", "r1 1")))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,132 @@
|
|||
/*
|
||||
* Copyright (C) 2017-2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.crdt.CounterSpec.PlainCounter.{ Decrement, Get, Increment }
|
||||
import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing }
|
||||
import akka.persistence.typed.{ ReplicaId, ReplicationBaseSpec }
|
||||
|
||||
object CounterSpec {
|
||||
|
||||
object PlainCounter {
|
||||
sealed trait Command
|
||||
case class Get(reply: ActorRef[Long]) extends Command
|
||||
case object Increment extends Command
|
||||
case object Decrement extends Command
|
||||
}
|
||||
|
||||
import ReplicationBaseSpec._
|
||||
|
||||
def apply(
|
||||
entityId: String,
|
||||
replicaId: ReplicaId,
|
||||
snapshotEvery: Long = 100,
|
||||
eventProbe: Option[ActorRef[Counter.Updated]] = None) =
|
||||
Behaviors.setup[PlainCounter.Command] { context =>
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("CounterSpec", entityId, replicaId),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { ctx =>
|
||||
EventSourcedBehavior[PlainCounter.Command, Counter.Updated, Counter](
|
||||
ctx.persistenceId,
|
||||
Counter.empty,
|
||||
(state, command) =>
|
||||
command match {
|
||||
case PlainCounter.Increment =>
|
||||
context.log.info("Increment. Current state {}", state.value)
|
||||
Effect.persist(Counter.Updated(1))
|
||||
case PlainCounter.Decrement =>
|
||||
Effect.persist(Counter.Updated(-1))
|
||||
case Get(replyTo) =>
|
||||
context.log.info("Get request. {} {}", state.value, state.value.longValue)
|
||||
replyTo ! state.value.longValue
|
||||
Effect.none
|
||||
},
|
||||
(counter, event) => {
|
||||
eventProbe.foreach(_ ! event)
|
||||
counter.applyOperation(event)
|
||||
}).snapshotWhen { (_, _, seqNr) =>
|
||||
seqNr % snapshotEvery == 0
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class CounterSpec extends ReplicationBaseSpec {
|
||||
|
||||
import CounterSpec._
|
||||
import ReplicationBaseSpec._
|
||||
|
||||
"Replicated entity using CRDT counter" should {
|
||||
"replicate" in {
|
||||
val id = nextEntityId
|
||||
val r1 = spawn(apply(id, R1))
|
||||
val r2 = spawn(apply(id, R2))
|
||||
val r1Probe = createTestProbe[Long]()
|
||||
val r2Probe = createTestProbe[Long]()
|
||||
|
||||
r1 ! Increment
|
||||
r1 ! Increment
|
||||
|
||||
eventually {
|
||||
r1 ! Get(r1Probe.ref)
|
||||
r1Probe.expectMessage(2L)
|
||||
r2 ! Get(r2Probe.ref)
|
||||
r2Probe.expectMessage(2L)
|
||||
}
|
||||
|
||||
for (n <- 1 to 10) {
|
||||
if (n % 2 == 0) r1 ! Increment
|
||||
else r1 ! Decrement
|
||||
}
|
||||
for (_ <- 1 to 10) {
|
||||
r2 ! Increment
|
||||
}
|
||||
|
||||
eventually {
|
||||
r1 ! Get(r1Probe.ref)
|
||||
r1Probe.expectMessage(12L)
|
||||
r2 ! Get(r2Probe.ref)
|
||||
r2Probe.expectMessage(12L)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"recover from snapshot" in {
|
||||
val id = nextEntityId
|
||||
|
||||
{
|
||||
val r1 = spawn(apply(id, R1, 2))
|
||||
val r2 = spawn(apply(id, R2, 2))
|
||||
val r1Probe = createTestProbe[Long]()
|
||||
val r2Probe = createTestProbe[Long]()
|
||||
|
||||
r1 ! Increment
|
||||
r1 ! Increment
|
||||
|
||||
eventually {
|
||||
r1 ! Get(r1Probe.ref)
|
||||
r1Probe.expectMessage(2L)
|
||||
r2 ! Get(r2Probe.ref)
|
||||
r2Probe.expectMessage(2L)
|
||||
}
|
||||
}
|
||||
{
|
||||
val r2EventProbe = createTestProbe[Counter.Updated]()
|
||||
val r2 = spawn(apply(id, R2, 2, Some(r2EventProbe.ref)))
|
||||
val r2Probe = createTestProbe[Long]()
|
||||
eventually {
|
||||
r2 ! Get(r2Probe.ref)
|
||||
r2Probe.expectMessage(2L)
|
||||
}
|
||||
|
||||
r2EventProbe.expectNoMessage()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
|
||||
import akka.actor.typed.{ ActorRef, Behavior }
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing }
|
||||
import akka.persistence.typed.{ ReplicaId, ReplicationBaseSpec }
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
|
||||
object LwwSpec {
|
||||
|
||||
import ReplicationBaseSpec._
|
||||
|
||||
sealed trait Command
|
||||
final case class Update(item: String, timestamp: Long, error: ActorRef[String]) extends Command
|
||||
final case class Get(replyTo: ActorRef[Registry]) extends Command
|
||||
|
||||
sealed trait Event extends CborSerializable
|
||||
final case class Changed(item: String, timestamp: LwwTime) extends Event
|
||||
|
||||
final case class Registry(item: String, updatedTimestamp: LwwTime) extends CborSerializable
|
||||
|
||||
object LwwRegistry {
|
||||
|
||||
def apply(entityId: String, replica: ReplicaId): Behavior[Command] = {
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("LwwRegistrySpec", entityId, replica),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, Event, Registry](
|
||||
replicationContext.persistenceId,
|
||||
Registry("", LwwTime(Long.MinValue, replicationContext.replicaId)),
|
||||
(state, command) =>
|
||||
command match {
|
||||
case Update(s, timestmap, error) =>
|
||||
if (s == "") {
|
||||
error ! "bad value"
|
||||
Effect.none
|
||||
} else {
|
||||
Effect.persist(Changed(s, state.updatedTimestamp.increase(timestmap, replicationContext.replicaId)))
|
||||
}
|
||||
case Get(replyTo) =>
|
||||
replyTo ! state
|
||||
Effect.none
|
||||
},
|
||||
(state, event) =>
|
||||
event match {
|
||||
case Changed(s, timestamp) =>
|
||||
if (timestamp.isAfter(state.updatedTimestamp)) Registry(s, timestamp)
|
||||
else state
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
class LwwSpec extends ReplicationBaseSpec {
|
||||
import LwwSpec._
|
||||
import ReplicationBaseSpec._
|
||||
|
||||
class Setup {
|
||||
val entityId = nextEntityId
|
||||
val r1 = spawn(LwwRegistry.apply(entityId, R1))
|
||||
val r2 = spawn(LwwRegistry.apply(entityId, R2))
|
||||
val r1Probe = createTestProbe[String]()
|
||||
val r2Probe = createTestProbe[String]()
|
||||
val r1GetProbe = createTestProbe[Registry]()
|
||||
val r2GetProbe = createTestProbe[Registry]()
|
||||
}
|
||||
|
||||
"Lww Replicated Event Sourced Behavior" should {
|
||||
"replicate a single event" in new Setup {
|
||||
r1 ! Update("a1", 1L, r1Probe.ref)
|
||||
eventually {
|
||||
val probe = createTestProbe[Registry]()
|
||||
r2 ! Get(probe.ref)
|
||||
probe.expectMessage(Registry("a1", LwwTime(1L, R1)))
|
||||
}
|
||||
}
|
||||
|
||||
"resolve conflict" in new Setup {
|
||||
r1 ! Update("a1", 1L, r1Probe.ref)
|
||||
r2 ! Update("b1", 2L, r2Probe.ref)
|
||||
eventually {
|
||||
r1 ! Get(r1GetProbe.ref)
|
||||
r2 ! Get(r2GetProbe.ref)
|
||||
r1GetProbe.expectMessage(Registry("b1", LwwTime(2L, R2)))
|
||||
r2GetProbe.expectMessage(Registry("b1", LwwTime(2L, R2)))
|
||||
}
|
||||
}
|
||||
|
||||
"have deterministic tiebreak when the same time" in new Setup {
|
||||
r1 ! Update("a1", 1L, r1Probe.ref)
|
||||
r2 ! Update("b1", 1L, r2Probe.ref)
|
||||
// R1 < R2
|
||||
eventually {
|
||||
r1 ! Get(r1GetProbe.ref)
|
||||
r2 ! Get(r2GetProbe.ref)
|
||||
r1GetProbe.expectMessage(Registry("a1", LwwTime(1L, R1)))
|
||||
r2GetProbe.expectMessage(Registry("a1", LwwTime(1L, R1)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,102 @@
|
|||
/*
|
||||
* Copyright (C) 2017-2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
|
||||
import akka.actor.typed.{ ActorRef, Behavior }
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing }
|
||||
import akka.persistence.typed.{ ReplicaId, ReplicationBaseSpec }
|
||||
import ORSetSpec.ORSetEntity._
|
||||
import akka.persistence.typed.ReplicationBaseSpec.{ R1, R2 }
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.crdt.ORSetSpec.ORSetEntity
|
||||
|
||||
import scala.util.Random
|
||||
|
||||
object ORSetSpec {
|
||||
|
||||
import ReplicationBaseSpec._
|
||||
|
||||
object ORSetEntity {
|
||||
sealed trait Command
|
||||
final case class Get(replyTo: ActorRef[Set[String]]) extends Command
|
||||
final case class Add(elem: String) extends Command
|
||||
final case class AddAll(elems: Set[String]) extends Command
|
||||
final case class Remove(elem: String) extends Command
|
||||
|
||||
def apply(entityId: String, replica: ReplicaId): Behavior[ORSetEntity.Command] = {
|
||||
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("ORSetSpec", entityId, replica),
|
||||
AllReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, ORSet.DeltaOp, ORSet[String]](
|
||||
replicationContext.persistenceId,
|
||||
ORSet(replica),
|
||||
(state, command) =>
|
||||
command match {
|
||||
case Add(elem) =>
|
||||
Effect.persist(state + elem)
|
||||
case AddAll(elems) =>
|
||||
Effect.persist(state.addAll(elems.toSet))
|
||||
case Remove(elem) =>
|
||||
Effect.persist(state - elem)
|
||||
case Get(replyTo) =>
|
||||
Effect.none.thenRun(state => replyTo ! state.elements)
|
||||
|
||||
},
|
||||
(state, operation) => state.applyOperation(operation))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class ORSetSpec extends ReplicationBaseSpec {
|
||||
|
||||
class Setup {
|
||||
val entityId = nextEntityId
|
||||
val r1 = spawn(ORSetEntity.apply(entityId, R1))
|
||||
val r2 = spawn(ORSetEntity.apply(entityId, R2))
|
||||
val r1GetProbe = createTestProbe[Set[String]]()
|
||||
val r2GetProbe = createTestProbe[Set[String]]()
|
||||
|
||||
def assertForAllReplicas(state: Set[String]): Unit = {
|
||||
eventually {
|
||||
r1 ! Get(r1GetProbe.ref)
|
||||
r1GetProbe.expectMessage(state)
|
||||
r2 ! Get(r2GetProbe.ref)
|
||||
r2GetProbe.expectMessage(state)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def randomDelay(): Unit = {
|
||||
// exercise different timing scenarios
|
||||
Thread.sleep(Random.nextInt(200).toLong)
|
||||
}
|
||||
|
||||
"ORSet Replicated Entity" should {
|
||||
|
||||
"support concurrent updates" in new Setup {
|
||||
r1 ! Add("a1")
|
||||
r2 ! Add("b1")
|
||||
assertForAllReplicas(Set("a1", "b1"))
|
||||
r2 ! Remove("b1")
|
||||
assertForAllReplicas(Set("a1"))
|
||||
r2 ! Add("b1")
|
||||
for (n <- 2 to 10) {
|
||||
r1 ! Add(s"a$n")
|
||||
if (n % 3 == 0)
|
||||
randomDelay()
|
||||
r2 ! Add(s"b$n")
|
||||
}
|
||||
r1 ! AddAll((11 to 13).map(n => s"a$n").toSet)
|
||||
r2 ! AddAll((11 to 13).map(n => s"b$n").toSet)
|
||||
val expected = (1 to 13).flatMap(n => List(s"a$n", s"b$n")).toSet
|
||||
assertForAllReplicas(expected)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.jackson
|
||||
|
||||
import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit, SerializationTestKit }
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.crdt.{ Counter, LwwTime, ORSet }
|
||||
import akka.persistence.typed.jackson.ReplicatedEventSourcingJacksonSpec.{ WithCounter, WithLwwTime, WithOrSet }
|
||||
import akka.serialization.jackson.{ AkkaSerializationDeserializer, AkkaSerializationSerializer, JsonSerializable }
|
||||
import com.fasterxml.jackson.databind.annotation.{ JsonDeserialize, JsonSerialize }
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object ReplicatedEventSourcingJacksonSpec {
|
||||
final case class WithLwwTime(lwwTime: LwwTime) extends JsonSerializable
|
||||
final case class WithOrSet(
|
||||
@JsonDeserialize(using = classOf[AkkaSerializationDeserializer])
|
||||
@JsonSerialize(using = classOf[AkkaSerializationSerializer])
|
||||
orSet: ORSet[String])
|
||||
extends JsonSerializable
|
||||
final case class WithCounter(
|
||||
@JsonDeserialize(using = classOf[AkkaSerializationDeserializer])
|
||||
@JsonSerialize(using = classOf[AkkaSerializationSerializer])
|
||||
counter: Counter)
|
||||
extends JsonSerializable
|
||||
|
||||
}
|
||||
|
||||
class ReplicatedEventSourcingJacksonSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing {
|
||||
|
||||
private val serializationTestkit = new SerializationTestKit(system)
|
||||
|
||||
"RES jackson" should {
|
||||
"serialize LwwTime" in {
|
||||
val obj = WithLwwTime(LwwTime(5, ReplicaId("A")))
|
||||
serializationTestkit.verifySerialization(obj)
|
||||
}
|
||||
"serialize ORSet" in {
|
||||
val emptyOrSet = WithOrSet(ORSet.empty[String](ReplicaId("A")))
|
||||
serializationTestkit.verifySerialization(emptyOrSet)
|
||||
}
|
||||
"serialize Counter" in {
|
||||
val counter = WithCounter(Counter.empty)
|
||||
serializationTestkit.verifySerialization(counter)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,337 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.akka.persistence.typed
|
||||
|
||||
import java.time.Instant
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import docs.akka.persistence.typed.ReplicatedAuctionExampleSpec.AuctionEntity
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.scaladsl.ActorContext
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.LoggerOps
|
||||
import akka.actor.typed.scaladsl.TimerScheduler
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.RecoveryCompleted
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.persistence.typed.scaladsl.ReplicationContext
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
|
||||
object ReplicatedAuctionExampleSpec {
|
||||
|
||||
//#setup
|
||||
object AuctionEntity {
|
||||
|
||||
//#setup
|
||||
|
||||
//#commands
|
||||
type MoneyAmount = Int
|
||||
|
||||
case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originReplica: ReplicaId)
|
||||
|
||||
sealed trait Command extends CborSerializable
|
||||
case object Finish extends Command // A timer needs to schedule this event at each replica
|
||||
final case class OfferBid(bidder: String, offer: MoneyAmount) extends Command
|
||||
final case class GetHighestBid(replyTo: ActorRef[Bid]) extends Command
|
||||
final case class IsClosed(replyTo: ActorRef[Boolean]) extends Command
|
||||
private final case object Close extends Command // Internal, should not be sent from the outside
|
||||
//#commands
|
||||
|
||||
//#events
|
||||
sealed trait Event extends CborSerializable
|
||||
final case class BidRegistered(bid: Bid) extends Event
|
||||
final case class AuctionFinished(atReplica: ReplicaId) extends Event
|
||||
final case class WinnerDecided(atReplica: ReplicaId, winningBid: Bid, highestCounterOffer: MoneyAmount)
|
||||
extends Event
|
||||
//#events
|
||||
|
||||
//#phase
|
||||
/**
|
||||
* The auction passes through several workflow phases.
|
||||
* First, in `Running` `OfferBid` commands are accepted.
|
||||
*
|
||||
* `AuctionEntity` instances in all DCs schedule a `Finish` command
|
||||
* at a given time. That persists the `AuctionFinished` event and the
|
||||
* phase is in `Closing` until the auction is finished in all DCs.
|
||||
*
|
||||
* When the auction has been finished no more `OfferBid` commands are accepted.
|
||||
*
|
||||
* The auction is also finished immediately if `AuctionFinished` event from another
|
||||
* DC is seen before the scheduled `Finish` command. In that way the auction is finished
|
||||
* as quickly as possible in all DCs even though there might be some clock skew.
|
||||
*
|
||||
* One DC is responsible for finally deciding the winner and publishing the result.
|
||||
* All events must be collected from all DC before that can happen.
|
||||
* When the responsible DC has seen all `AuctionFinished` events from other DCs
|
||||
* all other events have also been propagated and it can persist `WinnerDecided` and
|
||||
* the auction is finally `Closed`.
|
||||
*
|
||||
*/
|
||||
sealed trait AuctionPhase
|
||||
case object Running extends AuctionPhase
|
||||
final case class Closing(finishedAtReplica: Set[ReplicaId]) extends AuctionPhase
|
||||
case object Closed extends AuctionPhase
|
||||
//#phase
|
||||
|
||||
//#state
|
||||
case class AuctionState(phase: AuctionPhase, highestBid: Bid, highestCounterOffer: MoneyAmount)
|
||||
extends CborSerializable {
|
||||
|
||||
def applyEvent(event: Event): AuctionState =
|
||||
event match {
|
||||
case BidRegistered(b) =>
|
||||
if (isHigherBid(b, highestBid))
|
||||
withNewHighestBid(b)
|
||||
else
|
||||
withTooLowBid(b)
|
||||
case AuctionFinished(atDc) =>
|
||||
phase match {
|
||||
case Running =>
|
||||
copy(phase = Closing(Set(atDc)))
|
||||
case Closing(alreadyFinishedDcs) =>
|
||||
copy(phase = Closing(alreadyFinishedDcs + atDc))
|
||||
case _ =>
|
||||
this
|
||||
}
|
||||
case _: WinnerDecided =>
|
||||
copy(phase = Closed)
|
||||
}
|
||||
|
||||
def withNewHighestBid(bid: Bid): AuctionState = {
|
||||
require(phase != Closed)
|
||||
require(isHigherBid(bid, highestBid))
|
||||
copy(highestBid = bid, highestCounterOffer = highestBid.offer // keep last highest bid around
|
||||
)
|
||||
}
|
||||
|
||||
def withTooLowBid(bid: Bid): AuctionState = {
|
||||
require(phase != Closed)
|
||||
require(isHigherBid(highestBid, bid))
|
||||
copy(highestCounterOffer = highestCounterOffer.max(bid.offer)) // update highest counter offer
|
||||
}
|
||||
|
||||
def isHigherBid(first: Bid, second: Bid): Boolean =
|
||||
first.offer > second.offer ||
|
||||
(first.offer == second.offer && first.timestamp.isBefore(second.timestamp)) || // if equal, first one wins
|
||||
// If timestamps are equal, choose by dc where the offer was submitted
|
||||
// In real auctions, this last comparison should be deterministic but unpredictable, so that submitting to a
|
||||
// particular DC would not be an advantage.
|
||||
(first.offer == second.offer && first.timestamp.equals(second.timestamp) && first.originReplica.id
|
||||
.compareTo(second.originReplica.id) < 0)
|
||||
}
|
||||
//#state
|
||||
|
||||
//#setup
|
||||
def apply(
|
||||
replica: ReplicaId,
|
||||
name: String,
|
||||
initialBid: AuctionEntity.Bid, // the initial bid is basically the minimum price bidden at start time by the owner
|
||||
closingAt: Instant,
|
||||
responsibleForClosing: Boolean,
|
||||
allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup[Command] { ctx =>
|
||||
Behaviors.withTimers { timers =>
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("auction", name, replica),
|
||||
allReplicas,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationCtx =>
|
||||
new AuctionEntity(ctx, replicationCtx, timers, closingAt, responsibleForClosing, allReplicas)
|
||||
.behavior(initialBid)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class AuctionEntity(
|
||||
context: ActorContext[AuctionEntity.Command],
|
||||
replicationContext: ReplicationContext,
|
||||
timers: TimerScheduler[AuctionEntity.Command],
|
||||
closingAt: Instant,
|
||||
responsibleForClosing: Boolean,
|
||||
allReplicas: Set[ReplicaId]) {
|
||||
import AuctionEntity._
|
||||
|
||||
private def behavior(initialBid: AuctionEntity.Bid): EventSourcedBehavior[Command, Event, AuctionState] =
|
||||
EventSourcedBehavior(
|
||||
replicationContext.persistenceId,
|
||||
AuctionState(phase = Running, highestBid = initialBid, highestCounterOffer = initialBid.offer),
|
||||
commandHandler,
|
||||
eventHandler).receiveSignal {
|
||||
case (state, RecoveryCompleted) => recoveryCompleted(state)
|
||||
}
|
||||
|
||||
private def recoveryCompleted(state: AuctionState): Unit = {
|
||||
if (shouldClose(state))
|
||||
context.self ! Close
|
||||
|
||||
val millisUntilClosing = closingAt.toEpochMilli - replicationContext.currentTimeMillis()
|
||||
timers.startSingleTimer(Finish, millisUntilClosing.millis)
|
||||
}
|
||||
//#setup
|
||||
|
||||
//#command-handler
|
||||
def commandHandler(state: AuctionState, command: Command): Effect[Event, AuctionState] = {
|
||||
state.phase match {
|
||||
case Closing(_) | Closed =>
|
||||
command match {
|
||||
case GetHighestBid(replyTo) =>
|
||||
replyTo ! state.highestBid.copy(offer = state.highestCounterOffer) // TODO this is not as described
|
||||
Effect.none
|
||||
case IsClosed(replyTo) =>
|
||||
replyTo ! (state.phase == Closed)
|
||||
Effect.none
|
||||
case Finish =>
|
||||
context.log.info("Finish")
|
||||
Effect.persist(AuctionFinished(replicationContext.replicaId))
|
||||
case Close =>
|
||||
context.log.info("Close")
|
||||
require(shouldClose(state))
|
||||
// TODO send email (before or after persisting)
|
||||
Effect.persist(WinnerDecided(replicationContext.replicaId, state.highestBid, state.highestCounterOffer))
|
||||
case _: OfferBid =>
|
||||
// auction finished, no more bids accepted
|
||||
Effect.unhandled
|
||||
}
|
||||
case Running =>
|
||||
command match {
|
||||
case OfferBid(bidder, offer) =>
|
||||
Effect.persist(
|
||||
BidRegistered(
|
||||
Bid(
|
||||
bidder,
|
||||
offer,
|
||||
Instant.ofEpochMilli(replicationContext.currentTimeMillis()),
|
||||
replicationContext.replicaId)))
|
||||
case GetHighestBid(replyTo) =>
|
||||
replyTo ! state.highestBid
|
||||
Effect.none
|
||||
case Finish =>
|
||||
Effect.persist(AuctionFinished(replicationContext.replicaId))
|
||||
case Close =>
|
||||
context.log.warn("Premature close")
|
||||
// Close should only be triggered when we have already finished
|
||||
Effect.unhandled
|
||||
case IsClosed(replyTo) =>
|
||||
replyTo ! false
|
||||
Effect.none
|
||||
}
|
||||
}
|
||||
}
|
||||
//#command-handler
|
||||
|
||||
//#event-handler
|
||||
def eventHandler(state: AuctionState, event: Event): AuctionState = {
|
||||
|
||||
val newState = state.applyEvent(event)
|
||||
context.log.infoN("Applying event {}. New start {}", event, newState)
|
||||
if (!replicationContext.recoveryRunning) {
|
||||
eventTriggers(event, newState)
|
||||
}
|
||||
newState
|
||||
|
||||
}
|
||||
|
||||
//#event-handler
|
||||
|
||||
//#event-triggers
|
||||
private def eventTriggers(event: Event, newState: AuctionState): Unit = {
|
||||
event match {
|
||||
case finished: AuctionFinished =>
|
||||
newState.phase match {
|
||||
case Closing(alreadyFinishedAtDc) =>
|
||||
context.log.infoN(
|
||||
"AuctionFinished at {}, already finished at [{}]",
|
||||
finished.atReplica,
|
||||
alreadyFinishedAtDc.mkString(", "))
|
||||
if (alreadyFinishedAtDc(replicationContext.replicaId)) {
|
||||
if (shouldClose(newState)) context.self ! Close
|
||||
} else {
|
||||
context.log.info("Sending finish to self")
|
||||
context.self ! Finish
|
||||
}
|
||||
|
||||
case _ => // no trigger for this state
|
||||
}
|
||||
case _ => // no trigger for this event
|
||||
}
|
||||
}
|
||||
|
||||
private def shouldClose(state: AuctionState): Boolean = {
|
||||
responsibleForClosing && (state.phase match {
|
||||
case Closing(alreadyFinishedAtDc) =>
|
||||
val allDone = allReplicas.diff(alreadyFinishedAtDc).isEmpty
|
||||
if (!allDone) {
|
||||
context.log.info2(
|
||||
s"Not closing auction as not all DCs have reported finished. All DCs: {}. Reported finished {}",
|
||||
allReplicas,
|
||||
alreadyFinishedAtDc)
|
||||
}
|
||||
allDone
|
||||
case _ =>
|
||||
false
|
||||
})
|
||||
}
|
||||
//#event-triggers
|
||||
|
||||
//#setup
|
||||
}
|
||||
//#setup
|
||||
}
|
||||
|
||||
class ReplicatedAuctionExampleSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
import ReplicatedAuctionExampleSpec.AuctionEntity._
|
||||
|
||||
"Auction example" should {
|
||||
|
||||
"work" in {
|
||||
val Replicas = Set(ReplicaId("DC-A"), ReplicaId("DC-B"))
|
||||
val auctionName = "old-skis"
|
||||
val initialBid = Bid("chbatey", 12, Instant.now(), ReplicaId("DC-A"))
|
||||
val closingAt = Instant.now().plusSeconds(10)
|
||||
|
||||
val dcAReplica: ActorRef[Command] = spawn(
|
||||
AuctionEntity(ReplicaId("DC-A"), auctionName, initialBid, closingAt, responsibleForClosing = true, Replicas))
|
||||
val dcBReplica: ActorRef[Command] = spawn(
|
||||
AuctionEntity(ReplicaId("DC-B"), auctionName, initialBid, closingAt, responsibleForClosing = false, Replicas))
|
||||
|
||||
dcAReplica ! OfferBid("me", 100)
|
||||
dcAReplica ! OfferBid("me", 99)
|
||||
dcAReplica ! OfferBid("me", 202)
|
||||
|
||||
eventually {
|
||||
val replyProbe = createTestProbe[Bid]()
|
||||
dcAReplica ! GetHighestBid(replyProbe.ref)
|
||||
val bid = replyProbe.expectMessageType[Bid]
|
||||
bid.offer shouldEqual 202
|
||||
}
|
||||
|
||||
dcAReplica ! Finish
|
||||
eventually {
|
||||
val finishProbe = createTestProbe[Boolean]()
|
||||
dcAReplica ! IsClosed(finishProbe.ref)
|
||||
finishProbe.expectMessage(true)
|
||||
}
|
||||
eventually {
|
||||
val finishProbe = createTestProbe[Boolean]()
|
||||
dcBReplica ! IsClosed(finishProbe.ref)
|
||||
finishProbe.expectMessage(true)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,170 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.akka.persistence.typed
|
||||
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.scaladsl.ActorContext
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.crdt.LwwTime
|
||||
import akka.persistence.typed.scaladsl._
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
|
||||
object ReplicatedBlogExampleSpec {
|
||||
|
||||
object BlogEntity {
|
||||
|
||||
object BlogState {
|
||||
val empty: BlogState = BlogState(None, LwwTime(Long.MinValue, ReplicaId("")), published = false)
|
||||
}
|
||||
final case class BlogState(content: Option[PostContent], contentTimestamp: LwwTime, published: Boolean)
|
||||
extends CborSerializable {
|
||||
def withContent(newContent: PostContent, timestamp: LwwTime): BlogState =
|
||||
copy(content = Some(newContent), contentTimestamp = timestamp)
|
||||
|
||||
def isEmpty: Boolean = content.isEmpty
|
||||
}
|
||||
|
||||
final case class PostContent(title: String, body: String) extends CborSerializable
|
||||
final case class Published(postId: String) extends Event
|
||||
|
||||
sealed trait Command extends CborSerializable
|
||||
final case class AddPost(postId: String, content: PostContent, replyTo: ActorRef[AddPostDone]) extends Command
|
||||
final case class AddPostDone(postId: String)
|
||||
final case class GetPost(postId: String, replyTo: ActorRef[PostContent]) extends Command
|
||||
final case class ChangeBody(postId: String, newContent: PostContent, replyTo: ActorRef[Done]) extends Command
|
||||
final case class Publish(postId: String, replyTo: ActorRef[Done]) extends Command
|
||||
|
||||
sealed trait Event extends CborSerializable
|
||||
final case class PostAdded(postId: String, content: PostContent, timestamp: LwwTime) extends Event
|
||||
final case class BodyChanged(postId: String, newContent: PostContent, timestamp: LwwTime) extends Event
|
||||
|
||||
def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = {
|
||||
Behaviors.setup[Command] { ctx =>
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("blog", entityId, replicaId),
|
||||
allReplicaIds,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, Event, BlogState](
|
||||
replicationContext.persistenceId,
|
||||
BlogState.empty,
|
||||
(state, cmd) => commandHandler(ctx, replicationContext, state, cmd),
|
||||
(state, event) => eventHandler(ctx, replicationContext, state, event))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#command-handler
|
||||
private def commandHandler(
|
||||
ctx: ActorContext[Command],
|
||||
replicationContext: ReplicationContext,
|
||||
state: BlogState,
|
||||
cmd: Command): Effect[Event, BlogState] = {
|
||||
cmd match {
|
||||
case AddPost(_, content, replyTo) =>
|
||||
val evt =
|
||||
PostAdded(
|
||||
replicationContext.entityId,
|
||||
content,
|
||||
state.contentTimestamp.increase(replicationContext.currentTimeMillis(), replicationContext.replicaId))
|
||||
Effect.persist(evt).thenRun { _ =>
|
||||
replyTo ! AddPostDone(replicationContext.entityId)
|
||||
}
|
||||
case ChangeBody(_, newContent, replyTo) =>
|
||||
val evt =
|
||||
BodyChanged(
|
||||
replicationContext.entityId,
|
||||
newContent,
|
||||
state.contentTimestamp.increase(replicationContext.currentTimeMillis(), replicationContext.replicaId))
|
||||
Effect.persist(evt).thenRun { _ =>
|
||||
replyTo ! Done
|
||||
}
|
||||
case p: Publish =>
|
||||
Effect.persist(Published("id")).thenRun { _ =>
|
||||
p.replyTo ! Done
|
||||
}
|
||||
case gp: GetPost =>
|
||||
ctx.log.info("GetPost {}", state.content)
|
||||
state.content.foreach(content => gp.replyTo ! content)
|
||||
Effect.none
|
||||
}
|
||||
}
|
||||
//#command-handler
|
||||
|
||||
//#event-handler
|
||||
private def eventHandler(
|
||||
ctx: ActorContext[Command],
|
||||
replicationContext: ReplicationContext,
|
||||
state: BlogState,
|
||||
event: Event): BlogState = {
|
||||
ctx.log.info(s"${replicationContext.entityId}:${replicationContext.replicaId} Received event $event")
|
||||
event match {
|
||||
case PostAdded(_, content, timestamp) =>
|
||||
if (timestamp.isAfter(state.contentTimestamp)) {
|
||||
val s = state.withContent(content, timestamp)
|
||||
ctx.log.info("Updating content. New content is {}", s)
|
||||
s
|
||||
} else {
|
||||
ctx.log.info("Ignoring event as timestamp is older")
|
||||
state
|
||||
}
|
||||
case BodyChanged(_, newContent, timestamp) =>
|
||||
if (timestamp.isAfter(state.contentTimestamp))
|
||||
state.withContent(newContent, timestamp)
|
||||
else state
|
||||
case Published(_) =>
|
||||
state.copy(published = true)
|
||||
}
|
||||
}
|
||||
//#event-handler
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicatedBlogExampleSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
import ReplicatedBlogExampleSpec.BlogEntity
|
||||
import ReplicatedBlogExampleSpec.BlogEntity._
|
||||
|
||||
"Blog Example" should {
|
||||
"work" in {
|
||||
val refDcA: ActorRef[Command] =
|
||||
spawn(BlogEntity("cat", ReplicaId("DC-A"), Set(ReplicaId("DC-A"), ReplicaId("DC-B"))))
|
||||
|
||||
val refDcB: ActorRef[Command] =
|
||||
spawn(BlogEntity("cat", ReplicaId("DC-B"), Set(ReplicaId("DC-A"), ReplicaId("DC-B"))))
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.actor.typed.scaladsl.AskPattern._
|
||||
import akka.util.Timeout
|
||||
implicit val timeout: Timeout = 3.seconds
|
||||
|
||||
val content = PostContent("cats are the bets", "yep")
|
||||
val response =
|
||||
refDcA.ask[AddPostDone](replyTo => AddPost("cat", content, replyTo)).futureValue
|
||||
|
||||
response shouldEqual AddPostDone("cat")
|
||||
|
||||
eventually {
|
||||
refDcA.ask[PostContent](replyTo => GetPost("cat", replyTo)).futureValue shouldEqual content
|
||||
}
|
||||
|
||||
eventually {
|
||||
refDcB.ask[PostContent](replyTo => GetPost("cat", replyTo)).futureValue shouldEqual content
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.akka.persistence.typed
|
||||
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, ReplicatedEventSourcing }
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent("never used")
|
||||
object ReplicatedEventSourcingCompileOnlySpec {
|
||||
|
||||
//#replicas
|
||||
val DCA = ReplicaId("DC-A")
|
||||
val DCB = ReplicaId("DC-B")
|
||||
val AllReplicas = Set(DCA, DCB)
|
||||
//#replicas
|
||||
|
||||
val queryPluginId = ""
|
||||
|
||||
trait Command
|
||||
trait State
|
||||
trait Event
|
||||
|
||||
//#factory-shared
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("entityTypeHint", "entityId", DCA),
|
||||
AllReplicas,
|
||||
queryPluginId) { context =>
|
||||
EventSourcedBehavior[Command, State, Event](???, ???, ???, ???)
|
||||
}
|
||||
//#factory-shared
|
||||
|
||||
//#factory
|
||||
ReplicatedEventSourcing.perReplicaJournalConfig(
|
||||
ReplicationId("entityTypeHint", "entityId", DCA),
|
||||
Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context =>
|
||||
EventSourcedBehavior[Command, State, Event](???, ???, ???, ???)
|
||||
}
|
||||
//#factory
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,104 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.akka.persistence.typed
|
||||
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.crdt.ORSet
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
|
||||
object ReplicatedMovieWatchListExampleSpec {
|
||||
//#movie-entity
|
||||
object MovieWatchList {
|
||||
sealed trait Command
|
||||
final case class AddMovie(movieId: String) extends Command
|
||||
final case class RemoveMovie(movieId: String) extends Command
|
||||
final case class GetMovieList(replyTo: ActorRef[MovieList]) extends Command
|
||||
final case class MovieList(movieIds: Set[String])
|
||||
|
||||
def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = {
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("movies", entityId, replicaId),
|
||||
allReplicaIds,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, ORSet.DeltaOp, ORSet[String]](
|
||||
replicationContext.persistenceId,
|
||||
ORSet.empty(replicationContext.replicaId),
|
||||
(state, cmd) => commandHandler(state, cmd),
|
||||
(state, event) => eventHandler(state, event))
|
||||
}
|
||||
}
|
||||
|
||||
private def commandHandler(state: ORSet[String], cmd: Command): Effect[ORSet.DeltaOp, ORSet[String]] = {
|
||||
cmd match {
|
||||
case AddMovie(movieId) =>
|
||||
Effect.persist(state + movieId)
|
||||
case RemoveMovie(movieId) =>
|
||||
Effect.persist(state - movieId)
|
||||
case GetMovieList(replyTo) =>
|
||||
replyTo ! MovieList(state.elements)
|
||||
Effect.none
|
||||
}
|
||||
}
|
||||
|
||||
private def eventHandler(state: ORSet[String], event: ORSet.DeltaOp): ORSet[String] = {
|
||||
state.applyOperation(event)
|
||||
}
|
||||
|
||||
}
|
||||
//#movie-entity
|
||||
|
||||
}
|
||||
|
||||
class ReplicatedMovieWatchListExampleSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
import ReplicatedMovieWatchListExampleSpec._
|
||||
|
||||
"MovieWatchList" must {
|
||||
"demonstrate ORSet" in {
|
||||
import MovieWatchList._
|
||||
|
||||
val Replicas = Set(ReplicaId("DC-A"), ReplicaId("DC-B"))
|
||||
|
||||
val dcAReplica: ActorRef[Command] = spawn(MovieWatchList("mylist", ReplicaId("DC-A"), Replicas))
|
||||
val dcBReplica: ActorRef[Command] = spawn(MovieWatchList("mylist", ReplicaId("DC-B"), Replicas))
|
||||
|
||||
val probeA = createTestProbe[MovieList]()
|
||||
val probeB = createTestProbe[MovieList]()
|
||||
|
||||
dcAReplica ! AddMovie("movie-15")
|
||||
dcAReplica ! AddMovie("movie-17")
|
||||
dcBReplica ! AddMovie("movie-20")
|
||||
|
||||
eventually {
|
||||
dcAReplica ! GetMovieList(probeA.ref)
|
||||
probeA.expectMessage(MovieList(Set("movie-15", "movie-17", "movie-20")))
|
||||
dcBReplica ! GetMovieList(probeB.ref)
|
||||
probeB.expectMessage(MovieList(Set("movie-15", "movie-17", "movie-20")))
|
||||
}
|
||||
|
||||
dcBReplica ! RemoveMovie("movie-17")
|
||||
eventually {
|
||||
dcAReplica ! GetMovieList(probeA.ref)
|
||||
probeA.expectMessage(MovieList(Set("movie-15", "movie-20")))
|
||||
dcBReplica ! GetMovieList(probeB.ref)
|
||||
probeB.expectMessage(MovieList(Set("movie-15", "movie-20")))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,121 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package docs.akka.persistence.typed
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import docs.akka.persistence.typed.ReplicatedShoppingCartExampleSpec.ShoppingCart.CartItems
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.testkit.PersistenceTestKitPlugin
|
||||
import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.ReplicationId
|
||||
import akka.persistence.typed.crdt.Counter
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.EventSourcedBehavior
|
||||
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
|
||||
object ReplicatedShoppingCartExampleSpec {
|
||||
|
||||
//#shopping-cart
|
||||
object ShoppingCart {
|
||||
|
||||
type ProductId = String
|
||||
|
||||
sealed trait Command extends CborSerializable
|
||||
final case class AddItem(id: ProductId, count: Int) extends Command
|
||||
final case class RemoveItem(id: ProductId, count: Int) extends Command
|
||||
final case class GetCartItems(replyTo: ActorRef[CartItems]) extends Command
|
||||
final case class CartItems(items: Map[ProductId, Int]) extends CborSerializable
|
||||
|
||||
sealed trait Event extends CborSerializable
|
||||
final case class ItemUpdated(id: ProductId, update: Counter.Updated) extends Event
|
||||
|
||||
final case class State(items: Map[ProductId, Counter])
|
||||
|
||||
def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = {
|
||||
ReplicatedEventSourcing.commonJournalConfig(
|
||||
ReplicationId("blog", entityId, replicaId),
|
||||
allReplicaIds,
|
||||
PersistenceTestKitReadJournal.Identifier) { replicationContext =>
|
||||
EventSourcedBehavior[Command, Event, State](
|
||||
replicationContext.persistenceId,
|
||||
State(Map.empty),
|
||||
(state, cmd) => commandHandler(state, cmd),
|
||||
(state, event) => eventHandler(state, event))
|
||||
}
|
||||
}
|
||||
|
||||
private def commandHandler(state: State, cmd: Command): Effect[Event, State] = {
|
||||
cmd match {
|
||||
case AddItem(productId, count) =>
|
||||
Effect.persist(ItemUpdated(productId, Counter.Updated(count)))
|
||||
case RemoveItem(productId, count) =>
|
||||
Effect.persist(ItemUpdated(productId, Counter.Updated(-count)))
|
||||
case GetCartItems(replyTo) =>
|
||||
val items = state.items.collect {
|
||||
case (id, counter) if counter.value > 0 => id -> counter.value.toInt
|
||||
}
|
||||
replyTo ! CartItems(items)
|
||||
Effect.none
|
||||
}
|
||||
}
|
||||
|
||||
private def eventHandler(state: State, event: Event): State = {
|
||||
event match {
|
||||
case ItemUpdated(id, update) =>
|
||||
val newItems = state.items.get(id) match {
|
||||
case Some(counter) => state.items + (id -> counter.applyOperation(update))
|
||||
case None => state.items + (id -> Counter.empty.applyOperation(update))
|
||||
}
|
||||
State(newItems)
|
||||
}
|
||||
}
|
||||
}
|
||||
//#shopping-cart
|
||||
}
|
||||
|
||||
class ReplicatedShoppingCartExampleSpec
|
||||
extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config)
|
||||
with AnyWordSpecLike
|
||||
with LogCapturing {
|
||||
import ReplicatedShoppingCartExampleSpec.ShoppingCart
|
||||
|
||||
"Replicated shopping cart" should {
|
||||
"work" in {
|
||||
val cartId = UUID.randomUUID().toString
|
||||
|
||||
val refDcA: ActorRef[ShoppingCart.Command] =
|
||||
spawn(ShoppingCart(cartId, ReplicaId("DC-A"), Set(ReplicaId("DC-A"), ReplicaId("DC-B"))))
|
||||
|
||||
val refDcB: ActorRef[ShoppingCart.Command] =
|
||||
spawn(ShoppingCart(cartId, ReplicaId("DC-B"), Set(ReplicaId("DC-A"), ReplicaId("DC-B"))))
|
||||
|
||||
val fidgetSpinnerId = "T2912"
|
||||
val rubicsCubeId = "T1302"
|
||||
|
||||
refDcA ! ShoppingCart.AddItem(fidgetSpinnerId, 10)
|
||||
refDcB ! ShoppingCart.AddItem(rubicsCubeId, 10)
|
||||
refDcA ! ShoppingCart.AddItem(rubicsCubeId, 10)
|
||||
refDcA ! ShoppingCart.AddItem(fidgetSpinnerId, 10)
|
||||
refDcB ! ShoppingCart.AddItem(fidgetSpinnerId, 10)
|
||||
refDcA ! ShoppingCart.RemoveItem(fidgetSpinnerId, 10)
|
||||
refDcA ! ShoppingCart.AddItem(rubicsCubeId, 10)
|
||||
refDcB ! ShoppingCart.RemoveItem(rubicsCubeId, 10)
|
||||
|
||||
val replyProbe = createTestProbe[CartItems]()
|
||||
|
||||
eventually {
|
||||
refDcA ! ShoppingCart.GetCartItems(replyProbe.ref)
|
||||
replyProbe.expectMessage(CartItems(Map(fidgetSpinnerId -> 20, rubicsCubeId -> 20)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load diff
|
|
@ -0,0 +1,4 @@
|
|||
# Changes to internal/private/do not extend
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withReplication")
|
||||
ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.*")
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withEventPublishing")
|
||||
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* Copyright (C) 2017-2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
syntax = "proto2";
|
||||
|
||||
option java_package = "akka.persistence.typed.serialization";
|
||||
option optimize_for = SPEED;
|
||||
import "ContainerFormats.proto";
|
||||
|
||||
message Counter {
|
||||
required bytes value = 1;
|
||||
}
|
||||
|
||||
message CounterUpdate {
|
||||
required bytes delta = 1;
|
||||
}
|
||||
|
||||
message ORSet {
|
||||
required string originDc = 1;
|
||||
required VersionVector vvector = 2;
|
||||
repeated VersionVector dots = 3;
|
||||
repeated string stringElements = 4;
|
||||
repeated sint32 intElements = 5 [packed=true];
|
||||
repeated sint64 longElements = 6 [packed=true];
|
||||
repeated Payload otherElements = 7;
|
||||
}
|
||||
|
||||
message ORSetDeltaGroup {
|
||||
message Entry {
|
||||
required ORSetDeltaOp operation = 1;
|
||||
required ORSet underlying = 2;
|
||||
}
|
||||
|
||||
repeated Entry entries = 1;
|
||||
}
|
||||
|
||||
enum ORSetDeltaOp {
|
||||
Add = 0;
|
||||
Remove = 1;
|
||||
Full = 2;
|
||||
}
|
||||
|
||||
message VersionVector {
|
||||
message Entry {
|
||||
required string key = 1;
|
||||
required int64 version = 2;
|
||||
}
|
||||
repeated Entry entries = 1;
|
||||
}
|
||||
|
||||
message ReplicatedEventMetadata {
|
||||
required string originReplica = 1;
|
||||
required int64 originSequenceNr = 2;
|
||||
required VersionVector versionVector = 3;
|
||||
required bool concurrent = 4;
|
||||
}
|
||||
|
||||
message ReplicatedSnapshotMetadata {
|
||||
message Seen {
|
||||
required string replicaId = 1;
|
||||
required int64 sequenceNr = 2;
|
||||
}
|
||||
required VersionVector version = 1;
|
||||
repeated Seen seenPerReplica = 2;
|
||||
}
|
||||
|
||||
message ReplicatedPublishedEventMetaData {
|
||||
optional string replicaId = 1;
|
||||
optional VersionVector versionVector = 2;
|
||||
}
|
||||
|
||||
message PublishedEvent {
|
||||
optional string persistenceId = 1;
|
||||
optional int64 sequenceNr = 2;
|
||||
optional Payload payload = 3;
|
||||
optional int64 timestamp = 4;
|
||||
optional ReplicatedPublishedEventMetaData metadata = 5;
|
||||
}
|
||||
|
|
@ -1,3 +1,21 @@
|
|||
akka.actor {
|
||||
|
||||
serialization-identifiers."akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer" = 40
|
||||
|
||||
serializers.replicated-event-sourcing = "akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer"
|
||||
|
||||
serialization-bindings {
|
||||
"akka.persistence.typed.internal.VersionVector" = replicated-event-sourcing
|
||||
"akka.persistence.typed.crdt.Counter" = replicated-event-sourcing
|
||||
"akka.persistence.typed.crdt.Counter$Updated" = replicated-event-sourcing
|
||||
"akka.persistence.typed.crdt.ORSet" = replicated-event-sourcing
|
||||
"akka.persistence.typed.crdt.ORSet$DeltaOp" = replicated-event-sourcing
|
||||
"akka.persistence.typed.internal.ReplicatedEventMetadata" = replicated-event-sourcing
|
||||
"akka.persistence.typed.internal.ReplicatedSnapshotMetadata" = replicated-event-sourcing
|
||||
"akka.persistence.typed.internal.PublishedEventImpl" = replicated-event-sourcing
|
||||
}
|
||||
}
|
||||
|
||||
akka.persistence.typed {
|
||||
|
||||
# Persistent actors stash while recovering or persisting events,
|
||||
|
|
@ -21,7 +39,6 @@ akka.persistence.typed {
|
|||
# enables automatic DEBUG level logging of messages stashed automatically by an EventSourcedBehavior,
|
||||
# this may happen while it receives commands while it is recovering events or while it is persisting events
|
||||
log-stashing = off
|
||||
|
||||
}
|
||||
|
||||
akka.reliable-delivery {
|
||||
|
|
|
|||
|
|
@ -124,7 +124,6 @@ object PersistenceId {
|
|||
*/
|
||||
def ofUniqueId(id: String): PersistenceId =
|
||||
new PersistenceId(id)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
import java.util.Optional
|
||||
|
||||
import akka.annotation.DoNotInherit
|
||||
import akka.persistence.typed.internal.ReplicatedPublishedEventMetaData
|
||||
|
||||
/**
|
||||
* When using event publishing the events published to the system event stream will be in this form.
|
||||
*
|
||||
* Not for user extension
|
||||
*/
|
||||
@DoNotInherit
|
||||
trait PublishedEvent {
|
||||
|
||||
/** Scala API: When emitted from an Replicated Event Sourcing actor this will contain the replica id */
|
||||
def replicatedMetaData: Option[ReplicatedPublishedEventMetaData]
|
||||
|
||||
/** Java API: When emitted from an Replicated Event Sourcing actor this will contain the replica id */
|
||||
def getReplicatedMetaData: Optional[ReplicatedPublishedEventMetaData]
|
||||
|
||||
def persistenceId: PersistenceId
|
||||
def sequenceNumber: Long
|
||||
|
||||
/** User event */
|
||||
def event: Any
|
||||
def timestamp: Long
|
||||
def tags: Set[String]
|
||||
}
|
||||
|
|
@ -0,0 +1,10 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
/**
|
||||
* Identifies a replica in Replicated Event Sourcing, could be a datacenter name or a logical identifier.
|
||||
*/
|
||||
final case class ReplicaId(id: String)
|
||||
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed
|
||||
|
||||
object ReplicationId {
|
||||
private[akka] val Separator = "|"
|
||||
def fromString(id: String): ReplicationId = {
|
||||
val split = id.split("\\|")
|
||||
require(split.size == 3, s"invalid replication id $id")
|
||||
ReplicationId(split(0), split(1), ReplicaId(split(2)))
|
||||
}
|
||||
|
||||
/**
|
||||
* @param typeName The name of the entity type e.g. account, user. Made part of the persistence id so that entity ids don't need to be unique across different replicated entities
|
||||
* @param entityId The unique entity id
|
||||
* @param replicaId The unique identity for this entity. The underlying persistence id will include the replica.
|
||||
*/
|
||||
def apply(typeName: String, entityId: String, replicaId: ReplicaId): ReplicationId =
|
||||
new ReplicationId(typeName, entityId, replicaId)
|
||||
}
|
||||
|
||||
/**
|
||||
* @param typeName The name of the entity type e.g. account, user. Made part of the persistence id so that entity ids don't need to be unique across different replicated entities
|
||||
* @param entityId The unique entity id
|
||||
* @param replicaId The unique identity for this entity. The underlying persistence id will include the replica.
|
||||
*/
|
||||
final class ReplicationId(val typeName: String, val entityId: String, val replicaId: ReplicaId) {
|
||||
import ReplicationId._
|
||||
if (typeName.contains(Separator))
|
||||
throw new IllegalArgumentException(
|
||||
s"entityTypeHint [$typeName] contains [$Separator] which is a reserved character")
|
||||
|
||||
if (entityId.contains(Separator))
|
||||
throw new IllegalArgumentException(s"entityId [$entityId] contains [$Separator] which is a reserved character")
|
||||
|
||||
if (replicaId.id.contains(Separator))
|
||||
throw new IllegalArgumentException(
|
||||
s"replicaId [${replicaId.id}] contains [$Separator] which is a reserved character")
|
||||
|
||||
private val id: String = s"$typeName$Separator$entityId$Separator${replicaId.id}"
|
||||
|
||||
def persistenceId: PersistenceId = PersistenceId.ofUniqueId(id)
|
||||
|
||||
override def toString: String = s"ReplicationId($typeName, $entityId, $replicaId)"
|
||||
|
||||
def withReplica(newReplica: ReplicaId): ReplicationId = {
|
||||
new ReplicationId(typeName, entityId, newReplica)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
import akka.annotation.ApiMayChange
|
||||
|
||||
@ApiMayChange
|
||||
object Counter {
|
||||
val empty: Counter = Counter(0)
|
||||
|
||||
final case class Updated(delta: BigInt) {
|
||||
|
||||
/**
|
||||
* JAVA API
|
||||
*/
|
||||
def this(delta: java.math.BigInteger) = this(delta: BigInt)
|
||||
|
||||
/**
|
||||
* JAVA API
|
||||
*/
|
||||
def this(delta: Int) = this(delta: BigInt)
|
||||
}
|
||||
}
|
||||
|
||||
@ApiMayChange
|
||||
final case class Counter(value: BigInt) extends OpCrdt[Counter.Updated] {
|
||||
|
||||
override type T = Counter
|
||||
|
||||
override def applyOperation(event: Counter.Updated): Counter =
|
||||
copy(value = value + event.delta)
|
||||
}
|
||||
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
import akka.annotation.ApiMayChange
|
||||
import akka.persistence.typed.ReplicaId
|
||||
|
||||
/**
|
||||
* Utility class for comparing timestamp replica
|
||||
* identifier when implementing last-writer wins.
|
||||
*/
|
||||
@ApiMayChange
|
||||
final case class LwwTime(timestamp: Long, originReplica: ReplicaId) {
|
||||
|
||||
/**
|
||||
* Create a new `LwwTime` that has a `timestamp` that is
|
||||
* `max` of the given timestamp and previous timestamp + 1,
|
||||
* i.e. monotonically increasing.
|
||||
*/
|
||||
def increase(t: Long, replicaId: ReplicaId): LwwTime =
|
||||
LwwTime(math.max(timestamp + 1, t), replicaId)
|
||||
|
||||
/**
|
||||
* Compare this `LwwTime` with the `other`.
|
||||
* Greatest timestamp wins. If both timestamps are
|
||||
* equal the `dc` identifiers are compared and the
|
||||
* one sorted first in alphanumeric order wins.
|
||||
*/
|
||||
def isAfter(other: LwwTime): Boolean = {
|
||||
if (timestamp > other.timestamp) true
|
||||
else if (timestamp < other.timestamp) false
|
||||
else if (other.originReplica.id.compareTo(originReplica.id) > 0) true
|
||||
else false
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,501 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.util.HashCode
|
||||
import akka.annotation.{ ApiMayChange, InternalApi }
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.crdt.ORSet.DeltaOp
|
||||
import akka.persistence.typed.internal.{ ManyVersionVector, OneVersionVector, VersionVector }
|
||||
|
||||
@ApiMayChange
|
||||
object ORSet {
|
||||
def empty[A](originReplica: ReplicaId): ORSet[A] = new ORSet(originReplica.id, Map.empty, VersionVector.empty)
|
||||
def apply[A](originReplica: ReplicaId): ORSet[A] = empty(originReplica)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def create[A](originReplica: ReplicaId): ORSet[A] = empty(originReplica)
|
||||
|
||||
/**
|
||||
* Extract the [[ORSet#elements]].
|
||||
*/
|
||||
def unapply[A](s: ORSet[A]): Option[Set[A]] = Some(s.elements)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] type Dot = VersionVector
|
||||
|
||||
sealed trait DeltaOp {
|
||||
def merge(that: DeltaOp): DeltaOp
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] sealed abstract class AtomicDeltaOp[A] extends DeltaOp {
|
||||
def underlying: ORSet[A]
|
||||
}
|
||||
|
||||
/** INTERNAL API */
|
||||
@InternalApi private[akka] final case class AddDeltaOp[A](underlying: ORSet[A]) extends AtomicDeltaOp[A] {
|
||||
|
||||
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||
case AddDeltaOp(u) =>
|
||||
// Note that we only merge deltas originating from the same DC
|
||||
AddDeltaOp(
|
||||
new ORSet(
|
||||
underlying.originReplica,
|
||||
concatElementsMap(u.elementsMap.asInstanceOf[Map[A, Dot]]),
|
||||
underlying.vvector.merge(u.vvector)))
|
||||
case _: AtomicDeltaOp[A] => DeltaGroup(Vector(this, that))
|
||||
case DeltaGroup(ops) => DeltaGroup(this +: ops)
|
||||
}
|
||||
|
||||
private def concatElementsMap(thatMap: Map[A, Dot]): Map[A, Dot] = {
|
||||
if (thatMap.size == 1) {
|
||||
val head = thatMap.head
|
||||
underlying.elementsMap.updated(head._1, head._2)
|
||||
} else
|
||||
underlying.elementsMap ++ thatMap
|
||||
}
|
||||
}
|
||||
|
||||
/** INTERNAL API */
|
||||
@InternalApi private[akka] final case class RemoveDeltaOp[A](underlying: ORSet[A]) extends AtomicDeltaOp[A] {
|
||||
if (underlying.size != 1)
|
||||
throw new IllegalArgumentException(s"RemoveDeltaOp should contain one removed element, but was $underlying")
|
||||
|
||||
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||
case _: AtomicDeltaOp[A] => DeltaGroup(Vector(this, that)) // keep it simple for removals
|
||||
case DeltaGroup(ops) => DeltaGroup(this +: ops)
|
||||
}
|
||||
}
|
||||
|
||||
/** INTERNAL API: Used for `clear` but could be used for other cases also */
|
||||
@InternalApi private[akka] final case class FullStateDeltaOp[A](underlying: ORSet[A]) extends AtomicDeltaOp[A] {
|
||||
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||
case _: AtomicDeltaOp[A] => DeltaGroup(Vector(this, that))
|
||||
case DeltaGroup(ops) => DeltaGroup(this +: ops)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final case class DeltaGroup[A](ops: immutable.IndexedSeq[DeltaOp]) extends DeltaOp {
|
||||
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||
case thatAdd: AddDeltaOp[A] =>
|
||||
// merge AddDeltaOp into last AddDeltaOp in the group, if possible
|
||||
ops.last match {
|
||||
case thisAdd: AddDeltaOp[A] => DeltaGroup(ops.dropRight(1) :+ thisAdd.merge(thatAdd))
|
||||
case _ => DeltaGroup(ops :+ thatAdd)
|
||||
}
|
||||
case DeltaGroup(thatOps) => DeltaGroup(ops ++ thatOps)
|
||||
case _ => DeltaGroup(ops :+ that)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Subtract the `vvector` from the `dot`.
|
||||
* What this means is that any (dc, version) pair in
|
||||
* `dot` that is <= an entry in `vvector` is removed from `dot`.
|
||||
* Example [{a, 3}, {b, 2}, {d, 14}, {g, 22}] -
|
||||
* [{a, 4}, {b, 1}, {c, 1}, {d, 14}, {e, 5}, {f, 2}] =
|
||||
* [{b, 2}, {g, 22}]
|
||||
*/
|
||||
@InternalApi private[akka] def subtractDots(dot: Dot, vvector: VersionVector): Dot = {
|
||||
|
||||
@tailrec def dropDots(remaining: List[(String, Long)], acc: List[(String, Long)]): List[(String, Long)] =
|
||||
remaining match {
|
||||
case Nil => acc
|
||||
case (d @ (node, v1)) :: rest =>
|
||||
val v2 = vvector.versionAt(node)
|
||||
if (v2 >= v1)
|
||||
// dot is dominated by version vector, drop it
|
||||
dropDots(rest, acc)
|
||||
else
|
||||
dropDots(rest, d :: acc)
|
||||
}
|
||||
|
||||
if (dot.isEmpty)
|
||||
VersionVector.empty
|
||||
else {
|
||||
dot match {
|
||||
case OneVersionVector(node, v1) =>
|
||||
// if dot is dominated by version vector, drop it
|
||||
if (vvector.versionAt(node) >= v1) VersionVector.empty
|
||||
else dot
|
||||
|
||||
case ManyVersionVector(vs) =>
|
||||
val remaining = vs.toList
|
||||
val newDots = dropDots(remaining, Nil)
|
||||
VersionVector(newDots)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* @see [[ORSet#merge]]
|
||||
*/
|
||||
@InternalApi private[akka] def mergeCommonKeys[A](
|
||||
commonKeys: Set[A],
|
||||
lhs: ORSet[A],
|
||||
rhs: ORSet[A]): Map[A, ORSet.Dot] =
|
||||
mergeCommonKeys(commonKeys.iterator, lhs, rhs)
|
||||
|
||||
private def mergeCommonKeys[A](commonKeys: Iterator[A], lhs: ORSet[A], rhs: ORSet[A]): Map[A, ORSet.Dot] = {
|
||||
commonKeys.foldLeft(Map.empty[A, ORSet.Dot]) {
|
||||
case (acc, k) =>
|
||||
val lhsDots = lhs.elementsMap(k)
|
||||
val rhsDots = rhs.elementsMap(k)
|
||||
(lhsDots, rhsDots) match {
|
||||
case (OneVersionVector(n1, v1), OneVersionVector(n2, v2)) =>
|
||||
if (n1 == n2 && v1 == v2)
|
||||
// one single common dot
|
||||
acc.updated(k, lhsDots)
|
||||
else {
|
||||
// no common, lhsUniqueDots == lhsDots, rhsUniqueDots == rhsDots
|
||||
val lhsKeep = ORSet.subtractDots(lhsDots, rhs.vvector)
|
||||
val rhsKeep = ORSet.subtractDots(rhsDots, lhs.vvector)
|
||||
val merged = lhsKeep.merge(rhsKeep)
|
||||
// Perfectly possible that an item in both sets should be dropped
|
||||
if (merged.isEmpty) acc
|
||||
else acc.updated(k, merged)
|
||||
}
|
||||
case (ManyVersionVector(lhsVs), ManyVersionVector(rhsVs)) =>
|
||||
val commonDots = lhsVs.filter {
|
||||
case (thisDotNode, v) => rhsVs.get(thisDotNode).exists(_ == v)
|
||||
}
|
||||
val commonDotsKeys = commonDots.keys
|
||||
val lhsUniqueDots = lhsVs -- commonDotsKeys
|
||||
val rhsUniqueDots = rhsVs -- commonDotsKeys
|
||||
val lhsKeep = ORSet.subtractDots(VersionVector(lhsUniqueDots), rhs.vvector)
|
||||
val rhsKeep = ORSet.subtractDots(VersionVector(rhsUniqueDots), lhs.vvector)
|
||||
val merged = lhsKeep.merge(rhsKeep).merge(VersionVector(commonDots))
|
||||
// Perfectly possible that an item in both sets should be dropped
|
||||
if (merged.isEmpty) acc
|
||||
else acc.updated(k, merged)
|
||||
case (ManyVersionVector(lhsVs), OneVersionVector(n2, v2)) =>
|
||||
val commonDots = lhsVs.filter {
|
||||
case (n1, v1) => v1 == v2 && n1 == n2
|
||||
}
|
||||
val commonDotsKeys = commonDots.keys
|
||||
val lhsUniqueDots = lhsVs -- commonDotsKeys
|
||||
val rhsUnique = if (commonDotsKeys.isEmpty) rhsDots else VersionVector.empty
|
||||
val lhsKeep = ORSet.subtractDots(VersionVector(lhsUniqueDots), rhs.vvector)
|
||||
val rhsKeep = ORSet.subtractDots(rhsUnique, lhs.vvector)
|
||||
val merged = lhsKeep.merge(rhsKeep).merge(VersionVector(commonDots))
|
||||
// Perfectly possible that an item in both sets should be dropped
|
||||
if (merged.isEmpty) acc
|
||||
else acc.updated(k, merged)
|
||||
case (OneVersionVector(n1, v1), ManyVersionVector(rhsVs)) =>
|
||||
val commonDots = rhsVs.filter {
|
||||
case (n2, v2) => v1 == v2 && n1 == n2
|
||||
}
|
||||
val commonDotsKeys = commonDots.keys
|
||||
val lhsUnique = if (commonDotsKeys.isEmpty) lhsDots else VersionVector.empty
|
||||
val rhsUniqueDots = rhsVs -- commonDotsKeys
|
||||
val lhsKeep = ORSet.subtractDots(lhsUnique, rhs.vvector)
|
||||
val rhsKeep = ORSet.subtractDots(VersionVector(rhsUniqueDots), lhs.vvector)
|
||||
val merged = lhsKeep.merge(rhsKeep).merge(VersionVector(commonDots))
|
||||
// Perfectly possible that an item in both sets should be dropped
|
||||
if (merged.isEmpty) acc
|
||||
else acc.updated(k, merged)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* @see [[ORSet#merge]]
|
||||
*/
|
||||
@InternalApi private[akka] def mergeDisjointKeys[A](
|
||||
keys: Set[A],
|
||||
elementsMap: Map[A, ORSet.Dot],
|
||||
vvector: VersionVector,
|
||||
accumulator: Map[A, ORSet.Dot]): Map[A, ORSet.Dot] =
|
||||
mergeDisjointKeys(keys.iterator, elementsMap, vvector, accumulator)
|
||||
|
||||
private def mergeDisjointKeys[A](
|
||||
keys: Iterator[A],
|
||||
elementsMap: Map[A, ORSet.Dot],
|
||||
vvector: VersionVector,
|
||||
accumulator: Map[A, ORSet.Dot]): Map[A, ORSet.Dot] = {
|
||||
keys.foldLeft(accumulator) {
|
||||
case (acc, k) =>
|
||||
val dots = elementsMap(k)
|
||||
if (vvector > dots || vvector == dots)
|
||||
acc
|
||||
else {
|
||||
// Optimise the set of stored dots to include only those unseen
|
||||
val newDots = subtractDots(dots, vvector)
|
||||
acc.updated(k, newDots)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Implements a 'Observed Remove Set' operation based CRDT, also called a 'OR-Set'.
|
||||
* Elements can be added and removed any number of times. Concurrent add wins
|
||||
* over remove.
|
||||
*
|
||||
* It is not implemented as in the paper
|
||||
* <a href="http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf">A comprehensive study of Convergent and Commutative Replicated Data Types</a>.
|
||||
* This is more space efficient and doesn't accumulate garbage for removed elements.
|
||||
* It is described in the paper
|
||||
* <a href="https://hal.inria.fr/file/index/docid/738680/filename/RR-8083.pdf">An optimized conflict-free replicated set</a>
|
||||
* The implementation is inspired by the Riak DT <a href="https://github.com/basho/riak_dt/blob/develop/src/riak_dt_orswot.erl">
|
||||
* riak_dt_orswot</a>.
|
||||
*
|
||||
* The ORSet has a version vector that is incremented when an element is added to
|
||||
* the set. The `DC -> count` pair for that increment is stored against the
|
||||
* element as its "birth dot". Every time the element is re-added to the set,
|
||||
* its "birth dot" is updated to that of the `DC -> count` version vector entry
|
||||
* resulting from the add. When an element is removed, we simply drop it, no tombstones.
|
||||
*
|
||||
* When an element exists in replica A and not replica B, is it because A added
|
||||
* it and B has not yet seen that, or that B removed it and A has not yet seen that?
|
||||
* In this implementation we compare the `dot` of the present element to the version vector
|
||||
* in the Set it is absent from. If the element dot is not "seen" by the Set version vector,
|
||||
* that means the other set has yet to see this add, and the item is in the merged
|
||||
* Set. If the Set version vector dominates the dot, that means the other Set has removed this
|
||||
* element already, and the item is not in the merged Set.
|
||||
*
|
||||
* This class is immutable, i.e. "modifying" methods return a new instance.
|
||||
*/
|
||||
@ApiMayChange
|
||||
final class ORSet[A] private[akka] (
|
||||
val originReplica: String,
|
||||
private[akka] val elementsMap: Map[A, ORSet.Dot],
|
||||
private[akka] val vvector: VersionVector)
|
||||
extends OpCrdt[DeltaOp] {
|
||||
|
||||
type T = ORSet[A]
|
||||
type D = ORSet.DeltaOp
|
||||
|
||||
/**
|
||||
* Scala API
|
||||
*/
|
||||
def elements: Set[A] = elementsMap.keySet
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getElements(): java.util.Set[A] = {
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
elements.asJava
|
||||
}
|
||||
|
||||
def contains(a: A): Boolean = elementsMap.contains(a)
|
||||
|
||||
def isEmpty: Boolean = elementsMap.isEmpty
|
||||
|
||||
def size: Int = elementsMap.size
|
||||
|
||||
/**
|
||||
* Adds an element to the set
|
||||
*/
|
||||
def +(element: A): ORSet.DeltaOp = add(element)
|
||||
|
||||
/**
|
||||
* Adds an element to the set
|
||||
*/
|
||||
def add(element: A): ORSet.DeltaOp = {
|
||||
val newVvector = vvector + originReplica
|
||||
val newDot = VersionVector(originReplica, newVvector.versionAt(originReplica))
|
||||
ORSet.AddDeltaOp(new ORSet(originReplica, Map(element -> newDot), newDot))
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Add several elements to the set.
|
||||
* `elems` must not be empty.
|
||||
*/
|
||||
def addAll(elems: java.util.Set[A]): ORSet.DeltaOp = {
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
addAll(elems.asScala.toSet)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API: Add several elements to the set.
|
||||
* `elems` must not be empty.
|
||||
*/
|
||||
def addAll(elems: Set[A]): ORSet.DeltaOp = {
|
||||
if (elems.size == 0) throw new IllegalArgumentException("addAll elems must not be empty")
|
||||
else if (elems.size == 1) add(elems.head)
|
||||
else {
|
||||
val (first, rest) = elems.splitAt(1)
|
||||
val firstOp = add(first.head)
|
||||
val (mergedOps, _) = rest.foldLeft((firstOp, applyOperation(firstOp))) {
|
||||
case ((op, state), elem) =>
|
||||
val nextOp = state.add(elem)
|
||||
val mergedOp = op.merge(nextOp)
|
||||
(mergedOp, state.applyOperation(nextOp))
|
||||
}
|
||||
mergedOps
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes an element from the set.
|
||||
*/
|
||||
def -(element: A): ORSet.DeltaOp = remove(element)
|
||||
|
||||
/**
|
||||
* Removes an element from the set.
|
||||
*/
|
||||
def remove(element: A): ORSet.DeltaOp = {
|
||||
val deltaDot = VersionVector(originReplica, vvector.versionAt(originReplica))
|
||||
ORSet.RemoveDeltaOp(new ORSet(originReplica, Map(element -> deltaDot), vvector))
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Remove several elements from the set.
|
||||
* `elems` must not be empty.
|
||||
*/
|
||||
def removeAll(elems: java.util.Set[A]): ORSet.DeltaOp = {
|
||||
import akka.util.ccompat.JavaConverters._
|
||||
removeAll(elems.asScala.toSet)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API: Remove several elements from the set.
|
||||
* `elems` must not be empty.
|
||||
*/
|
||||
def removeAll(elems: Set[A]): ORSet.DeltaOp = {
|
||||
if (elems.size == 0) throw new IllegalArgumentException("removeAll elems must not be empty")
|
||||
else if (elems.size == 1) remove(elems.head)
|
||||
else {
|
||||
val (first, rest) = elems.splitAt(1)
|
||||
val firstOp = remove(first.head)
|
||||
val (mergedOps, _) = rest.foldLeft((firstOp, applyOperation(firstOp))) {
|
||||
case ((op, state), elem) =>
|
||||
val nextOp = state.remove(elem)
|
||||
val mergedOp = op.merge(nextOp)
|
||||
(mergedOp, state.applyOperation(nextOp))
|
||||
}
|
||||
mergedOps
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes all elements from the set, but keeps the history.
|
||||
* This has the same result as using [[#remove]] for each
|
||||
* element, but it is more efficient.
|
||||
*/
|
||||
def clear(): ORSet.DeltaOp = {
|
||||
val newFullState = new ORSet[A](originReplica, elementsMap = Map.empty, vvector)
|
||||
ORSet.FullStateDeltaOp(newFullState)
|
||||
}
|
||||
|
||||
/**
|
||||
* When element is in this Set but not in that Set:
|
||||
* Compare the "birth dot" of the present element to the version vector in the Set it is absent from.
|
||||
* If the element dot is not "seen" by other Set version vector, that means the other set has yet to
|
||||
* see this add, and the element is to be in the merged Set.
|
||||
* If the other Set version vector dominates the dot, that means the other Set has removed
|
||||
* the element already, and the element is not to be in the merged Set.
|
||||
*
|
||||
* When element in both this Set and in that Set:
|
||||
* Some dots may still need to be shed. If this Set has dots that the other Set does not have,
|
||||
* and the other Set version vector dominates those dots, then we need to drop those dots.
|
||||
* Keep only common dots, and dots that are not dominated by the other sides version vector
|
||||
*/
|
||||
private def merge(that: ORSet[A], addDeltaOp: Boolean): ORSet[A] = {
|
||||
if (this eq that) this
|
||||
else {
|
||||
val commonKeys =
|
||||
if (this.elementsMap.size < that.elementsMap.size)
|
||||
this.elementsMap.keysIterator.filter(that.elementsMap.contains)
|
||||
else
|
||||
that.elementsMap.keysIterator.filter(this.elementsMap.contains)
|
||||
val entries00 = ORSet.mergeCommonKeys(commonKeys, this, that)
|
||||
val entries0 =
|
||||
if (addDeltaOp)
|
||||
entries00 ++ this.elementsMap.filter { case (elem, _) => !that.elementsMap.contains(elem) } else {
|
||||
val thisUniqueKeys = this.elementsMap.keysIterator.filterNot(that.elementsMap.contains)
|
||||
ORSet.mergeDisjointKeys(thisUniqueKeys, this.elementsMap, that.vvector, entries00)
|
||||
}
|
||||
val thatUniqueKeys = that.elementsMap.keysIterator.filterNot(this.elementsMap.contains)
|
||||
val entries = ORSet.mergeDisjointKeys(thatUniqueKeys, that.elementsMap, this.vvector, entries0)
|
||||
val mergedVvector = this.vvector.merge(that.vvector)
|
||||
|
||||
new ORSet(originReplica, entries, mergedVvector)
|
||||
}
|
||||
}
|
||||
|
||||
override def applyOperation(thatDelta: ORSet.DeltaOp): ORSet[A] = {
|
||||
thatDelta match {
|
||||
case d: ORSet.AddDeltaOp[A] => merge(d.underlying, addDeltaOp = true)
|
||||
case d: ORSet.RemoveDeltaOp[A] => mergeRemoveDelta(d)
|
||||
case d: ORSet.FullStateDeltaOp[A] => merge(d.underlying, addDeltaOp = false)
|
||||
case ORSet.DeltaGroup(ops) =>
|
||||
ops.foldLeft(this) {
|
||||
case (acc, op: ORSet.AddDeltaOp[A]) => acc.merge(op.underlying, addDeltaOp = true)
|
||||
case (acc, op: ORSet.RemoveDeltaOp[A]) => acc.mergeRemoveDelta(op)
|
||||
case (acc, op: ORSet.FullStateDeltaOp[A]) => acc.merge(op.underlying, addDeltaOp = false)
|
||||
case (_, _: ORSet.DeltaGroup[A]) =>
|
||||
throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def mergeRemoveDelta(thatDelta: ORSet.RemoveDeltaOp[A]): ORSet[A] = {
|
||||
val that = thatDelta.underlying
|
||||
val (elem, thatDot) = that.elementsMap.head
|
||||
def deleteDots = that.vvector.versionsIterator
|
||||
def deleteDotsNodes = deleteDots.map { case (dotNode, _) => dotNode }
|
||||
val newElementsMap = {
|
||||
val thisDotOption = this.elementsMap.get(elem)
|
||||
val deleteDotsAreGreater = deleteDots.forall {
|
||||
case (dotNode, dotV) =>
|
||||
thisDotOption match {
|
||||
case Some(thisDot) => thisDot.versionAt(dotNode) <= dotV
|
||||
case None => false
|
||||
}
|
||||
}
|
||||
if (deleteDotsAreGreater) {
|
||||
thisDotOption match {
|
||||
case Some(thisDot) =>
|
||||
if (thisDot.versionsIterator.forall { case (thisDotNode, _) => deleteDotsNodes.contains(thisDotNode) })
|
||||
elementsMap - elem
|
||||
else elementsMap
|
||||
case None =>
|
||||
elementsMap
|
||||
}
|
||||
} else
|
||||
elementsMap
|
||||
}
|
||||
|
||||
val newVvector = vvector.merge(thatDot)
|
||||
new ORSet(originReplica, newElementsMap, newVvector)
|
||||
}
|
||||
|
||||
// this class cannot be a `case class` because we need different `unapply`
|
||||
|
||||
override def toString: String = s"OR$elements"
|
||||
|
||||
override def equals(o: Any): Boolean = o match {
|
||||
case other: ORSet[_] =>
|
||||
originReplica == other.originReplica && vvector == other.vvector && elementsMap == other.elementsMap
|
||||
case _ => false
|
||||
}
|
||||
|
||||
override def hashCode: Int = {
|
||||
var result = HashCode.SEED
|
||||
result = HashCode.hash(result, originReplica)
|
||||
result = HashCode.hash(result, elementsMap)
|
||||
result = HashCode.hash(result, vvector)
|
||||
result
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,15 @@
|
|||
/*
|
||||
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.typed.crdt
|
||||
|
||||
import akka.annotation.{ ApiMayChange, DoNotInherit }
|
||||
|
||||
@ApiMayChange
|
||||
@DoNotInherit
|
||||
trait OpCrdt[Operation] { self =>
|
||||
type T <: OpCrdt[Operation] { type T = self.T }
|
||||
|
||||
def applyOperation(op: Operation): T
|
||||
}
|
||||
|
|
@ -6,14 +6,13 @@ package akka.persistence.typed.internal
|
|||
|
||||
import scala.concurrent.ExecutionContext
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
import org.slf4j.{ Logger, MDC }
|
||||
|
||||
import akka.actor.{ ActorRef, Cancellable }
|
||||
import akka.actor.{ Cancellable, ActorRef => ClassicActorRef }
|
||||
import akka.actor.typed.Signal
|
||||
import akka.actor.typed.scaladsl.ActorContext
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence._
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.{ EventAdapter, PersistenceId, SnapshotAdapter }
|
||||
import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, RetentionCriteria }
|
||||
import akka.util.OptionVal
|
||||
|
|
@ -48,7 +47,9 @@ private[akka] final class BehaviorSetup[C, E, S](
|
|||
val retention: RetentionCriteria,
|
||||
var holdingRecoveryPermit: Boolean,
|
||||
val settings: EventSourcedSettings,
|
||||
val stashState: StashState) {
|
||||
val stashState: StashState,
|
||||
val replication: Option[ReplicationSetup],
|
||||
val publishEvents: Boolean) {
|
||||
|
||||
import BehaviorSetup._
|
||||
import InternalProtocol.RecoveryTickEvent
|
||||
|
|
@ -57,10 +58,12 @@ private[akka] final class BehaviorSetup[C, E, S](
|
|||
|
||||
val persistence: Persistence = Persistence(context.system.toClassic)
|
||||
|
||||
val journal: ActorRef = persistence.journalFor(settings.journalPluginId)
|
||||
val snapshotStore: ActorRef = persistence.snapshotStoreFor(settings.snapshotPluginId)
|
||||
val journal: ClassicActorRef = persistence.journalFor(settings.journalPluginId)
|
||||
val snapshotStore: ClassicActorRef = persistence.snapshotStoreFor(settings.snapshotPluginId)
|
||||
|
||||
def selfClassic: ActorRef = context.self.toClassic
|
||||
val replicaId: Option[ReplicaId] = replication.map(_.replicaId)
|
||||
|
||||
def selfClassic: ClassicActorRef = context.self.toClassic
|
||||
|
||||
private var mdcPhase = PersistenceMdc.Initializing
|
||||
def log: Logger = {
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
|
||||
package akka.persistence.typed.internal
|
||||
|
||||
import java.util.Optional
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
|
|
@ -24,6 +25,7 @@ import akka.persistence.JournalProtocol
|
|||
import akka.persistence.Recovery
|
||||
import akka.persistence.RecoveryPermitter
|
||||
import akka.persistence.SnapshotProtocol
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.typed.DeleteEventsCompleted
|
||||
import akka.persistence.typed.DeleteEventsFailed
|
||||
import akka.persistence.typed.DeleteSnapshotsCompleted
|
||||
|
|
@ -32,6 +34,8 @@ import akka.persistence.typed.DeletionTarget
|
|||
import akka.persistence.typed.EventAdapter
|
||||
import akka.persistence.typed.NoOpEventAdapter
|
||||
import akka.persistence.typed.PersistenceId
|
||||
import akka.persistence.typed.PublishedEvent
|
||||
import akka.persistence.typed.ReplicaId
|
||||
import akka.persistence.typed.SnapshotAdapter
|
||||
import akka.persistence.typed.SnapshotCompleted
|
||||
import akka.persistence.typed.SnapshotFailed
|
||||
|
|
@ -69,6 +73,11 @@ private[akka] object EventSourcedBehaviorImpl {
|
|||
*/
|
||||
final case class GetState[State](replyTo: ActorRef[State]) extends InternalProtocol
|
||||
|
||||
/**
|
||||
* Used to start the replication stream at the correct sequence number
|
||||
*/
|
||||
final case class GetSeenSequenceNr(replica: ReplicaId, replyTo: ActorRef[Long]) extends InternalProtocol
|
||||
|
||||
}
|
||||
|
||||
@InternalApi
|
||||
|
|
@ -87,7 +96,9 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
|||
recovery: Recovery = Recovery(),
|
||||
retention: RetentionCriteria = RetentionCriteria.disabled,
|
||||
supervisionStrategy: SupervisorStrategy = SupervisorStrategy.stop,
|
||||
override val signalHandler: PartialFunction[(State, Signal), Unit] = PartialFunction.empty)
|
||||
override val signalHandler: PartialFunction[(State, Signal), Unit] = PartialFunction.empty,
|
||||
replication: Option[ReplicationSetup] = None,
|
||||
publishEvents: Boolean = true)
|
||||
extends EventSourcedBehavior[Command, Event, State] {
|
||||
|
||||
import EventSourcedBehaviorImpl.WriterIdentity
|
||||
|
|
@ -150,7 +161,9 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
|||
retention,
|
||||
holdingRecoveryPermit = false,
|
||||
settings = settings,
|
||||
stashState = stashState)
|
||||
stashState = stashState,
|
||||
replication = replication,
|
||||
publishEvents = publishEvents)
|
||||
|
||||
// needs to accept Any since we also can get messages from the journal
|
||||
// not part of the user facing Command protocol
|
||||
|
|
@ -237,6 +250,16 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
|||
override def withRecovery(recovery: TypedRecovery): EventSourcedBehavior[Command, Event, State] = {
|
||||
copy(recovery = recovery.toClassic)
|
||||
}
|
||||
|
||||
override def withEventPublishing(enabled: Boolean): EventSourcedBehavior[Command, Event, State] = {
|
||||
copy(publishEvents = enabled)
|
||||
}
|
||||
|
||||
override private[akka] def withReplication(
|
||||
context: ReplicationContextImpl): EventSourcedBehavior[Command, Event, State] = {
|
||||
copy(
|
||||
replication = Some(ReplicationSetup(context.replicationId.replicaId, context.replicasAndQueryPlugins, context)))
|
||||
}
|
||||
}
|
||||
|
||||
/** Protocol used internally by the eventsourced behaviors. */
|
||||
|
|
@ -247,4 +270,92 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
|||
final case class SnapshotterResponse(msg: akka.persistence.SnapshotProtocol.Response) extends InternalProtocol
|
||||
final case class RecoveryTickEvent(snapshot: Boolean) extends InternalProtocol
|
||||
final case class IncomingCommand[C](c: C) extends InternalProtocol
|
||||
|
||||
final case class ReplicatedEventEnvelope[E](event: ReplicatedEvent[E], ack: ActorRef[ReplicatedEventAck.type])
|
||||
extends InternalProtocol
|
||||
|
||||
}
|
||||
|
||||
object ReplicatedEventMetadata {
|
||||
|
||||
/**
|
||||
* For a journal supporting Replicated Event Sourcing needing to add test coverage, use this instance as metadata and defer
|
||||
* to the built in serializer for serialization format
|
||||
*/
|
||||
@ApiMayChange
|
||||
def instanceForJournalTest: Any = ReplicatedEventMetadata(ReplicaId("DC-A"), 1L, VersionVector.empty + "DC-A", true)
|
||||
}
|
||||
|
||||
/**
|
||||
* @param originReplica Where the event originally was created
|
||||
* @param originSequenceNr The original sequenceNr in the origin DC
|
||||
* @param version The version with which the event was persisted at the different DC. The same event will have different version vectors
|
||||
* at each location as they are received at different times
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] final case class ReplicatedEventMetadata(
|
||||
originReplica: ReplicaId,
|
||||
originSequenceNr: Long,
|
||||
version: VersionVector,
|
||||
concurrent: Boolean) // whether when the event handler was executed the event was concurrent
|
||||
|
||||
object ReplicatedSnapshotMetadata {
|
||||
|
||||
/**
|
||||
* For a snapshot store supporting Replicated Event Sourcing needing to add test coverage, use this instance as metadata and defer
|
||||
* to the built in serializer for serialization format
|
||||
*/
|
||||
@ApiMayChange
|
||||
def instanceForSnapshotStoreTest: Any =
|
||||
ReplicatedSnapshotMetadata(
|
||||
VersionVector.empty + "DC-B" + "DC-A",
|
||||
Map(ReplicaId("DC-A") -> 1L, ReplicaId("DC-B") -> 1L))
|
||||
|
||||
}
|
||||
|
||||
@InternalApi
|
||||
private[akka] final case class ReplicatedSnapshotMetadata(version: VersionVector, seenPerReplica: Map[ReplicaId, Long])
|
||||
|
||||
/**
|
||||
* An event replicated from a different replica.
|
||||
*
|
||||
* The version is for when it was persisted at the other replica. At the current replica it will be
|
||||
* merged with the current local version.
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] final case class ReplicatedEvent[E](
|
||||
event: E,
|
||||
originReplica: ReplicaId,
|
||||
originSequenceNr: Long,
|
||||
originVersion: VersionVector)
|
||||
@InternalApi
|
||||
private[akka] case object ReplicatedEventAck
|
||||
|
||||
final class ReplicatedPublishedEventMetaData(val replicaId: ReplicaId, private[akka] val version: VersionVector)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi
|
||||
private[akka] final case class PublishedEventImpl(
|
||||
persistenceId: PersistenceId,
|
||||
sequenceNumber: Long,
|
||||
payload: Any,
|
||||
timestamp: Long,
|
||||
replicatedMetaData: Option[ReplicatedPublishedEventMetaData])
|
||||
extends PublishedEvent
|
||||
with InternalProtocol {
|
||||
import scala.compat.java8.OptionConverters._
|
||||
|
||||
def tags: Set[String] = payload match {
|
||||
case t: Tagged => t.tags
|
||||
case _ => Set.empty
|
||||
}
|
||||
|
||||
def event: Any = payload match {
|
||||
case Tagged(event, _) => event
|
||||
case _ => payload
|
||||
}
|
||||
|
||||
override def getReplicatedMetaData: Optional[ReplicatedPublishedEventMetaData] = replicatedMetaData.asJava
|
||||
}
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue